Merge branch 'master' into multi_auth_methods

This commit is contained in:
Arthur Passos 2024-08-13 10:30:01 -03:00
commit e8a40d9d52
169 changed files with 2187 additions and 751 deletions

View File

@ -99,10 +99,9 @@ upload_data() {
# iterating over globs will cause redundant file variable to be
# a path to a file, not a filename
# shellcheck disable=SC2045
for file in $(ls "${data_path}"); do
echo "${file}";
./mc cp "${data_path}"/"${file}" clickminio/test/"${file}";
done
if [ -d "${data_path}" ]; then
./mc cp --recursive "${data_path}"/ clickminio/test/
fi
}
setup_aws_credentials() {

View File

@ -14,7 +14,7 @@ Each functional test sends one or multiple queries to the running ClickHouse ser
Tests are located in `queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from ClickHouse and it is available to general public.
Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`.
Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`.
:::note
A common mistake when testing data types `DateTime` and `DateTime64` is assuming that the server uses a specific time zone (e.g. "UTC"). This is not the case, time zones in CI test runs
@ -38,7 +38,7 @@ For more options, see `tests/clickhouse-test --help`. You can simply run all tes
### Adding a New Test
To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client --multiquery < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`.
To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`.
Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables.

View File

@ -143,6 +143,18 @@ value can be specified at session, profile or query level using setting [query_c
Entries in the query cache are compressed by default. This reduces the overall memory consumption at the cost of slower writes into / reads
from the query cache. To disable compression, use setting [query_cache_compress_entries](settings/settings.md#query-cache-compress-entries).
Sometimes it is useful to keep multiple results for the same query cached. This can be achieved using setting
[query_cache_tag](settings/settings.md#query-cache-tag) that acts as as a label (or namespace) for a query cache entries. The query cache
considers results of the same query with different tags different.
Example for creating three different query cache entries for the same query:
```sql
SELECT 1 SETTINGS use_query_cache = true; -- query_cache_tag is implicitly '' (empty string)
SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'tag 1';
SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'tag 2';
```
ClickHouse reads table data in blocks of [max_block_size](settings/settings.md#setting-max_block_size) rows. Due to filtering, aggregation,
etc., result blocks are typically much smaller than 'max_block_size' but there are also cases where they are much bigger. Setting
[query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results) (enabled by default) controls if result blocks

View File

@ -1800,6 +1800,17 @@ Possible values:
Default value: `0`.
## query_cache_tag {#query-cache-tag}
A string which acts as a label for [query cache](../query-cache.md) entries.
The same queries with different tags are considered different by the query cache.
Possible values:
- Any string
Default value: `''`
## query_cache_max_size_in_bytes {#query-cache-max-size-in-bytes}
The maximum amount of memory (in bytes) the current user may allocate in the [query cache](../query-cache.md). 0 means unlimited.
@ -5627,6 +5638,12 @@ Disable all insert and mutations (alter table update / alter table delete / alte
Default value: `false`.
## use_hive_partitioning
When enabled, ClickHouse will detect Hive-style partitioning in path (`/name=value/`) in file-like table engines [File](../../engines/table-engines/special/file.md#hive-style-partitioning)/[S3](../../engines/table-engines/integrations/s3.md#hive-style-partitioning)/[URL](../../engines/table-engines/special/url.md#hive-style-partitioning)/[HDFS](../../engines/table-engines/integrations/hdfs.md#hive-style-partitioning)/[AzureBlobStorage](../../engines/table-engines/integrations/azureBlobStorage.md#hive-style-partitioning) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`.
Default value: `false`.
## allow_experimental_time_series_table {#allow-experimental-time-series-table}
Allows creation of tables with the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine.

View File

@ -9,6 +9,7 @@ Columns:
- `query` ([String](../../sql-reference/data-types/string.md)) — Query string.
- `result_size` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Size of the query cache entry.
- `tag` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — Tag of the query cache entry.
- `stale` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is stale.
- `shared` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is shared between multiple users.
- `compressed` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is compressed.
@ -26,6 +27,7 @@ Row 1:
──────
query: SELECT 1 SETTINGS use_query_cache = 1
result_size: 128
tag:
stale: 0
shared: 0
compressed: 1

View File

@ -4189,3 +4189,94 @@ Result:
│ 32 │
└─────────────────────────────┘
```
## getSubcolumn
Takes a table expression or identifier and constant string with the name of the sub-column, and returns the requested sub-column extracted from the expression.
**Syntax**
```sql
getSubcolumn(col_name, subcol_name)
```
**Arguments**
- `col_name` — Table expression or identifier. [Expression](../syntax.md/#expressions), [Identifier](../syntax.md/#identifiers).
- `subcol_name` — The name of the sub-column. [String](../data-types/string.md).
**Returned value**
- Returns the extracted sub-column.
**Example**
Query:
```sql
CREATE TABLE t_arr (arr Array(Tuple(subcolumn1 UInt32, subcolumn2 String))) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO t_arr VALUES ([(1, 'Hello'), (2, 'World')]), ([(3, 'This'), (4, 'is'), (5, 'subcolumn')]);
SELECT getSubcolumn(arr, 'subcolumn1'), getSubcolumn(arr, 'subcolumn2') FROM t_arr;
```
Result:
```response
┌─getSubcolumn(arr, 'subcolumn1')─┬─getSubcolumn(arr, 'subcolumn2')─┐
1. │ [1,2] │ ['Hello','World'] │
2. │ [3,4,5] │ ['This','is','subcolumn'] │
└─────────────────────────────────┴─────────────────────────────────┘
```
## getTypeSerializationStreams
Enumerates stream paths of a data type.
:::note
This function is intended for use by developers.
:::
**Syntax**
```sql
getTypeSerializationStreams(col)
```
**Arguments**
- `col` — Column or string representation of a data-type from which the data type will be detected.
**Returned value**
- Returns an array with all the serialization sub-stream paths.[Array](../data-types/array.md)([String](../data-types/string.md)).
**Examples**
Query:
```sql
SELECT getTypeSerializationStreams(tuple('a', 1, 'b', 2));
```
Result:
```response
┌─getTypeSerializationStreams(('a', 1, 'b', 2))─────────────────────────────────────────────────────────────────────────┐
1. │ ['{TupleElement(1), Regular}','{TupleElement(2), Regular}','{TupleElement(3), Regular}','{TupleElement(4), Regular}'] │
└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
```
Query:
```sql
SELECT getTypeSerializationStreams('Map(String, Int64)');
```
Result:
```response
┌─getTypeSerializationStreams('Map(String, Int64)')────────────────────────────────────────────────────────────────┐
1. │ ['{ArraySizes}','{ArrayElements, TupleElement(keys), Regular}','{ArrayElements, TupleElement(values), Regular}'] │
└──────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
```

View File

@ -8,26 +8,28 @@ sidebar_label: STATISTICS
The following operations are available:
- `ALTER TABLE [db].table ADD STATISTICS (columns list) TYPE (type list)` - Adds statistic description to tables metadata.
- `ALTER TABLE [db].table ADD STATISTICS [IF NOT EXISTS] (column list) TYPE (type list)` - Adds statistic description to tables metadata.
- `ALTER TABLE [db].table MODIFY STATISTICS (columns list) TYPE (type list)` - Modifies statistic description to tables metadata.
- `ALTER TABLE [db].table MODIFY STATISTICS (column list) TYPE (type list)` - Modifies statistic description to tables metadata.
- `ALTER TABLE [db].table DROP STATISTICS (columns list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns.
- `ALTER TABLE [db].table DROP STATISTICS [IF EXISTS] (column list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns.
- `ALTER TABLE [db].table CLEAR STATISTICS (columns list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`.
- `ALTER TABLE [db].table CLEAR STATISTICS [IF EXISTS] (column list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`.
- `ALTER TABLE [db.]table MATERIALIZE STATISTICS (columns list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
- `ALTER TABLE [db.]table MATERIALIZE STATISTICS [IF EXISTS] (column list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
The first two commands are lightweight in a sense that they only change metadata or remove files.
Also, they are replicated, syncing statistics metadata via ZooKeeper.
There is an example adding two statistics types to two columns:
## Example:
Adding two statistics types to two columns:
```
ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq;
```
:::note
Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants).
Statistic are supported only for [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine tables (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants).
:::

View File

@ -77,3 +77,16 @@ SELECT count(*) FROM azureBlobStorage('DefaultEndpointsProtocol=https;AccountNam
**See Also**
- [AzureBlobStorage Table Engine](/docs/en/engines/table-engines/integrations/azureBlobStorage.md)
## Hive-style partitioning {#hive-style-partitioning}
When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`.
**Example**
Use virtual column, created with Hive-style partitioning
``` sql
SET use_hive_partitioning = 1;
SELECT * from azureBlobStorage(config, storage_account_url='...', container='...', blob_path='http://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42;
```

View File

@ -206,6 +206,19 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3
- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`.
- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`.
## Hive-style partitioning {#hive-style-partitioning}
When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`.
**Example**
Use virtual column, created with Hive-style partitioning
``` sql
SET use_hive_partitioning = 1;
SELECT * from file('data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42;
```
## Settings {#settings}
- [engine_file_empty_if_not_exists](/docs/en/operations/settings/settings.md#engine-file-empty_if-not-exists) - allows to select empty data from a file that doesn't exist. Disabled by default.

View File

@ -100,6 +100,19 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin
- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`.
- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`.
## Hive-style partitioning {#hive-style-partitioning}
When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`.
**Example**
Use virtual column, created with Hive-style partitioning
``` sql
SET use_hive_partitioning = 1;
SELECT * from HDFS('hdfs://hdfs1:9000/data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42;
```
## Storage Settings {#storage-settings}
- [hdfs_truncate_on_insert](/docs/en/operations/settings/settings.md#hdfs_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default.

View File

@ -274,6 +274,19 @@ FROM s3(
- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. In case of archive shows uncompressed file size of the file inside the archive.
- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`.
## Hive-style partitioning {#hive-style-partitioning}
When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`.
**Example**
Use virtual column, created with Hive-style partitioning
``` sql
SET use_hive_partitioning = 1;
SELECT * from s3('s3://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42;
```
## Storage Settings {#storage-settings}
- [s3_truncate_on_insert](/docs/en/operations/settings/settings.md#s3_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default.

View File

@ -55,6 +55,19 @@ Character `|` inside patterns is used to specify failover addresses. They are it
- `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`.
- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`.
## Hive-style partitioning {#hive-style-partitioning}
When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`.
**Example**
Use virtual column, created with Hive-style partitioning
``` sql
SET use_hive_partitioning = 1;
SELECT * from url('http://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42;
```
## Storage Settings {#storage-settings}
- [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default.

View File

@ -1307,6 +1307,7 @@ try
throw ErrnoException(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Input must be seekable file (it will be read twice)");
SingleReadBufferIterator read_buffer_iterator(std::move(file));
schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, context_const);
}
else

View File

@ -490,6 +490,8 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
/// process_list_element_holder is used to make an element in ProcessList live while BACKUP is working asynchronously.
auto process_list_element = context_in_use->getProcessListElement();
/// Update context to preserve query information in processlist (settings, current_database)
process_list_element->updateContext(context_in_use);
thread_pool.scheduleOrThrowOnError(
[this,
@ -853,6 +855,8 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
/// process_list_element_holder is used to make an element in ProcessList live while RESTORE is working asynchronously.
auto process_list_element = context_in_use->getProcessListElement();
/// Update context to preserve query information in processlist (settings, current_database)
process_list_element->updateContext(context_in_use);
thread_pool.scheduleOrThrowOnError(
[this,

View File

@ -676,6 +676,7 @@ class IColumn;
M(Bool, query_cache_squash_partial_results, true, "Squash partial result blocks to blocks of size 'max_block_size'. Reduces performance of inserts into the query cache but improves the compressability of cache entries.", 0) \
M(Seconds, query_cache_ttl, 60, "After this time in seconds entries in the query cache become stale", 0) \
M(Bool, query_cache_share_between_users, false, "Allow other users to read entry in the query cache", 0) \
M(String, query_cache_tag, "", "A string which acts as a label for query cache entries. The same queries with different tags are considered different by the query cache.", 0) \
M(Bool, enable_sharing_sets_for_mutations, true, "Allow sharing set objects build for IN subqueries between different tasks of the same mutation. This reduces memory usage and CPU consumption", 0) \
\
M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \
@ -921,6 +922,7 @@ class IColumn;
M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \
M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \
M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \
M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\
\
M(Bool, allow_statistics_optimize, false, "Allows using statistics to optimize queries", 0) ALIAS(allow_statistic_optimize) \
M(Bool, allow_experimental_statistics, false, "Allows using statistics", 0) ALIAS(allow_experimental_statistic) \

View File

@ -80,8 +80,10 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"restore_replace_external_engines_to_null", false, false, "New setting."},
{"input_format_json_max_depth", 1000000, 1000, "It was unlimited in previous versions, but that was unsafe."},
{"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"},
{"use_hive_partitioning", false, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines."},
{"allow_experimental_kafka_offsets_storage_in_keeper", false, false, "Allow the usage of experimental Kafka storage engine that stores the committed offsets in ClickHouse Keeper"},
{"allow_archive_path_syntax", true, true, "Added new setting to allow disabling archive path syntax."},
{"query_cache_tag", "", "", "New setting for labeling query cache settings."},
{"allow_experimental_time_series_table", false, false, "Added new setting to allow the TimeSeries table engine"},
{"enable_analyzer", 1, 1, "Added an alias to a setting `allow_experimental_analyzer`."},
{"optimize_functions_to_subcolumns", false, true, "Enabled settings by default"},

View File

@ -164,7 +164,7 @@ try
return {*iterator_data.cached_columns, *format_name};
}
schemas_for_union_mode.emplace_back(iterator_data.cached_columns->getAll(), read_buffer_iterator.getLastFileName());
schemas_for_union_mode.emplace_back(iterator_data.cached_columns->getAll(), read_buffer_iterator.getLastFilePath());
continue;
}
@ -250,7 +250,7 @@ try
if (!names_and_types.empty())
read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types));
schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFileName());
schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFilePath());
}
catch (...)
{
@ -411,7 +411,7 @@ try
throw Exception(ErrorCodes::CANNOT_DETECT_FORMAT, "The data format cannot be detected by the contents of the files. You can specify the format manually");
read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types));
schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFileName());
schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFilePath());
}
if (format_name && mode == SchemaInferenceMode::DEFAULT)
@ -527,9 +527,9 @@ try
}
catch (Exception & e)
{
auto file_name = read_buffer_iterator.getLastFileName();
if (!file_name.empty())
e.addMessage(fmt::format("(in file/uri {})", file_name));
auto file_path = read_buffer_iterator.getLastFilePath();
if (!file_path.empty())
e.addMessage(fmt::format("(in file/uri {})", file_path));
throw;
}

View File

@ -56,8 +56,8 @@ struct IReadBufferIterator
/// Set auto detected format name.
virtual void setFormatName(const String & /*format_name*/) {}
/// Get last processed file name for better exception messages.
virtual String getLastFileName() const { return ""; }
/// Get last processed file path for better exception messages.
virtual String getLastFilePath() const { return ""; }
/// Return true if method recreateLastReadBuffer is implemented.
virtual bool supportsLastReadBufferRecreation() const { return false; }

View File

@ -128,7 +128,7 @@ namespace
bool isQueryCacheRelatedSetting(const String & setting_name)
{
return setting_name.starts_with("query_cache_") || setting_name.ends_with("_query_cache");
return (setting_name.starts_with("query_cache_") || setting_name.ends_with("_query_cache")) && setting_name != "query_cache_tag";
}
class RemoveQueryCacheSettingsMatcher
@ -242,11 +242,18 @@ QueryCache::Key::Key(
, expires_at(expires_at_)
, is_compressed(is_compressed_)
, query_string(queryStringFromAST(ast_))
, tag(settings.query_cache_tag)
{
}
QueryCache::Key::Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_)
: QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST, current database, user name/roles
QueryCache::Key::Key(
ASTPtr ast_,
const String & current_database,
const Settings & settings,
std::optional<UUID> user_id_,
const std::vector<UUID> & current_user_roles_)
: QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false)
/// ^^ dummy values for everything != AST, current database, user name/roles
{
}

View File

@ -88,6 +88,11 @@ public:
/// SYSTEM.QUERY_CACHE.
const String query_string;
/// A tag (namespace) for distinguish multiple entries of the same query.
/// This member has currently no use besides that SYSTEM.QUERY_CACHE can populate the 'tag' column conveniently without having to
/// compute the tag from the query AST.
const String tag;
/// Ctor to construct a Key for writing into query cache.
Key(ASTPtr ast_,
const String & current_database,
@ -99,7 +104,10 @@ public:
bool is_compressed);
/// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name).
Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_);
Key(ASTPtr ast_,
const String & current_database,
const Settings & settings,
std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_);
bool operator==(const Key & other) const;
};

View File

@ -244,6 +244,9 @@ public:
/// Same as checkTimeLimit but it never throws
[[nodiscard]] bool checkTimeLimitSoft();
/// Use it in case of the query left in background to execute asynchronously
void updateContext(ContextWeakPtr weak_context) { context = std::move(weak_context); }
/// Get the reference for the start of the query. Used to synchronize with other Stopwatches
UInt64 getQueryCPUStartTime() { return watch.getStart(); }
};

View File

@ -97,24 +97,9 @@ namespace
void formatCurrentGrantsElements(const AccessRightsElements & elements, const IAST::FormatSettings & settings)
{
for (size_t i = 0; i != elements.size(); ++i)
{
const auto & element = elements[i];
bool next_element_on_same_db_and_table = false;
if (i != elements.size() - 1)
{
const auto & next_element = elements[i + 1];
if (element.sameDatabaseAndTableAndParameter(next_element))
next_element_on_same_db_and_table = true;
}
if (!next_element_on_same_db_and_table)
{
settings.ostr << " ";
formatONClause(element, settings);
}
}
settings.ostr << "(";
formatElementsWithoutOptions(elements, settings);
settings.ostr << ")";
}
}

View File

@ -445,7 +445,7 @@ StorageHive::StorageHive(
storage_metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, storage_metadata.columns, getContext());
setInMemoryMetadata(storage_metadata);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()));
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), getContext()));
}
void StorageHive::lazyInitialize()

View File

@ -3517,7 +3517,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
const auto & new_column = new_metadata.getColumns().get(command.column_name);
if (!old_column.type->equals(*new_column.type))
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"ALTER types of column {} with statistics is not not safe "
"ALTER types of column {} with statistics is not safe "
"because it can change the representation of statistics",
backQuoteIfNeed(command.column_name));
}

View File

@ -47,6 +47,8 @@ void MessageQueueSink::consume(Chunk & chunk)
if (columns.empty())
return;
/// The formatter might hold pointers to buffer (e.g. if PeekableWriteBuffer is used), which means the formatter
/// needs to be reset after buffer might reallocate its memory. In this exact case after restarting the buffer.
if (row_format)
{
size_t row = 0;
@ -61,12 +63,12 @@ void MessageQueueSink::consume(Chunk & chunk)
row_format->writeRow(columns, row);
}
row_format->finalize();
row_format->resetFormatter();
producer->produce(buffer->str(), i, columns, row - 1);
/// Reallocate buffer if it's capacity is large then DBMS_DEFAULT_BUFFER_SIZE,
/// because most likely in this case we serialized abnormally large row
/// and won't need this large allocated buffer anymore.
buffer->restart(DBMS_DEFAULT_BUFFER_SIZE);
row_format->resetFormatter();
}
}
else
@ -74,12 +76,11 @@ void MessageQueueSink::consume(Chunk & chunk)
format->write(getHeader().cloneWithColumns(chunk.detachColumns()));
format->finalize();
producer->produce(buffer->str(), chunk.getNumRows(), columns, chunk.getNumRows() - 1);
format->resetFormatter();
buffer->restart();
format->resetFormatter();
}
}
void MessageQueueSink::onCancel() noexcept
{
try

View File

@ -91,8 +91,9 @@ public:
{
ConfigurationPtr configuration = base_configuration->clone();
configuration->setPaths(metadata->getDataFiles());
std::string sample_path;
return Storage::resolveSchemaFromData(
object_storage_, configuration, format_settings_, local_context);
object_storage_, configuration, format_settings_, sample_path, local_context);
}
}

View File

@ -132,7 +132,7 @@ void ReadBufferIterator::setFormatName(const String & format_name)
format = format_name;
}
String ReadBufferIterator::getLastFileName() const
String ReadBufferIterator::getLastFilePath() const
{
if (current_object_info)
return current_object_info->getPath();

View File

@ -33,7 +33,7 @@ public:
void setResultingSchema(const ColumnsDescription & columns) override;
String getLastFileName() const override;
String getLastFilePath() const override;
void setFormatName(const String & format_name) override;

View File

@ -1,4 +1,5 @@
#include <Storages/ObjectStorage/StorageObjectStorage.h>
#include <Core/ColumnWithTypeAndName.h>
#include <Core/Settings.h>
#include <Formats/FormatFactory.h>
@ -33,6 +34,33 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
String StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context)
{
auto query_settings = configuration->getQuerySettings(context);
/// We don't want to throw an exception if there are no files with specified path.
query_settings.throw_on_zero_files_match = false;
bool local_distributed_processing = distributed_processing;
if (context->getSettingsRef().use_hive_partitioning)
local_distributed_processing = false;
auto file_iterator = StorageObjectStorageSource::createFileIterator(
configuration,
query_settings,
object_storage,
local_distributed_processing,
context,
{}, // predicate
metadata.getColumns().getAll(), // virtual_columns
nullptr, // read_keys
{} // file_progress_callback
);
if (auto file = file_iterator->next(0))
return file->getPath();
return "";
}
StorageObjectStorage::StorageObjectStorage(
ConfigurationPtr configuration_,
ObjectStoragePtr object_storage_,
@ -53,7 +81,9 @@ StorageObjectStorage::StorageObjectStorage(
, log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName())))
{
ColumnsDescription columns{columns_};
resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, context);
std::string sample_path;
resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, sample_path, context);
configuration->check(context);
StorageInMemoryMetadata metadata;
@ -61,7 +91,10 @@ StorageObjectStorage::StorageObjectStorage(
metadata.setConstraints(constraints_);
metadata.setComment(comment);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns()));
if (sample_path.empty() && context->getSettingsRef().use_hive_partitioning)
sample_path = getPathSample(metadata, context);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context, sample_path, format_settings));
setInMemoryMetadata(metadata);
}
@ -198,7 +231,7 @@ private:
return;
auto context = getContext();
iterator_wrapper = StorageObjectStorageSource::createFileIterator(
configuration, object_storage, distributed_processing,
configuration, configuration->getQuerySettings(context), object_storage, distributed_processing,
context, predicate, virtual_columns, nullptr, context->getFileProgressCallback());
}
};
@ -350,6 +383,7 @@ std::unique_ptr<ReadBufferIterator> StorageObjectStorage::createReadBufferIterat
{
auto file_iterator = StorageObjectStorageSource::createFileIterator(
configuration,
configuration->getQuerySettings(context),
object_storage,
false/* distributed_processing */,
context,
@ -366,33 +400,41 @@ ColumnsDescription StorageObjectStorage::resolveSchemaFromData(
const ObjectStoragePtr & object_storage,
const ConfigurationPtr & configuration,
const std::optional<FormatSettings> & format_settings,
std::string & sample_path,
const ContextPtr & context)
{
ObjectInfos read_keys;
auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context);
return readSchemaFromFormat(configuration->format, format_settings, *iterator, context);
auto schema = readSchemaFromFormat(configuration->format, format_settings, *iterator, context);
sample_path = iterator->getLastFilePath();
return schema;
}
std::string StorageObjectStorage::resolveFormatFromData(
const ObjectStoragePtr & object_storage,
const ConfigurationPtr & configuration,
const std::optional<FormatSettings> & format_settings,
std::string & sample_path,
const ContextPtr & context)
{
ObjectInfos read_keys;
auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context);
return detectFormatAndReadSchema(format_settings, *iterator, context).second;
auto format_and_schema = detectFormatAndReadSchema(format_settings, *iterator, context).second;
sample_path = iterator->getLastFilePath();
return format_and_schema;
}
std::pair<ColumnsDescription, std::string> StorageObjectStorage::resolveSchemaAndFormatFromData(
const ObjectStoragePtr & object_storage,
const ConfigurationPtr & configuration,
const std::optional<FormatSettings> & format_settings,
std::string & sample_path,
const ContextPtr & context)
{
ObjectInfos read_keys;
auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context);
auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, context);
sample_path = iterator->getLastFilePath();
configuration->format = format;
return std::pair(columns, format);
}

View File

@ -102,23 +102,28 @@ public:
const ObjectStoragePtr & object_storage,
const ConfigurationPtr & configuration,
const std::optional<FormatSettings> & format_settings,
std::string & sample_path,
const ContextPtr & context);
static std::string resolveFormatFromData(
const ObjectStoragePtr & object_storage,
const ConfigurationPtr & configuration,
const std::optional<FormatSettings> & format_settings,
std::string & sample_path,
const ContextPtr & context);
static std::pair<ColumnsDescription, std::string> resolveSchemaAndFormatFromData(
const ObjectStoragePtr & object_storage,
const ConfigurationPtr & configuration,
const std::optional<FormatSettings> & format_settings,
std::string & sample_path,
const ContextPtr & context);
protected:
virtual void updateConfiguration(ContextPtr local_context);
String getPathSample(StorageInMemoryMetadata metadata, ContextPtr context);
virtual ReadFromFormatInfo prepareReadingFromFormat(
const Strings & requested_columns,
const StorageSnapshotPtr & storage_snapshot,

View File

@ -1,6 +1,8 @@
#include "Storages/ObjectStorage/StorageObjectStorageCluster.h"
#include <Common/Exception.h>
#include <Core/Settings.h>
#include <Formats/FormatFactory.h>
#include <Parsers/queryToString.h>
#include <Processors/Sources/RemoteSource.h>
#include <QueryPipeline/RemoteQueryExecutor.h>
@ -19,6 +21,28 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
String StorageObjectStorageCluster::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context)
{
auto query_settings = configuration->getQuerySettings(context);
/// We don't want to throw an exception if there are no files with specified path.
query_settings.throw_on_zero_files_match = false;
auto file_iterator = StorageObjectStorageSource::createFileIterator(
configuration,
query_settings,
object_storage,
false, // distributed_processing
context,
{}, // predicate
metadata.getColumns().getAll(), // virtual_columns
nullptr, // read_keys
{} // file_progress_callback
);
if (auto file = file_iterator->next(0))
return file->getPath();
return "";
}
StorageObjectStorageCluster::StorageObjectStorageCluster(
const String & cluster_name_,
ConfigurationPtr configuration_,
@ -33,14 +57,18 @@ StorageObjectStorageCluster::StorageObjectStorageCluster(
, object_storage(object_storage_)
{
ColumnsDescription columns{columns_};
resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, {}, context_);
std::string sample_path;
resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, {}, sample_path, context_);
configuration->check(context_);
StorageInMemoryMetadata metadata;
metadata.setColumns(columns);
metadata.setConstraints(constraints_);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns()));
if (sample_path.empty() && context_->getSettingsRef().use_hive_partitioning)
sample_path = getPathSample(metadata, context_);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_, sample_path));
setInMemoryMetadata(metadata);
}
@ -83,8 +111,8 @@ RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExten
const ActionsDAG::Node * predicate, const ContextPtr & local_context) const
{
auto iterator = StorageObjectStorageSource::createFileIterator(
configuration, object_storage, /* distributed_processing */false, local_context,
predicate, virtual_columns, nullptr, local_context->getFileProgressCallback());
configuration, configuration->getQuerySettings(local_context), object_storage, /* distributed_processing */false,
local_context, predicate, virtual_columns, nullptr, local_context->getFileProgressCallback());
auto callback = std::make_shared<std::function<String()>>([iterator]() mutable -> String
{

View File

@ -27,6 +27,8 @@ public:
RemoteQueryExecutor::Extension getTaskIteratorExtension(
const ActionsDAG::Node * predicate, const ContextPtr & context) const override;
String getPathSample(StorageInMemoryMetadata metadata, ContextPtr context);
private:
void updateQueryToSendIfNeeded(
ASTPtr & query,

View File

@ -99,6 +99,7 @@ std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier(
std::shared_ptr<StorageObjectStorageSource::IIterator> StorageObjectStorageSource::createFileIterator(
ConfigurationPtr configuration,
const StorageObjectStorage::QuerySettings & query_settings,
ObjectStoragePtr object_storage,
bool distributed_processing,
const ContextPtr & local_context,
@ -116,7 +117,6 @@ std::shared_ptr<StorageObjectStorageSource::IIterator> StorageObjectStorageSourc
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Expression can not have wildcards inside {} name", configuration->getNamespaceType());
auto settings = configuration->getQuerySettings(local_context);
const bool is_archive = configuration->isArchive();
std::unique_ptr<IIterator> iterator;
@ -125,8 +125,8 @@ std::shared_ptr<StorageObjectStorageSource::IIterator> StorageObjectStorageSourc
/// Iterate through disclosed globs and make a source for each file
iterator = std::make_unique<GlobIterator>(
object_storage, configuration, predicate, virtual_columns,
local_context, is_archive ? nullptr : read_keys, settings.list_object_keys_size,
settings.throw_on_zero_files_match, file_progress_callback);
local_context, is_archive ? nullptr : read_keys, query_settings.list_object_keys_size,
query_settings.throw_on_zero_files_match, file_progress_callback);
}
else
{
@ -148,7 +148,7 @@ std::shared_ptr<StorageObjectStorageSource::IIterator> StorageObjectStorageSourc
iterator = std::make_unique<KeysIterator>(
object_storage, copy_configuration, virtual_columns, is_archive ? nullptr : read_keys,
settings.ignore_non_existent_file, file_progress_callback);
query_settings.ignore_non_existent_file, file_progress_callback);
}
if (is_archive)
@ -198,15 +198,17 @@ Chunk StorageObjectStorageSource::generate()
const auto & object_info = reader.getObjectInfo();
const auto & filename = object_info->getFileName();
chassert(object_info->metadata);
VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk(
chunk,
read_from_format_info.requested_virtual_columns,
{.path = getUniqueStoragePathIdentifier(*configuration, *object_info, false),
.size = object_info->isArchive() ? object_info->fileSizeInArchive() : object_info->metadata->size_bytes,
.filename = &filename,
.last_modified = object_info->metadata->last_modified,
.etag = &(object_info->metadata->etag)
});
{
.path = getUniqueStoragePathIdentifier(*configuration, *object_info, false),
.size = object_info->isArchive() ? object_info->fileSizeInArchive() : object_info->metadata->size_bytes,
.filename = &filename,
.last_modified = object_info->metadata->last_modified,
.etag = &(object_info->metadata->etag)
}, getContext(), read_from_format_info.columns_description);
const auto & partition_columns = configuration->getPartitionColumns();
if (!partition_columns.empty() && chunk_size && chunk.hasColumns())

View File

@ -52,6 +52,7 @@ public:
static std::shared_ptr<IIterator> createFileIterator(
ConfigurationPtr configuration,
const StorageObjectStorage::QuerySettings & query_settings,
ObjectStoragePtr object_storage,
bool distributed_processing,
const ContextPtr & local_context,

View File

@ -49,19 +49,20 @@ void resolveSchemaAndFormat(
ObjectStoragePtr object_storage,
const StorageObjectStorage::ConfigurationPtr & configuration,
std::optional<FormatSettings> format_settings,
std::string & sample_path,
const ContextPtr & context)
{
if (columns.empty())
{
if (format == "auto")
std::tie(columns, format) =
StorageObjectStorage::resolveSchemaAndFormatFromData(object_storage, configuration, format_settings, context);
StorageObjectStorage::resolveSchemaAndFormatFromData(object_storage, configuration, format_settings, sample_path, context);
else
columns = StorageObjectStorage::resolveSchemaFromData(object_storage, configuration, format_settings, context);
columns = StorageObjectStorage::resolveSchemaFromData(object_storage, configuration, format_settings, sample_path, context);
}
else if (format == "auto")
{
format = StorageObjectStorage::resolveFormatFromData(object_storage, configuration, format_settings, context);
format = StorageObjectStorage::resolveFormatFromData(object_storage, configuration, format_settings, sample_path, context);
}
if (!columns.hasOnlyOrdinary())

View File

@ -19,6 +19,7 @@ void resolveSchemaAndFormat(
ObjectStoragePtr object_storage,
const StorageObjectStorage::ConfigurationPtr & configuration,
std::optional<FormatSettings> format_settings,
std::string & sample_path,
const ContextPtr & context);
}

View File

@ -524,7 +524,7 @@ Chunk ObjectStorageQueueSource::generateImpl()
{
.path = path,
.size = reader.getObjectInfo()->metadata->size_bytes
});
}, getContext(), read_from_format_info.columns_description);
return chunk;
}

View File

@ -161,14 +161,15 @@ StorageObjectStorageQueue::StorageObjectStorageQueue(
configuration->check(context_);
ColumnsDescription columns{columns_};
resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, context_);
std::string sample_path;
resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, sample_path, context_);
configuration->check(context_);
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()));
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_));
setInMemoryMetadata(storage_metadata);
LOG_INFO(log, "Using zookeeper path: {}", zk_path.string());

View File

@ -53,6 +53,7 @@
#include <Common/logger_useful.h>
#include <Common/ProfileEvents.h>
#include <Common/re2.h>
#include <Formats/SchemaInferenceUtils.h>
#include "base/defines.h"
#include <Core/Settings.h>
@ -516,7 +517,7 @@ namespace
StorageFile::getSchemaCache(getContext()).addManyColumns(cache_keys, columns);
}
String getLastFileName() const override
String getLastFilePath() const override
{
if (current_index != 0)
return paths[current_index - 1];
@ -793,7 +794,7 @@ namespace
format = format_name;
}
String getLastFileName() const override
String getLastFilePath() const override
{
return last_read_file_path;
}
@ -1112,7 +1113,8 @@ void StorageFile::setStorageMetadata(CommonArguments args)
storage_metadata.setConstraints(args.constraints);
storage_metadata.setComment(args.comment);
setInMemoryMetadata(storage_metadata);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()));
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), paths.empty() ? "" : paths[0], format_settings));
}
@ -1466,7 +1468,7 @@ Chunk StorageFileSource::generate()
.size = current_file_size,
.filename = (filename_override.has_value() ? &filename_override.value() : nullptr),
.last_modified = current_file_last_modified
});
}, getContext(), columns_description);
return chunk;
}

View File

@ -61,7 +61,7 @@ StorageFileCluster::StorageFileCluster(
storage_metadata.setConstraints(constraints_);
setInMemoryMetadata(storage_metadata);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()));
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, paths.empty() ? "" : paths[0]));
}
void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context)

View File

@ -36,6 +36,8 @@
#include <Common/thread_local_rng.h>
#include <Common/logger_useful.h>
#include <Common/re2.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Core/ServerSettings.h>
#include <Core/Settings.h>
#include <IO/ReadWriteBufferFromHTTP.h>
@ -90,11 +92,22 @@ static const std::vector<std::shared_ptr<re2::RE2>> optional_regex_keys = {
std::make_shared<re2::RE2>(R"(headers.header\[[0-9]*\].value)"),
};
static bool urlWithGlobs(const String & uri)
bool urlWithGlobs(const String & uri)
{
return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos;
}
String getSampleURI(String uri, ContextPtr context)
{
if (urlWithGlobs(uri))
{
auto uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements);
if (!uris.empty())
return uris[0];
}
return uri;
}
static ConnectionTimeouts getHTTPTimeouts(ContextPtr context)
{
return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings().keep_alive_timeout);
@ -153,7 +166,8 @@ IStorageURLBase::IStorageURLBase(
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()));
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_, getSampleURI(uri, context_), format_settings));
}
@ -414,13 +428,14 @@ Chunk StorageURLSource::generate()
size_t chunk_size = 0;
if (input_format)
chunk_size = input_format->getApproxBytesReadForChunk();
progress(num_rows, chunk_size ? chunk_size : chunk.bytes());
VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk(
chunk, requested_virtual_columns,
{
.path = curr_uri.getPath(),
.size = current_file_size
});
.size = current_file_size,
}, getContext(), columns_description);
return chunk;
}
@ -839,7 +854,7 @@ namespace
format = format_name;
}
String getLastFileName() const override { return current_url_option; }
String getLastFilePath() const override { return current_url_option; }
bool supportsLastReadBufferRecreation() const override { return true; }
@ -1160,6 +1175,7 @@ void ReadFromURL::createIterator(const ActionsDAG::Node * predicate)
void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
createIterator(nullptr);
const auto & settings = context->getSettingsRef();
if (is_empty_glob)
{
@ -1170,7 +1186,6 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil
Pipes pipes;
pipes.reserve(num_streams);
const auto & settings = context->getSettingsRef();
const size_t max_parsing_threads = num_streams >= settings.max_parsing_threads ? 1 : (settings.max_parsing_threads / num_streams);
for (size_t i = 0; i < num_streams; ++i)

View File

@ -141,6 +141,9 @@ private:
virtual Block getHeaderBlock(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) const = 0;
};
bool urlWithGlobs(const String & uri);
String getSampleURI(String uri, ContextPtr context);
class StorageURLSource : public SourceWithKeyCondition, WithContext
{

View File

@ -76,7 +76,7 @@ StorageURLCluster::StorageURLCluster(
storage_metadata.setConstraints(constraints_);
setInMemoryMetadata(storage_metadata);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()));
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, getSampleURI(uri, context)));
}
void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context)

View File

@ -1,6 +1,7 @@
#include "StorageSystemQueryCache.h"
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Cache/QueryCache.h>
#include <Interpreters/Context.h>
@ -15,6 +16,7 @@ ColumnsDescription StorageSystemQueryCache::getColumnsDescription()
{
{"query", std::make_shared<DataTypeString>(), "Query string."},
{"result_size", std::make_shared<DataTypeUInt64>(), "Size of the query cache entry."},
{"tag", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "Tag of the query cache entry."},
{"stale", std::make_shared<DataTypeUInt8>(), "If the query cache entry is stale."},
{"shared", std::make_shared<DataTypeUInt8>(), "If the query cache entry is shared between multiple users."},
{"compressed", std::make_shared<DataTypeUInt8>(), "If the query cache entry is compressed."},
@ -51,11 +53,12 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr
res_columns[0]->insert(key.query_string); /// approximates the original query string
res_columns[1]->insert(QueryCache::QueryCacheEntryWeight()(*query_result));
res_columns[2]->insert(key.expires_at < std::chrono::system_clock::now());
res_columns[3]->insert(key.is_shared);
res_columns[4]->insert(key.is_compressed);
res_columns[5]->insert(std::chrono::system_clock::to_time_t(key.expires_at));
res_columns[6]->insert(key.ast_hash.low64); /// query cache considers aliases (issue #56258)
res_columns[2]->insert(key.tag);
res_columns[3]->insert(key.expires_at < std::chrono::system_clock::now());
res_columns[4]->insert(key.is_shared);
res_columns[5]->insert(key.is_compressed);
res_columns[6]->insert(std::chrono::system_clock::to_time_t(key.expires_at));
res_columns[7]->insert(key.ast_hash.low64); /// query cache considers aliases (issue #56258)
}
}

View File

@ -1,51 +1,66 @@
#include <Storages/VirtualColumnUtils.h>
#include <memory>
#include <stack>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnSet.h>
#include <Columns/ColumnsCommon.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/FilterDescription.h>
#include <Core/NamesAndTypes.h>
#include <Core/TypeId.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionsLogical.h>
#include <Functions/IFunction.h>
#include <Functions/IFunctionAdaptors.h>
#include <Functions/indexHint.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/misc.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/makeASTForLogicalFunction.h>
#include <Processors/Executors/CompletedPipelineExecutor.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnsCommon.h>
#include <Columns/FilterDescription.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/Sinks/EmptySink.h>
#include <Processors/Executors/CompletedPipelineExecutor.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/VirtualColumnUtils.h>
#include <IO/WriteHelpers.h>
#include <Common/re2.h>
#include <Common/typeid_cast.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Formats/EscapingRuleUtils.h>
#include <Formats/FormatFactory.h>
#include <Core/Settings.h>
#include "Functions/FunctionsLogical.h"
#include "Functions/IFunction.h"
#include "Functions/IFunctionAdaptors.h"
#include "Functions/indexHint.h"
#include <Interpreters/convertFieldToType.h>
#include <Parsers/makeASTForLogicalFunction.h>
#include <Columns/ColumnSet.h>
#include <Functions/FunctionHelpers.h>
#include <Interpreters/ActionsVisitor.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
}
namespace VirtualColumnUtils
{
@ -114,7 +129,29 @@ NameSet getVirtualNamesForFileLikeStorage()
return {"_path", "_file", "_size", "_time", "_etag"};
}
VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns)
std::unordered_map<std::string, std::string> parseHivePartitioningKeysAndValues(const String & path, const ColumnsDescription & storage_columns)
{
std::string pattern = "([^/]+)=([^/]+)/";
re2::StringPiece input_piece(path);
std::unordered_map<std::string, std::string> key_values;
std::string key, value;
std::unordered_set<String> used_keys;
while (RE2::FindAndConsume(&input_piece, pattern, &key, &value))
{
if (used_keys.contains(key))
throw Exception(ErrorCodes::INCORRECT_DATA, "Path '{}' to file with enabled hive-style partitioning contains duplicated partition key {}, only unique keys are allowed", path, key);
used_keys.insert(key);
auto col_name = "_" + key;
while (storage_columns.has(col_name))
col_name = "_" + col_name;
key_values[col_name] = value;
}
return key_values;
}
VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, const ContextPtr & context, const std::string & path, std::optional<FormatSettings> format_settings_)
{
VirtualColumnsDescription desc;
@ -132,6 +169,22 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription
add_virtual("_time", makeNullable(std::make_shared<DataTypeDateTime>()));
add_virtual("_etag", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()));
if (context->getSettingsRef().use_hive_partitioning)
{
auto map = parseHivePartitioningKeysAndValues(path, storage_columns);
auto format_settings = format_settings_ ? *format_settings_ : getFormatSettings(context);
for (auto & item : map)
{
auto type = tryInferDataTypeByEscapingRule(item.second, format_settings, FormatSettings::EscapingRule::Raw);
if (type == nullptr)
type = std::make_shared<DataTypeString>();
if (type->canBeInsideLowCardinality())
add_virtual(item.first, std::make_shared<DataTypeLowCardinality>(type));
else
add_virtual(item.first, type);
}
}
return desc;
}
@ -191,8 +244,12 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & paths, const
void addRequestedFileLikeStorageVirtualsToChunk(
Chunk & chunk, const NamesAndTypesList & requested_virtual_columns,
VirtualsForFileLikeStorage virtual_values)
VirtualsForFileLikeStorage virtual_values, ContextPtr context, const ColumnsDescription & columns)
{
std::unordered_map<std::string, std::string> hive_map;
if (context->getSettingsRef().use_hive_partitioning)
hive_map = parseHivePartitioningKeysAndValues(virtual_values.path, columns);
for (const auto & virtual_column : requested_virtual_columns)
{
if (virtual_column.name == "_path")
@ -226,6 +283,10 @@ void addRequestedFileLikeStorageVirtualsToChunk(
else
chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst());
}
else if (auto it = hive_map.find(virtual_column.getNameInStorage()); it != hive_map.end())
{
chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), convertFieldToType(Field(it->second), *virtual_column.type))->convertToFullColumnIfConst());
}
else if (virtual_column.name == "_etag")
{
if (virtual_values.etag)

View File

@ -5,6 +5,7 @@
#include <Parsers/IAST_fwd.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/VirtualColumnsDescription.h>
#include <Formats/FormatSettings.h>
#include <unordered_set>
@ -68,7 +69,11 @@ auto extractSingleValueFromBlock(const Block & block, const String & name)
}
NameSet getVirtualNamesForFileLikeStorage();
VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns);
VirtualColumnsDescription getVirtualsForFileLikeStorage(
const ColumnsDescription & storage_columns,
const ContextPtr & context,
const std::string & sample_path = "",
std::optional<FormatSettings> format_settings_ = std::nullopt);
std::optional<ActionsDAG> createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns);
@ -100,7 +105,7 @@ struct VirtualsForFileLikeStorage
void addRequestedFileLikeStorageVirtualsToChunk(
Chunk & chunk, const NamesAndTypesList & requested_virtual_columns,
VirtualsForFileLikeStorage virtual_values);
VirtualsForFileLikeStorage virtual_values, ContextPtr context, const ColumnsDescription & columns);
}
}

View File

@ -84,7 +84,8 @@ ColumnsDescription TableFunctionObjectStorage<
context->checkAccess(getSourceAccessType());
ColumnsDescription columns;
auto storage = getObjectStorage(context, !is_insert_query);
resolveSchemaAndFormat(columns, configuration->format, storage, configuration, std::nullopt, context);
std::string sample_path;
resolveSchemaAndFormat(columns, configuration->format, storage, configuration, std::nullopt, sample_path, context);
return columns;
}
else

View File

@ -69,9 +69,9 @@ def get_changed_tests_to_run(pr_info, repo_path):
return []
for fpath in changed_files:
if "tests/integration/test_" in fpath:
if re.search(r"tests/integration/test_.*/test.*\.py", fpath) is not None:
logging.info("File %s changed and seems like integration test", fpath)
result.add(fpath.split("/")[2])
result.add("/".join(fpath.split("/")[2:]))
return filter_existing_tests(result, repo_path)

View File

@ -162,9 +162,13 @@
"test_storage_kafka/test.py::test_system_kafka_consumers_rebalance_mv",
"test_storage_kafka/test.py::test_formats_errors",
"test_storage_kafka/test.py::test_multiple_read_in_materialized_views",
"test_storage_kafka/test.py::test_kafka_null_message",
"test_storage_kafka/test_produce_http_interface.py::test_kafka_produce_http_interface_row_based_format",
"test_storage_kerberized_kafka/test.py::test_kafka_json_as_string",
"test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_request_new_ticket_after_expiration",
"test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc",
"test_storage_kerberized_kafka/test.py::test_kafka_config_from_sql_named_collection"
]

View File

@ -21,14 +21,14 @@ def start_cluster():
def test_cluster(start_cluster):
assert (
node1.query(
"SELECT hostName() FROM clusterAllReplicas('one_shard_two_nodes', system.one)"
"SELECT hostName() FROM clusterAllReplicas('one_shard_two_nodes', system.one) ORDER BY ALL"
)
== "node1\nnode2\n"
)
assert set(
node1.query(
"""SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) ORDER BY dummy"""
"""SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) ORDER BY ALL"""
).splitlines()
) == {"node1\t0", "node2\t0"}
@ -48,7 +48,7 @@ def test_global_in(start_cluster):
assert set(
node1.query(
"""SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) where dummy GLOBAL IN u"""
"""SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) where dummy GLOBAL IN u ORDER BY ALL"""
).splitlines()
) == {"node1\t0", "node2\t0"}
@ -63,7 +63,7 @@ def test_global_in(start_cluster):
def test_skip_unavailable_replica(start_cluster, cluster):
assert (
node1.query(
f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) settings skip_unavailable_shards=1"
f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) ORDER BY ALL settings skip_unavailable_shards=1"
)
== "node1\nnode2\n"
)
@ -81,5 +81,5 @@ def test_error_on_unavailable_replica(start_cluster, cluster):
# so when skip_unavailable_shards=0 - any unavailable replica should lead to an error
with pytest.raises(QueryRuntimeException):
node1.query(
f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) settings skip_unavailable_shards=0"
f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) ORDER BY ALL settings skip_unavailable_shards=0"
)

View File

@ -5,6 +5,7 @@ import json
import logging
import os
import io
import re
import random
import threading
import time
@ -134,6 +135,7 @@ def test_create_table_connection_string(cluster):
Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_create_connection_string', 'CSV')
""",
)
azure_query(node, "DROP TABLE IF EXISTS test_create_table_conn_string")
def test_create_table_account_string(cluster):
@ -143,6 +145,7 @@ def test_create_table_account_string(cluster):
f"CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}',"
f"'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')",
)
azure_query(node, "DROP TABLE IF EXISTS test_create_table_account_url")
def test_simple_write_account_string(cluster):
@ -156,6 +159,7 @@ def test_simple_write_account_string(cluster):
azure_query(node, "INSERT INTO test_simple_write VALUES (1, 'a')")
print(get_azure_file_content("test_simple_write.csv", port))
assert get_azure_file_content("test_simple_write.csv", port) == '1,"a"\n'
azure_query(node, "DROP TABLE test_simple_write")
def test_simple_write_connection_string(cluster):
@ -169,6 +173,7 @@ def test_simple_write_connection_string(cluster):
azure_query(node, "INSERT INTO test_simple_write_connection_string VALUES (1, 'a')")
print(get_azure_file_content("test_simple_write_c.csv", port))
assert get_azure_file_content("test_simple_write_c.csv", port) == '1,"a"\n'
azure_query(node, "DROP TABLE test_simple_write_connection_string")
def test_simple_write_named_collection_1(cluster):
@ -184,7 +189,7 @@ def test_simple_write_named_collection_1(cluster):
)
print(get_azure_file_content("test_simple_write_named.csv", port))
assert get_azure_file_content("test_simple_write_named.csv", port) == '1,"a"\n'
azure_query(node, "TRUNCATE TABLE test_simple_write_named_collection_1")
azure_query(node, "DROP TABLE test_simple_write_named_collection_1")
def test_simple_write_named_collection_2(cluster):
@ -201,6 +206,7 @@ def test_simple_write_named_collection_2(cluster):
)
print(get_azure_file_content("test_simple_write_named_2.csv", port))
assert get_azure_file_content("test_simple_write_named_2.csv", port) == '1,"a"\n'
azure_query(node, "DROP TABLE test_simple_write_named_collection_2")
def test_partition_by(cluster):
@ -222,6 +228,7 @@ def test_partition_by(cluster):
assert "1,2,3\n" == get_azure_file_content("test_3.csv", port)
assert "3,2,1\n" == get_azure_file_content("test_1.csv", port)
assert "78,43,45\n" == get_azure_file_content("test_45.csv", port)
azure_query(node, "DROP TABLE test_partitioned_write")
def test_partition_by_string_column(cluster):
@ -242,6 +249,7 @@ def test_partition_by_string_column(cluster):
assert '1,"foo/bar"\n' == get_azure_file_content("test_foo/bar.csv", port)
assert '3,"йцук"\n' == get_azure_file_content("test_йцук.csv", port)
assert '78,"你好"\n' == get_azure_file_content("test_你好.csv", port)
azure_query(node, "DROP TABLE test_partitioned_string_write")
def test_partition_by_const_column(cluster):
@ -260,6 +268,7 @@ def test_partition_by_const_column(cluster):
)
azure_query(node, f"INSERT INTO test_partitioned_const_write VALUES {values}")
assert values_csv == get_azure_file_content("test_88.csv", port)
azure_query(node, "DROP TABLE test_partitioned_const_write")
def test_truncate(cluster):
@ -275,6 +284,7 @@ def test_truncate(cluster):
azure_query(node, "TRUNCATE TABLE test_truncate")
with pytest.raises(Exception):
print(get_azure_file_content("test_truncate.csv", port))
azure_query(node, "DROP TABLE test_truncate")
def test_simple_read_write(cluster):
@ -291,6 +301,7 @@ def test_simple_read_write(cluster):
assert get_azure_file_content("test_simple_read_write.csv", port) == '1,"a"\n'
print(azure_query(node, "SELECT * FROM test_simple_read_write"))
assert azure_query(node, "SELECT * FROM test_simple_read_write") == "1\ta\n"
azure_query(node, "DROP TABLE test_simple_read_write")
def test_create_new_files_on_insert(cluster):
@ -343,6 +354,7 @@ def test_overwrite(cluster):
result = azure_query(node, f"select count() from test_overwrite")
assert int(result) == 200
azure_query(node, f"DROP TABLE test_overwrite")
def test_insert_with_path_with_globs(cluster):
@ -355,6 +367,7 @@ def test_insert_with_path_with_globs(cluster):
node.query_and_get_error(
f"insert into table function test_insert_globs SELECT number, randomString(100) FROM numbers(500)"
)
azure_query(node, f"DROP TABLE test_insert_globs")
def test_put_get_with_globs(cluster):
@ -363,6 +376,7 @@ def test_put_get_with_globs(cluster):
node = cluster.instances["node"] # type: ClickHouseInstance
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
max_path = ""
used_names = []
for i in range(10):
for j in range(10):
path = "{}/{}_{}/{}.csv".format(
@ -371,6 +385,8 @@ def test_put_get_with_globs(cluster):
max_path = max(path, max_path)
values = f"({i},{j},{i + j})"
used_names.append(f"test_put_{i}_{j}")
azure_query(
node,
f"CREATE TABLE test_put_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, "
@ -391,6 +407,9 @@ def test_put_get_with_globs(cluster):
bucket="cont", max_path=max_path
)
]
azure_query(node, "DROP TABLE test_glob_select")
for name in used_names:
azure_query(node, f"DROP TABLE {name}")
def test_azure_glob_scheherazade(cluster):
@ -399,12 +418,14 @@ def test_azure_glob_scheherazade(cluster):
values = "(1, 1, 1)"
nights_per_job = 1001 // 30
jobs = []
used_names = []
for night in range(0, 1001, nights_per_job):
def add_tales(start, end):
for i in range(start, end):
path = "night_{}/tale.csv".format(i)
unique_num = random.randint(1, 10000)
used_names.append(f"test_scheherazade_{i}_{unique_num}")
azure_query(
node,
f"CREATE TABLE test_scheherazade_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, "
@ -432,6 +453,9 @@ def test_azure_glob_scheherazade(cluster):
)
query = "select count(), sum(column1), sum(column2), sum(column3) from test_glob_select_scheherazade"
assert azure_query(node, query).splitlines() == ["1001\t1001\t1001\t1001"]
azure_query(node, "DROP TABLE test_glob_select_scheherazade")
for name in used_names:
azure_query(node, f"DROP TABLE {name}")
@pytest.mark.parametrize(
@ -505,6 +529,8 @@ def test_schema_inference_no_globs(cluster):
assert azure_query(node, query).splitlines() == [
"499500\t2890\t332833500\ttest_schema_inference_no_globs.csv\tcont/test_schema_inference_no_globs.csv"
]
azure_query(node, f"DROP TABLE test_schema_inference_src")
azure_query(node, f"DROP TABLE test_select_inference")
def test_schema_inference_from_globs(cluster):
@ -513,6 +539,7 @@ def test_schema_inference_from_globs(cluster):
node = cluster.instances["node"] # type: ClickHouseInstance
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
max_path = ""
used_names = []
for i in range(10):
for j in range(10):
path = "{}/{}_{}/{}.csv".format(
@ -520,6 +547,7 @@ def test_schema_inference_from_globs(cluster):
)
max_path = max(path, max_path)
values = f"({i},{j},{i + j})"
used_names.append(f"test_schema_{i}_{j}")
azure_query(
node,
@ -545,6 +573,9 @@ def test_schema_inference_from_globs(cluster):
bucket="cont", max_path=max_path
)
]
azure_query(node, "DROP TABLE test_glob_select_inference")
for name in used_names:
azure_query(node, f"DROP TABLE {name}")
def test_simple_write_account_string_table_function(cluster):
@ -594,7 +625,7 @@ def test_simple_write_named_collection_1_table_function(cluster):
azure_query(
node,
"TRUNCATE TABLE drop_table",
"DROP TABLE drop_table",
)
@ -605,6 +636,7 @@ def test_simple_write_named_collection_2_table_function(cluster):
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}',"
f" container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')",
settings={"azure_truncate_on_insert": 1},
)
print(get_azure_file_content("test_simple_write_named_2_tf.csv", port))
assert get_azure_file_content("test_simple_write_named_2_tf.csv", port) == '1,"a"\n'
@ -628,6 +660,7 @@ def test_put_get_with_globs_tf(cluster):
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}',"
f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}",
settings={"azure_truncate_on_insert": 1},
)
query = (
f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, "
@ -648,7 +681,7 @@ def test_schema_inference_no_globs_tf(cluster):
query = (
f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', "
f"container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') "
f"SELECT number, toString(number), number * number FROM numbers(1000)"
f"SELECT number, toString(number), number * number FROM numbers(1000) SETTINGS azure_truncate_on_insert=1"
)
azure_query(node, query)
@ -681,7 +714,7 @@ def test_schema_inference_from_globs_tf(cluster):
f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', "
f"container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}"
)
azure_query(node, query)
azure_query(node, query, settings={"azure_truncate_on_insert": 1})
query = (
f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, "
@ -708,6 +741,7 @@ def test_partition_by_tf(cluster):
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', "
f"'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', "
f"'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}",
settings={"azure_truncate_on_insert": 1},
)
assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv", port)
@ -727,6 +761,7 @@ def test_filter_using_file(cluster):
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont', '{filename}', "
f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', "
f"'{table_format}') PARTITION BY {partition_by} VALUES {values}",
settings={"azure_truncate_on_insert": 1},
)
query = (
@ -744,7 +779,7 @@ def test_read_subcolumns(cluster):
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv', "
f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto',"
f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)",
f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS azure_truncate_on_insert=1",
)
azure_query(
@ -794,7 +829,7 @@ def test_read_subcolumn_time(cluster):
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumn_time.tsv', "
f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto',"
f" 'a UInt32') select (42)",
f" 'a UInt32') select (42) SETTINGS azure_truncate_on_insert=1",
)
res = node.query(
@ -825,6 +860,7 @@ def test_function_signatures(cluster):
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32') VALUES (1),(2),(3)",
settings={"azure_truncate_on_insert": 1},
)
# " - connection_string, container_name, blobpath\n"
@ -939,11 +975,13 @@ def test_union_schema_inference_mode(cluster):
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference1.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'a UInt32') VALUES (1)",
settings={"azure_truncate_on_insert": 1},
)
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'b UInt32') VALUES (2)",
settings={"azure_truncate_on_insert": 1},
)
node.query("system drop schema cache for azure")
@ -981,6 +1019,7 @@ def test_union_schema_inference_mode(cluster):
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference3.jsonl', '{account_name}', '{account_key}', 'CSV', 'auto', 's String') VALUES ('Error')",
settings={"azure_truncate_on_insert": 1},
)
error = azure_query(
@ -1002,7 +1041,7 @@ def test_schema_inference_cache(cluster):
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.jsonl', '{account_name}', '{account_key}') "
f"select * from numbers(100)",
f"select * from numbers(100) SETTINGS azure_truncate_on_insert=1",
)
time.sleep(1)
@ -1209,19 +1248,19 @@ def test_filtering_by_file_or_path(cluster):
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_filter1.tsv', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1",
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1 SETTINGS azure_truncate_on_insert=1",
)
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_filter2.tsv', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 2",
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 2 SETTINGS azure_truncate_on_insert=1",
)
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_filter3.tsv', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 3",
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 3 SETTINGS azure_truncate_on_insert=1",
)
node.query(
@ -1233,7 +1272,7 @@ def test_filtering_by_file_or_path(cluster):
node.query("SYSTEM FLUSH LOGS")
result = node.query(
f"SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query ilike '%select%azure%test_filter%' AND type='QueryFinish'"
f"SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query ilike '%select%azure%test_filter%' AND type='QueryFinish' ORDER BY event_time_microseconds DESC LIMIT 1"
)
assert int(result) == 1
@ -1245,19 +1284,19 @@ def test_size_virtual_column(cluster):
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_size_virtual_column1.tsv', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1",
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1 SETTINGS azure_truncate_on_insert=1",
)
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_size_virtual_column2.tsv', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 11",
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 11 SETTINGS azure_truncate_on_insert=1",
)
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_size_virtual_column3.tsv', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 111",
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 111 SETTINGS azure_truncate_on_insert=1",
)
result = azure_query(
@ -1280,7 +1319,7 @@ def test_format_detection(cluster):
account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt64, y String') select number as x, 'str_' || toString(number) from numbers(0)",
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt64, y String') select number as x, 'str_' || toString(number) from numbers(0) SETTINGS azure_truncate_on_insert=1",
)
azure_query(
@ -1350,7 +1389,7 @@ def test_write_to_globbed_partitioned_path(cluster):
account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="
error = azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_data_*_{{_partition_id}}', '{account_name}', '{account_key}', 'CSV', 'auto', 'x UInt64') partition by 42 select 42",
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_data_*_{{_partition_id}}', '{account_name}', '{account_key}', 'CSV', 'auto', 'x UInt64') partition by 42 select 42 SETTINGS azure_truncate_on_insert=1",
expect_error="true",
)
@ -1462,3 +1501,115 @@ def test_insert_create_new_file(cluster):
assert TSV(res) == TSV(
"test_create_new_file.csv\t1\ntest_create_new_file.1.csv\t2\n"
)
def test_hive_partitioning_with_one_parameter(cluster):
# type: (ClickHouseCluster) -> None
node = cluster.instances["node"] # type: ClickHouseInstance
table_format = "column1 String, column2 String"
values = f"('Elizabeth', 'Gordon')"
path = "a/column1=Elizabeth/sample.csv"
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}',"
f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}",
settings={"azure_truncate_on_insert": 1},
)
query = (
f"SELECT column1, column2, _file, _path, _column1 FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}')"
)
assert azure_query(
node, query, settings={"use_hive_partitioning": 1}
).splitlines() == [
"Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format(
bucket="cont", max_path=path
)
]
query = (
f"SELECT column2 FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;"
)
assert azure_query(
node, query, settings={"use_hive_partitioning": 1}
).splitlines() == ["Gordon"]
def test_hive_partitioning_with_two_parameters(cluster):
# type: (ClickHouseCluster) -> None
node = cluster.instances["node"] # type: ClickHouseInstance
table_format = "column1 String, column2 String"
values_1 = f"('Elizabeth', 'Gordon')"
values_2 = f"('Emilia', 'Gregor')"
path = "a/column1=Elizabeth/column2=Gordon/sample.csv"
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}',"
f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}",
settings={"azure_truncate_on_insert": 1},
)
query = (
f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;"
)
assert azure_query(
node, query, settings={"use_hive_partitioning": 1}
).splitlines() == [
"Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format(
bucket="cont", max_path=path
)
]
query = (
f"SELECT column1 FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;"
)
assert azure_query(
node, query, settings={"use_hive_partitioning": 1}
).splitlines() == ["Elizabeth"]
query = (
f"SELECT column1 FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;"
)
assert azure_query(
node, query, settings={"use_hive_partitioning": 1}
).splitlines() == ["Elizabeth"]
def test_hive_partitioning_without_setting(cluster):
# type: (ClickHouseCluster) -> None
node = cluster.instances["node"] # type: ClickHouseInstance
table_format = "column1 String, column2 String"
values_1 = f"('Elizabeth', 'Gordon')"
values_2 = f"('Emilia', 'Gregor')"
path = "a/column1=Elizabeth/column2=Gordon/sample.csv"
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}',"
f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}",
settings={"azure_truncate_on_insert": 1},
)
query = (
f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;"
)
pattern = re.compile(
r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL
)
with pytest.raises(Exception, match=pattern):
azure_query(node, query, settings={"use_hive_partitioning": 0})

View File

@ -72,6 +72,7 @@ def test_select_all(cluster):
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1',"
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') "
f"VALUES (1, 'a'), (2, 'b')",
settings={"azure_truncate_on_insert": 1},
)
print(get_azure_file_content("test_cluster_select_all.csv", port))
@ -101,6 +102,7 @@ def test_count(cluster):
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', "
f"'auto', 'key UInt64') VALUES (1), (2)",
settings={"azure_truncate_on_insert": 1},
)
print(get_azure_file_content("test_cluster_count.csv", port))
@ -129,6 +131,7 @@ def test_union_all(cluster):
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_parquet_union_all', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', "
f"'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')",
settings={"azure_truncate_on_insert": 1},
)
pure_azure = azure_query(
@ -180,6 +183,7 @@ def test_skip_unavailable_shards(cluster):
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_skip_unavailable.csv', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', "
f"'auto', 'a UInt64') VALUES (1), (2)",
settings={"azure_truncate_on_insert": 1},
)
result = azure_query(
node,
@ -200,6 +204,7 @@ def test_unset_skip_unavailable_shards(cluster):
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_unset_skip_unavailable.csv', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', "
f"'auto', 'a UInt64') VALUES (1), (2)",
settings={"azure_truncate_on_insert": 1},
)
result = azure_query(
node,
@ -218,6 +223,7 @@ def test_cluster_with_named_collection(cluster):
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_with_named_collection.csv', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', "
f"'auto', 'a UInt64') VALUES (1), (2)",
settings={"azure_truncate_on_insert": 1},
)
pure_azure = azure_query(
@ -249,6 +255,7 @@ def test_partition_parallel_reading_with_cluster(cluster):
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', '{filename}', 'devstoreaccount1', "
f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') "
f"PARTITION BY {partition_by} VALUES {values}",
settings={"azure_truncate_on_insert": 1},
)
assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv", port)
@ -272,12 +279,12 @@ def test_format_detection(cluster):
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10)",
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10) SETTINGS azure_truncate_on_insert=1",
)
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10, 10)",
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10, 10) SETTINGS azure_truncate_on_insert=1",
)
expected_desc_result = azure_query(

View File

@ -3,7 +3,9 @@ import os
import pytest
import uuid
import time
import re
from helpers.cluster import ClickHouseCluster, is_arm
from helpers.client import QueryRuntimeException
from helpers.test_tools import TSV
from pyhdfs import HdfsClient
@ -1255,6 +1257,55 @@ def test_respect_object_existence_on_partitioned_write(started_cluster):
assert int(result) == 44
def test_hive_partitioning_with_one_parameter(started_cluster):
hdfs_api = started_cluster.hdfs_api
hdfs_api.write_data(f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n")
assert hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") == f"Elizabeth\tGordon\n"
r = node1.query(
"SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')",
settings={"use_hive_partitioning": 1},
)
assert r == f"Elizabeth\n"
def test_hive_partitioning_with_two_parameters(started_cluster):
hdfs_api = started_cluster.hdfs_api
hdfs_api.write_data(
f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n"
)
assert (
hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2")
== f"Elizabeth\tGordon\n"
)
r = node1.query(
"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');",
settings={"use_hive_partitioning": 1},
)
assert r == f"Gordon\n"
def test_hive_partitioning_without_setting(started_cluster):
hdfs_api = started_cluster.hdfs_api
hdfs_api.write_data(
f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n"
)
assert (
hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2")
== f"Elizabeth\tGordon\n"
)
pattern = re.compile(
r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL
)
with pytest.raises(QueryRuntimeException, match=pattern):
node1.query(
f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');",
settings={"use_hive_partitioning": 0},
)
if __name__ == "__main__":
cluster.start()
input("Cluster created, press any key to destroy...")

View File

@ -0,0 +1,6 @@
@0x99f75f775fe63dae;
struct StringKeyValuePair {
key@0 : Text;
value@1 : Text;
}

View File

@ -0,0 +1 @@
(key = ${key:CSV}, value = ${value:CSV})

View File

@ -0,0 +1,6 @@
syntax = "proto3";
message StringKeyValuePair {
string key = 1;
string value = 2;
}

View File

@ -1019,7 +1019,7 @@ def test_kafka_formats(kafka_cluster, create_query_generator):
DROP TABLE IF EXISTS test.kafka_{format_name}_mv;
CREATE MATERIALIZED VIEW test.kafka_{format_name}_mv Engine=Log AS
CREATE MATERIALIZED VIEW test.kafka_{format_name}_mv ENGINE=MergeTree ORDER BY tuple() AS
SELECT *, _topic, _partition, _offset FROM test.kafka_{format_name};
""".format(
topic_name=topic_name,
@ -2460,7 +2460,7 @@ def test_kafka_commit_on_block_write(kafka_cluster, create_query_generator):
(generate_old_create_table_query, "kafka.*Committed offset 2.*virt2_[01]"),
(
generate_new_create_table_query,
r"kafka.*Saved offset 2[0-9]* for topic-partition \[virt2_[01]:[0-9]+",
r"kafka.*Saved offset 2 for topic-partition \[virt2_[01]:[0-9]+",
),
],
)
@ -2494,7 +2494,7 @@ def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line)
f"""
{create_query};
CREATE MATERIALIZED VIEW test.view Engine=Log AS
CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY tuple() AS
SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka;
"""
)
@ -2729,7 +2729,7 @@ def test_kafka_produce_key_timestamp(kafka_cluster, create_query_generator, log_
DROP TABLE IF EXISTS test.consumer;
{writer_create_query};
{reader_create_query};
CREATE MATERIALIZED VIEW test.view Engine=Log AS
CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY tuple() AS
SELECT key, value, inserted_key, toUnixTimestamp(inserted_timestamp), _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp) FROM test.kafka;
"""
)
@ -2865,7 +2865,7 @@ def test_kafka_produce_consume_avro(kafka_cluster, create_query_generator):
{writer_create_query};
{reader_create_query};
CREATE MATERIALIZED VIEW test.view Engine=Log AS
CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY tuple() AS
SELECT key, value FROM test.kafka;
"""
)
@ -3537,7 +3537,7 @@ def test_bad_reschedule(kafka_cluster, create_query_generator):
f"""
{create_query};
CREATE MATERIALIZED VIEW test.destination Engine=Log AS
CREATE MATERIALIZED VIEW test.destination ENGINE=MergeTree ORDER BY tuple() AS
SELECT
key,
now() as consume_ts,
@ -3745,7 +3745,7 @@ def test_kafka_unavailable(kafka_cluster, create_query_generator, do_direct_read
f"""
{create_query};
CREATE MATERIALIZED VIEW test.destination_unavailable Engine=Log AS
CREATE MATERIALIZED VIEW test.destination_unavailable ENGINE=MergeTree ORDER BY tuple() AS
SELECT
key,
now() as consume_ts,
@ -4267,12 +4267,12 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator
{create_query};
DROP TABLE IF EXISTS test.kafka_data_{format_name}_mv;
CREATE MATERIALIZED VIEW test.kafka_data_{format_name}_mv Engine=Log AS
CREATE MATERIALIZED VIEW test.kafka_data_{format_name}_mv ENGINE=MergeTree ORDER BY tuple() AS
SELECT *, _topic, _partition, _offset FROM test.kafka_{format_name}
WHERE length(_error) = 0;
DROP TABLE IF EXISTS test.kafka_errors_{format_name}_mv;
CREATE MATERIALIZED VIEW test.kafka_errors_{format_name}_mv Engine=Log AS
CREATE MATERIALIZED VIEW test.kafka_errors_{format_name}_mv ENGINE=MergeTree ORDER BY tuple() AS
SELECT {raw_message} as raw_message, _error as error, _topic as topic, _partition as partition, _offset as offset FROM test.kafka_{format_name}
WHERE length(_error) > 0;
"""
@ -4796,7 +4796,7 @@ def test_max_rows_per_message(kafka_cluster, create_query_generator):
DROP TABLE IF EXISTS test.kafka;
{create_query};
CREATE MATERIALIZED VIEW test.view Engine=Log AS
CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS
SELECT key, value FROM test.kafka;
"""
)
@ -4875,7 +4875,7 @@ def test_row_based_formats(kafka_cluster, create_query_generator):
{create_query};
CREATE MATERIALIZED VIEW test.view Engine=Log AS
CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS
SELECT key, value FROM test.{table_name};
INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows});
@ -4982,7 +4982,7 @@ def test_block_based_formats_2(kafka_cluster, create_query_generator):
{create_query};
CREATE MATERIALIZED VIEW test.view Engine=Log AS
CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS
SELECT key, value FROM test.{table_name};
INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0;
@ -5362,7 +5362,7 @@ def test_formats_errors(kafka_cluster):
input_format_with_names_use_header=0,
format_schema='key_value_message:Message';
CREATE MATERIALIZED VIEW test.view Engine=Log AS
CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS
SELECT key, value FROM test.{table_name};
"""
)

View File

@ -0,0 +1,243 @@
import time
import logging
import pytest
from helpers.cluster import ClickHouseCluster, is_arm
from helpers.test_tools import TSV
from kafka import KafkaAdminClient
from kafka.admin import NewTopic
if is_arm():
pytestmark = pytest.mark.skip
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance(
"instance",
main_configs=["configs/kafka.xml", "configs/named_collection.xml"],
user_configs=["configs/users.xml"],
with_kafka=True,
with_zookeeper=True, # For Replicated Table
macros={
"kafka_broker": "kafka1",
"kafka_topic_old": "old",
"kafka_group_name_old": "old",
"kafka_topic_new": "new",
"kafka_group_name_new": "new",
"kafka_client_id": "instance",
"kafka_format_json_each_row": "JSONEachRow",
},
clickhouse_path_dir="clickhouse_path",
)
@pytest.fixture(scope="module")
def kafka_cluster():
try:
cluster.start()
kafka_id = instance.cluster.kafka_docker_id
print(("kafka_id is {}".format(kafka_id)))
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def kafka_setup_teardown():
instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;")
# logging.debug("kafka is available - running test")
yield # run test
def kafka_create_topic(
admin_client,
topic_name,
num_partitions=1,
replication_factor=1,
max_retries=50,
config=None,
):
logging.debug(
f"Kafka create topic={topic_name}, num_partitions={num_partitions}, replication_factor={replication_factor}"
)
topics_list = [
NewTopic(
name=topic_name,
num_partitions=num_partitions,
replication_factor=replication_factor,
topic_configs=config,
)
]
retries = 0
while True:
try:
admin_client.create_topics(new_topics=topics_list, validate_only=False)
logging.debug("Admin client succeed")
return
except Exception as e:
retries += 1
time.sleep(0.5)
if retries < max_retries:
logging.warning(f"Failed to create topic {e}")
else:
raise
def kafka_delete_topic(admin_client, topic, max_retries=50):
result = admin_client.delete_topics([topic])
for topic, e in result.topic_error_codes:
if e == 0:
logging.debug(f"Topic {topic} deleted")
else:
logging.error(f"Failed to delete topic {topic}: {e}")
retries = 0
while True:
topics_listed = admin_client.list_topics()
logging.debug(f"TOPICS LISTED: {topics_listed}")
if topic not in topics_listed:
return
else:
retries += 1
time.sleep(0.5)
if retries > max_retries:
raise Exception(f"Failed to delete topics {topic}, {result}")
def test_kafka_produce_http_interface_row_based_format(kafka_cluster):
# reproduction of #61060 with validating the written messages
admin_client = KafkaAdminClient(
bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)
)
topic_prefix = "http_row_"
# It is important to have:
# - long enough messages
# - enough messages
# I don't know the exact requirement for message sizes, but it doesn't reproduce with short messages
# For the number of messages it seems like at least 3 messages is necessary
expected_key = "01234567890123456789"
expected_value = "aaaaabbbbbccccc"
insert_query_end = f"(key, value) VALUES ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}')"
insert_query_template = "INSERT INTO {table_name} " + insert_query_end
extra_settings = {
"Protobuf": ", kafka_schema = 'string_key_value.proto:StringKeyValuePair'",
"CapnProto": ", kafka_schema='string_key_value:StringKeyValuePair'",
"Template": ", format_template_row='string_key_value.format'",
}
# Only the formats that can be used both and input and output format are tested
# Reasons to exclude following formats:
# - JSONStrings: not actually an input format
# - ProtobufSingle: I cannot make it work to parse the messages. Probably something is broken,
# because the producer can write multiple rows into a same message, which makes them impossible to parse properly. Should added after #67549 is fixed.
# - ProtobufList: I didn't want to deal with the envelope and stuff
# - Npy: supports only single column
# - LineAsString: supports only single column
# - RawBLOB: supports only single column
formats_to_test = [
"TabSeparated",
"TabSeparatedRaw",
"TabSeparatedWithNames",
"TabSeparatedWithNamesAndTypes",
"TabSeparatedRawWithNames",
"TabSeparatedRawWithNamesAndTypes",
"Template",
"CSV",
"CSVWithNames",
"CSVWithNamesAndTypes",
"CustomSeparated",
"CustomSeparatedWithNames",
"CustomSeparatedWithNamesAndTypes",
"Values",
"JSON",
"JSONColumns",
"JSONColumnsWithMetadata",
"JSONCompact",
"JSONCompactColumns",
"JSONEachRow",
"JSONStringsEachRow",
"JSONCompactEachRow",
"JSONCompactEachRowWithNames",
"JSONCompactEachRowWithNamesAndTypes",
"JSONCompactStringsEachRow",
"JSONCompactStringsEachRowWithNames",
"JSONCompactStringsEachRowWithNamesAndTypes",
"JSONObjectEachRow",
"BSONEachRow",
"TSKV",
"Protobuf",
"Avro",
"Parquet",
"Arrow",
"ArrowStream",
"ORC",
"RowBinary",
"RowBinaryWithNames",
"RowBinaryWithNamesAndTypes",
"Native",
"CapnProto",
"MsgPack",
]
for format in formats_to_test:
logging.debug(f"Creating tables and writing messages to {format}")
topic = topic_prefix + format
kafka_create_topic(admin_client, topic)
extra_setting = extra_settings.get(format, "")
# kafka_max_rows_per_message is set to 2 to make sure every format produces at least 2 messages, thus increasing the chance of catching a bug
instance.query(
f"""
DROP TABLE IF EXISTS test.view_{topic};
DROP TABLE IF EXISTS test.consumer_{topic};
CREATE TABLE test.kafka_writer_{topic} (key String, value String)
ENGINE = Kafka
SETTINGS kafka_broker_list = 'kafka1:19092',
kafka_topic_list = '{topic}',
kafka_group_name = '{topic}',
kafka_format = '{format}',
kafka_max_rows_per_message = 2 {extra_setting};
CREATE TABLE test.kafka_{topic} (key String, value String)
ENGINE = Kafka
SETTINGS kafka_broker_list = 'kafka1:19092',
kafka_topic_list = '{topic}',
kafka_group_name = '{topic}',
kafka_format = '{format}' {extra_setting};
CREATE MATERIALIZED VIEW test.view_{topic} Engine=Log AS
SELECT key, value FROM test.kafka_{topic};
"""
)
instance.http_query(
insert_query_template.format(table_name="test.kafka_writer_" + topic),
method="POST",
)
expected = f"""\
{expected_key}\t{expected_value}
{expected_key}\t{expected_value}
{expected_key}\t{expected_value}
"""
# give some times for the readers to read the messages
for format in formats_to_test:
logging.debug(f"Checking result for {format}")
topic = topic_prefix + format
result = instance.query_with_retry(
f"SELECT * FROM test.view_{topic}",
check_callback=lambda res: res.count("\n") == 3,
)
assert TSV(result) == TSV(expected)
kafka_delete_topic(admin_client, topic)
if __name__ == "__main__":
cluster.start()
input("Cluster created, press any key to destroy...")
cluster.shutdown()

View File

@ -0,0 +1,37 @@
#!/usr/bin/env bash
# Tags: no-random-merge-tree-settings, no-random-settings
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
sql="toUInt16OrNull(arrayFirst((v, k) -> (k = '4Id'), arr[2], arr[1]))"
# Create the table and fill it
$CLICKHOUSE_CLIENT -n --query="
CREATE TABLE test_extract(str String, arr Array(Array(String)) ALIAS extractAllGroupsHorizontal(str, '\\W(\\w+)=(\"[^\"]*?\"|[^\",}]*)')) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY tuple();
INSERT INTO test_extract (str) WITH range(8) as range_arr, arrayMap(x-> concat(toString(x),'Id'), range_arr) as key, arrayMap(x -> rand() % 8, range_arr) as val, arrayStringConcat(arrayMap((x,y) -> concat(x,'=',toString(y)), key, val),',') as str SELECT str FROM numbers(500000);
ALTER TABLE test_extract ADD COLUMN 15Id Nullable(UInt16) DEFAULT $sql;"
function test()
{
# Execute two queries and compare if they have similar memory usage:
# The first query uses the default column value, while the second explicitly uses the same SQL as the default value.
# Follow https://github.com/ClickHouse/ClickHouse/issues/17317 for more info about the issue
where=$1
uuid_1=$(cat /proc/sys/kernel/random/uuid)
$CLICKHOUSE_CLIENT --query="SELECT uniq(15Id) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_1
uuid_2=$(cat /proc/sys/kernel/random/uuid)
$CLICKHOUSE_CLIENT --query="SELECT uniq($sql) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_2
$CLICKHOUSE_CLIENT -n --query="
SYSTEM FLUSH LOGS;
WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1),
memory_2 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2)
SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage OR
memory_2.memory_usage <= 1.2 * memory_1.memory_usage FROM memory_1, memory_2;"
}
test ""
test "PREWHERE 15Id < 4"
test "WHERE 15Id < 4"

View File

@ -1,13 +0,0 @@
-- Tags: no-random-merge-tree-settings
CREATE TABLE test_extract(str String, arr Array(Array(String)) ALIAS extractAllGroupsHorizontal(str, '\\W(\\w+)=("[^"]*?"|[^",}]*)')) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY tuple();
INSERT INTO test_extract (str) WITH range(8) as range_arr, arrayMap(x-> concat(toString(x),'Id'), range_arr) as key, arrayMap(x -> rand() % 8, range_arr) as val, arrayStringConcat(arrayMap((x,y) -> concat(x,'=',toString(y)), key, val),',') as str SELECT str FROM numbers(500000);
ALTER TABLE test_extract ADD COLUMN `15Id` Nullable(UInt16) DEFAULT toUInt16OrNull(arrayFirst((v, k) -> (k = '4Id'), arr[2], arr[1]));
SELECT uniq(15Id) FROM test_extract SETTINGS max_threads=1, max_memory_usage=100000000;
SELECT uniq(15Id) FROM test_extract PREWHERE 15Id < 4 SETTINGS max_threads=1, max_memory_usage=100000000;
SELECT uniq(15Id) FROM test_extract WHERE 15Id < 4 SETTINGS max_threads=1, max_memory_usage=100000000;

View File

@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# $2 - query
function execute_query()
{
${CLICKHOUSE_CLIENT} --opentelemetry_start_trace_probability=1 --query_id $1 -nq "
${CLICKHOUSE_CLIENT} --opentelemetry_start_trace_probability=1 --query_id $1 -q "
${2}
"
}
@ -18,7 +18,7 @@ function execute_query()
# so we only to check the db.statement only
function check_query_span_query_only()
{
${CLICKHOUSE_CLIENT} -nq "
${CLICKHOUSE_CLIENT} -q "
SYSTEM FLUSH LOGS;
SELECT attribute['db.statement'] as query
FROM system.opentelemetry_span_log
@ -31,7 +31,7 @@ ${CLICKHOUSE_CLIENT} -nq "
function check_query_span()
{
${CLICKHOUSE_CLIENT} -nq "
${CLICKHOUSE_CLIENT} -q "
SYSTEM FLUSH LOGS;
SELECT attribute['db.statement'] as query,
attribute['clickhouse.read_rows'] as read_rows,
@ -47,7 +47,7 @@ ${CLICKHOUSE_CLIENT} -nq "
#
# Set up
#
${CLICKHOUSE_CLIENT} -nq "
${CLICKHOUSE_CLIENT} -q "
DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.opentelemetry_test;
CREATE TABLE ${CLICKHOUSE_DATABASE}.opentelemetry_test (id UInt64) Engine=MergeTree Order By id;
"
@ -79,4 +79,4 @@ check_query_span $query_id
#
${CLICKHOUSE_CLIENT} -q "
DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.opentelemetry_test;
"
"

View File

@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -nm -q "
$CLICKHOUSE_CLIENT -m -q "
drop table if exists insert_select_progress_tcp;
create table insert_select_progress_tcp(s UInt16) engine = MergeTree order by s;
"

View File

@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -n -q "
$CLICKHOUSE_CLIENT -q "
DROP TABLE IF EXISTS test_table;
CREATE TABLE test_table
(
@ -74,7 +74,7 @@ $CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, arrayMap(lambda_argument -> l
$CLICKHOUSE_CLIENT -q "WITH 1 AS constant_value SELECT (SELECT constant_valu) SETTINGS enable_analyzer = 1;" 2>&1 \
| grep "Maybe you meant: \['constant_value'\]" &>/dev/null;
$CLICKHOUSE_CLIENT -n -q "
$CLICKHOUSE_CLIENT -q "
DROP TABLE IF EXISTS test_table_compound;
CREATE TABLE test_table_compound
(
@ -142,7 +142,7 @@ $CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constan
$CLICKHOUSE_CLIENT -q "WITH cast(tuple(1), 'Tuple(value_1 String)') AS constant_value SELECT (SELECT constant_value.value_) SETTINGS enable_analyzer = 1;" 2>&1 \
| grep "Maybe you meant: \['constant_value.value_1'\]" &>/dev/null;
$CLICKHOUSE_CLIENT -n -q "
$CLICKHOUSE_CLIENT -q "
DROP TABLE IF EXISTS test_table_1;
CREATE TABLE test_table_1
(
@ -185,7 +185,7 @@ $CLICKHOUSE_CLIENT -q "SELECT ((1))::Tuple(a Tuple(b UInt32)) AS t, t.a.c SETTIN
$CLICKHOUSE_CLIENT -q "SELECT 1";
$CLICKHOUSE_CLIENT -n -q "
$CLICKHOUSE_CLIENT -q "
DROP TABLE test_table;
DROP TABLE test_table_compound;
DROP TABLE test_table_1;

View File

@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
database=$($CLICKHOUSE_CLIENT -q 'SELECT currentDatabase()')
$CLICKHOUSE_CLIENT -nm -q "
$CLICKHOUSE_CLIENT -m -q "
DROP TABLE IF EXISTS test_02480_table;
DROP VIEW IF EXISTS test_02480_view;
CREATE TABLE test_02480_table (id Int64) ENGINE=MergeTree ORDER BY id;

View File

@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -n --query "
$CLICKHOUSE_CLIENT --query "
DROP TABLE IF EXISTS load_parts_refcounts SYNC;
CREATE TABLE load_parts_refcounts (id UInt32)

View File

@ -0,0 +1,12 @@
1
SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = \'abc\' abc
---
1
1
SELECT 1 SETTINGS use_query_cache = true
SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = \'abc\' abc
---
1
1
SELECT 1 SETTINGS use_query_cache = true abc
SELECT 1 SETTINGS use_query_cache = true def

View File

@ -0,0 +1,34 @@
-- Tags: no-parallel
-- Tag no-parallel: Messes with internal cache
SYSTEM DROP QUERY CACHE;
-- Store the result a single query with a tag in the query cache and check that the system table knows about the tag
SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'abc';
SELECT query, tag FROM system.query_cache;
SELECT '---';
SYSTEM DROP QUERY CACHE;
-- Store the result of the same query with two different tags. The cache should store two entries.
SELECT 1 SETTINGS use_query_cache = true; -- default query_cache_tag = ''
SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'abc';
SELECT query, tag FROM system.query_cache ORDER BY ALL;
SELECT '---';
SYSTEM DROP QUERY CACHE;
-- Like before but the tag is set standalone.
SET query_cache_tag = 'abc';
SELECT 1 SETTINGS use_query_cache = true;
SET query_cache_tag = 'def';
SELECT 1 SETTINGS use_query_cache = true;
SELECT query, tag FROM system.query_cache ORDER BY ALL;
SYSTEM DROP QUERY CACHE;

View File

@ -26,15 +26,15 @@ FROM
ORDER BY number DESC
)
ORDER BY number ASC"
$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;EXPLAIN $query"
$CLICKHOUSE_CLIENT -q "$DISABLE_OPTIMIZATION;EXPLAIN $query"
function run_query {
echo "-- query"
echo "$1"
echo "-- explain"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;EXPLAIN $1"
$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;EXPLAIN $1"
echo "-- execute"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$1"
$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;$1"
}
echo "-- Enabled query_plan_remove_redundant_sorting"

View File

@ -13,6 +13,6 @@ $CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SEL
$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferMMap" && echo 'Fail' || echo 0
$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferOrdinary" && echo 1 || echo 'Fail'
$CLICKHOUSE_CLIENT --storage_file_read_method=mmap -nq "SELECT * FROM file('/dev/null', 'LineAsString') FORMAT Null -- { serverError BAD_ARGUMENTS }"
$CLICKHOUSE_CLIENT --storage_file_read_method=mmap -q "SELECT * FROM file('/dev/null', 'LineAsString') FORMAT Null -- { serverError BAD_ARGUMENTS }"
rm $DATA_FILE

View File

@ -24,15 +24,15 @@ FROM
)
)"
$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;EXPLAIN $query"
$CLICKHOUSE_CLIENT -q "$DISABLE_OPTIMIZATION;EXPLAIN $query"
function run_query {
echo "-- query"
echo "$1"
echo "-- explain"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;EXPLAIN $1"
$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;EXPLAIN $1"
echo "-- execute"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$1"
$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;$1"
}
echo "-- Enabled $OPTIMIZATION_SETTING"

View File

@ -14,8 +14,8 @@ ${CLICKHOUSE_CLIENT} -q "CREATE USER user_${CLICKHOUSE_DATABASE} settings databa
${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO user_${CLICKHOUSE_DATABASE}"
${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}"
${CLICKHOUSE_CLIENT} -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 80 }"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 80 }"
${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db"
${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}"

View File

@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -nm -q "
$CLICKHOUSE_CLIENT -m -q "
drop table if exists data;
create table data (key Int) engine=MergeTree order by tuple() settings min_bytes_for_wide_part = '1G', compress_marks = 1;
insert into data values (1);

View File

@ -46,7 +46,7 @@ tables["wrong_metadata_compact"]="min_bytes_for_wide_part = 10000000"
for table in "${!tables[@]}"; do
settings="${tables[$table]}"
$CLICKHOUSE_CLIENT -n --query="
$CLICKHOUSE_CLIENT --query="
DROP TABLE IF EXISTS $table;
CREATE TABLE $table(
@ -69,7 +69,7 @@ for table in "${!tables[@]}"; do
wait_column "$table" "\`a1\` UInt64" || exit 2
$CLICKHOUSE_CLIENT -n --query="
$CLICKHOUSE_CLIENT --query="
-- { echoOn }
SELECT 'ECHO_ALIGNMENT_FIX' FORMAT Null;
@ -82,7 +82,7 @@ for table in "${!tables[@]}"; do
wait_mutation_loaded "$table" "b1 TO a" || exit 2
$CLICKHOUSE_CLIENT -n --query="
$CLICKHOUSE_CLIENT --query="
-- { echoOn }
SELECT 'ECHO_ALIGNMENT_FIX' FORMAT Null;
@ -94,7 +94,7 @@ for table in "${!tables[@]}"; do
wait_for_all_mutations "$table"
$CLICKHOUSE_CLIENT -n --query="
$CLICKHOUSE_CLIENT --query="
-- { echoOn }
SELECT 'ECHO_ALIGNMENT_FIX' FORMAT Null;

View File

@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "drop table if exists buffer_02572;
${CLICKHOUSE_CLIENT} --ignore-error --query "drop table if exists buffer_02572;
drop table if exists data_02572; drop table if exists copy_02572; drop table if exists mv_02572;"
${CLICKHOUSE_CLIENT} --query="create table copy_02572 (key Int) engine=Memory();"
@ -21,7 +21,7 @@ ${CLICKHOUSE_CLIENT} --query="insert into buffer_02572 values (1);"
if [ $(( $(date +%s) - start )) -gt 6 ]; then # clickhouse test cluster is overloaded, will skip
# ensure that the flush was not direct
${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;"
${CLICKHOUSE_CLIENT} --ignore-error --query "select * from data_02572; select * from copy_02572;"
fi
# we cannot use OPTIMIZE, this will attach query context, so let's wait
@ -31,11 +31,11 @@ for _ in {1..100}; do
done
${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;"
${CLICKHOUSE_CLIENT} --ignore-error --query "select * from data_02572; select * from copy_02572;"
${CLICKHOUSE_CLIENT} --query="system flush logs;"
${CLICKHOUSE_CLIENT} --query="select count() > 0, lower(status::String), errorCodeToName(exception_code)
from system.query_views_log where
view_name = concatWithSeparator('.', currentDatabase(), 'mv_02572') and
view_target = concatWithSeparator('.', currentDatabase(), 'copy_02572')
group by 2, 3;"
group by 2, 3;"

View File

@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -nm -q "
$CLICKHOUSE_CLIENT -m -q "
drop table if exists data;
create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9;
"
@ -26,7 +26,7 @@ read_methods=(
for read_method in "${read_methods[@]}"; do
query_id=$(random_str 10)
$CLICKHOUSE_CLIENT --query_id "$query_id" -q "select * from data format Null settings max_local_read_bandwidth='1M', local_filesystem_read_method='$read_method'"
$CLICKHOUSE_CLIENT -nm -q "
$CLICKHOUSE_CLIENT -m -q "
SYSTEM FLUSH LOGS;
SELECT
'$read_method',

View File

@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -nm -q "
$CLICKHOUSE_CLIENT -m -q "
drop table if exists data;
create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9;
"
@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -nm -q "
query_id=$(random_str 10)
# writes 1e6*8 bytes with 1M bandwith it should take (8-1)/1=7 seconds
$CLICKHOUSE_CLIENT --query_id "$query_id" -q "insert into data select * from numbers(1e6) settings max_local_write_bandwidth='1M'"
$CLICKHOUSE_CLIENT -nm -q "
$CLICKHOUSE_CLIENT -m -q "
SYSTEM FLUSH LOGS;
SELECT
query_duration_ms >= 7e3,

View File

@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -nm -q "
$CLICKHOUSE_CLIENT -m -q "
drop table if exists data;
create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9;
"
@ -15,7 +15,7 @@ $CLICKHOUSE_CLIENT -q "insert into data select * from numbers(1e6)"
query_id=$(random_str 10)
$CLICKHOUSE_CLIENT --query_id "$query_id" -q "backup table data to Disk('backups', '$CLICKHOUSE_DATABASE/data/backup1')" --max_backup_bandwidth=1M > /dev/null
$CLICKHOUSE_CLIENT -nm -q "
$CLICKHOUSE_CLIENT -m -q "
SYSTEM FLUSH LOGS;
SELECT
query_duration_ms >= 7e3,

View File

@ -23,7 +23,7 @@ function wait_for_alter()
done
}
${CLICKHOUSE_CLIENT} -n --query "
${CLICKHOUSE_CLIENT} --query "
DROP TABLE IF EXISTS t_limit_mutations SYNC;
CREATE TABLE t_limit_mutations (id UInt64, v UInt64)
@ -48,14 +48,14 @@ SELECT count() FROM system.mutations WHERE database = currentDatabase() AND tabl
SHOW CREATE TABLE t_limit_mutations;
"
${CLICKHOUSE_CLIENT} -n --query "
${CLICKHOUSE_CLIENT} --query "
ALTER TABLE t_limit_mutations UPDATE v = 6 WHERE 1 SETTINGS number_of_mutations_to_throw = 100;
ALTER TABLE t_limit_mutations MODIFY COLUMN v String SETTINGS number_of_mutations_to_throw = 100, alter_sync = 0;
"
wait_for_alter "String"
${CLICKHOUSE_CLIENT} -n --query "
${CLICKHOUSE_CLIENT} --query "
SELECT * FROM t_limit_mutations ORDER BY id;
SELECT count() FROM system.mutations WHERE database = currentDatabase() AND table = 't_limit_mutations' AND NOT is_done;
SHOW CREATE TABLE t_limit_mutations;
@ -65,7 +65,7 @@ ${CLICKHOUSE_CLIENT} --query "SYSTEM START MERGES t_limit_mutations"
wait_for_mutation "t_limit_mutations" "0000000003"
${CLICKHOUSE_CLIENT} -n --query "
${CLICKHOUSE_CLIENT} --query "
SELECT * FROM t_limit_mutations ORDER BY id;
SELECT count() FROM system.mutations WHERE database = currentDatabase() AND table = 't_limit_mutations' AND NOT is_done;
SHOW CREATE TABLE t_limit_mutations;

View File

@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -n --query "
${CLICKHOUSE_CLIENT} --query "
DROP TABLE IF EXISTS t_mt_async_insert;
DROP TABLE IF EXISTS t_mt_sync_insert;
@ -19,7 +19,7 @@ url="${CLICKHOUSE_URL}&async_insert=0&wait_for_async_insert=1"
${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_mt_async_insert VALUES (1, 'aa'), (2, 'bb')"
${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_mt_sync_insert VALUES (1, 'aa'), (2, 'bb')"
${CLICKHOUSE_CLIENT} -n --query "
${CLICKHOUSE_CLIENT} --query "
SELECT count() FROM t_mt_async_insert;
SELECT count() FROM t_mt_sync_insert;

View File

@ -7,12 +7,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CUR_DIR"/../shell_config.sh
echo "create table test (x UInt64) engine=Memory;
insert into test from infile 'data'; -- {clientError BAD_ARGUMENTS}" | $CLICKHOUSE_LOCAL -nm
insert into test from infile 'data'; -- {clientError BAD_ARGUMENTS}" | $CLICKHOUSE_LOCAL -m
echo "create table test (x UInt64) engine=Memory;
insert into test from infile 'data';" | $CLICKHOUSE_LOCAL -nm --ignore-error
insert into test from infile 'data';" | $CLICKHOUSE_LOCAL -m --ignore-error
echo "create table test (x UInt64) engine=Memory;
insert into test from infile 'data'; -- {clientError BAD_ARGUMENTS}
select 1" | $CLICKHOUSE_LOCAL -nm
select 1" | $CLICKHOUSE_LOCAL -m

View File

@ -1,14 +0,0 @@
CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
Test statistics count_min:
Prewhere info
Prewhere filter
Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed)
Test statistics multi-types:
Prewhere info
Prewhere filter
Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed)
Test LowCardinality and Nullable data type:
tab2

View File

@ -1,70 +0,0 @@
-- Tags: no-fasttest
DROP TABLE IF EXISTS tab SYNC;
SET allow_experimental_statistics = 1;
SET allow_statistics_optimize = 1;
SET allow_suspicious_low_cardinality_types=1;
SET mutations_sync = 2;
CREATE TABLE tab
(
a String,
b UInt64,
c Int64,
pk String,
) Engine = MergeTree() ORDER BY pk
SETTINGS min_bytes_for_wide_part = 0;
SHOW CREATE TABLE tab;
INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000;
SELECT 'Test statistics count_min:';
ALTER TABLE tab ADD STATISTICS a TYPE count_min;
ALTER TABLE tab ADD STATISTICS b TYPE count_min;
ALTER TABLE tab ADD STATISTICS c TYPE count_min;
ALTER TABLE tab MATERIALIZE STATISTICS a, b, c;
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '')
FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx
WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
ALTER TABLE tab DROP STATISTICS a, b, c;
SELECT 'Test statistics multi-types:';
ALTER TABLE tab ADD STATISTICS a TYPE count_min;
ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest;
ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest;
ALTER TABLE tab MATERIALIZE STATISTICS a, b, c;
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '')
FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/)
WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '')
FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/)
WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
ALTER TABLE tab DROP STATISTICS a, b, c;
DROP TABLE IF EXISTS tab SYNC;
SELECT 'Test LowCardinality and Nullable data type:';
DROP TABLE IF EXISTS tab2 SYNC;
SET allow_suspicious_low_cardinality_types=1;
CREATE TABLE tab2
(
a LowCardinality(Int64) STATISTICS(count_min),
b Nullable(Int64) STATISTICS(count_min),
c LowCardinality(Nullable(Int64)) STATISTICS(count_min),
pk String,
) Engine = MergeTree() ORDER BY pk;
select name from system.tables where name = 'tab2' and database = currentDatabase();
DROP TABLE IF EXISTS tab2 SYNC;

View File

@ -1,31 +1,6 @@
CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After insert
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), less(b, 10)) (removed)
10
0
After drop statistic
Prewhere info
Prewhere filter
Prewhere filter column: and(less(b, 10), less(a, 10)) (removed)
10
CREATE TABLE default.tab\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After add statistic
CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After materialize statistic
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), less(b, 10)) (removed)
20
After merge
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), less(b, 10)) (removed)
20
CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After rename
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), less(c, 10)) (removed)
20
CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192

View File

@ -1,59 +1,195 @@
-- Tests that various DDL statements create/drop/materialize statistics
-- Tags: no-fasttest
-- no-fasttest: 'count_min' sketches need a 3rd party library
-- Tests that DDL statements which create / drop / materialize statistics
SET mutations_sync = 1;
DROP TABLE IF EXISTS tab;
-- Error case: Can't create statistics when allow_experimental_statistics = 0
CREATE TABLE tab (col Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY }
SET allow_experimental_statistics = 1;
SET allow_statistics_optimize = 1;
-- Error case: Unknown statistics types are rejected
CREATE TABLE tab (col Float64 STATISTICS(no_statistics_type)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY }
-- Error case: The same statistics type can't exist more than once on a column
CREATE TABLE tab (col Float64 STATISTICS(tdigest, tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY }
SET allow_suspicious_low_cardinality_types = 1;
-- Statistics can only be created on columns of specific data types (depending on the statistics kind), (*)
-- tdigest requires data_type.isValueRepresentedByInteger
-- These types work:
CREATE TABLE tab (col UInt8 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col UInt256 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Float32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Decimal32(3) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Date STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Date32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col DateTime STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col DateTime64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col IPv4 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Nullable(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
-- These types don't work:
CREATE TABLE tab (col String STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col FixedString(1) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Array(Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col UUID STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col IPv6 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
-- uniq requires data_type.isValueRepresentedByInteger
-- These types work:
CREATE TABLE tab (col UInt8 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col UInt256 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Float32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Decimal32(3) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Date STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Date32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col DateTime STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col DateTime64 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col IPv4 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Nullable(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
-- These types don't work:
CREATE TABLE tab (col String STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col FixedString(1) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Array(Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col UUID STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col IPv6 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
-- count_min requires data_type.isValueRepresentedByInteger or data_type = (Fixed)String
-- These types work:
CREATE TABLE tab (col UInt8 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col UInt256 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Float32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Decimal32(3) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Date STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Date32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col DateTime STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col DateTime64 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col IPv4 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col Nullable(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col String STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
CREATE TABLE tab (col FixedString(1) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab;
-- These types don't work:
CREATE TABLE tab (col Array(Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col UUID STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab (col IPv6 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
-- CREATE TABLE was easy, ALTER is more fun
CREATE TABLE tab
(
a Float64 STATISTICS(tdigest),
b Int64 STATISTICS(tdigest),
pk String,
) Engine = MergeTree() ORDER BY pk
SETTINGS min_bytes_for_wide_part = 0;
f64 Float64,
f64_tdigest Float64 STATISTICS(tdigest),
f32 Float32,
s String,
a Array(Float64)
)
Engine = MergeTree()
ORDER BY tuple();
-- Error case: Unknown statistics types are rejected
-- (relevant for ADD and MODIFY)
ALTER TABLE tab ADD STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY }
ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY }
ALTER TABLE tab MODIFY STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY }
-- for some reason, ALTER TABLE tab MODIFY STATISTICS IF EXISTS is not supported
-- Error case: The same statistics type can't exist more than once on a column
-- (relevant for ADD and MODIFY)
-- Create the same statistics object twice
ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY }
ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY }
ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY }
-- Create an statistics which exists already
ALTER TABLE tab ADD STATISTICS f64_tdigest TYPE tdigest; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64_tdigest TYPE tdigest; -- no-op
ALTER TABLE tab MODIFY STATISTICS f64_tdigest TYPE tdigest; -- no-op
-- Error case: Column does not exist
-- (relevant for ADD, MODIFY, DROP, CLEAR, and MATERIALIZE)
-- Note that the results are unfortunately quite inconsistent ...
ALTER TABLE tab ADD STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab ADD STATISTICS IF NOT EXISTS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab MODIFY STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab DROP STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab DROP STATISTICS IF EXISTS no_such_column; -- no-op
ALTER TABLE tab CLEAR STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab CLEAR STATISTICS IF EXISTS no_such_column; -- no-op
ALTER TABLE tab MATERIALIZE STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS no_such_column; -- { serverError ILLEGAL_STATISTICS }
-- Error case: Column exists but has no statistics
-- (relevant for MODIFY, DROP, CLEAR, and MATERIALIZE)
-- Note that the results are unfortunately quite inconsistent ...
ALTER TABLE tab MODIFY STATISTICS s TYPE tdigest; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab DROP STATISTICS s; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab DROP STATISTICS IF EXISTS s; -- no-op
ALTER TABLE tab CLEAR STATISTICS s; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab CLEAR STATISTICS IF EXISTS s; -- no-op
ALTER TABLE tab MATERIALIZE STATISTICS s; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS s; -- { serverError ILLEGAL_STATISTICS }
-- We don't check systematically that that statistics can only be created via ALTER ADD STATISTICS on columns of specific data types (the
-- internal type validation code is tested already above, (*)). Only do a rudimentary check for each statistics type with a data type that
-- works and one that doesn't work.
-- tdigest
-- Works:
ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64;
ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64;
-- Doesn't work:
ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS }
-- uniq
-- Works:
ALTER TABLE tab ADD STATISTICS f64 TYPE uniq; ALTER TABLE tab DROP STATISTICS f64;
ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64;
-- Doesn't work:
ALTER TABLE tab ADD STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab MODIFY STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS }
-- count_min
-- Works:
ALTER TABLE tab ADD STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64;
ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64;
-- Doesn't work:
ALTER TABLE tab ADD STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab MODIFY STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS }
-- Any data type changes on columns with statistics are disallowed, for simplicity even if the new data type is compatible with all existing
-- statistics objects (e.g. tdigest can be created on Float64 and UInt64)
ALTER TABLE tab MODIFY COLUMN f64_tdigest UInt64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN }
-- Finally, do a full-circle test of a good case. Print table definition after each step.
-- Intentionally specifying _two_ columns and _two_ statistics types to have that also tested.
SHOW CREATE TABLE tab;
ALTER TABLE tab ADD STATISTICS f64, f32 TYPE tdigest, uniq;
SHOW CREATE TABLE tab;
ALTER TABLE tab MODIFY STATISTICS f64, f32 TYPE tdigest, uniq;
SHOW CREATE TABLE tab;
ALTER TABLE tab CLEAR STATISTICS f64, f32;
SHOW CREATE TABLE tab;
ALTER TABLE tab MATERIALIZE STATISTICS f64, f32;
SHOW CREATE TABLE tab;
ALTER TABLE tab DROP STATISTICS f64, f32;
SHOW CREATE TABLE tab;
INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000;
SELECT 'After insert';
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT count(*) FROM tab WHERE b < 10 and a < 10;
SELECT count(*) FROM tab WHERE b < NULL and a < '10';
ALTER TABLE tab DROP STATISTICS a, b;
SELECT 'After drop statistic';
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT count(*) FROM tab WHERE b < 10 and a < 10;
SHOW CREATE TABLE tab;
ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest;
SELECT 'After add statistic';
SHOW CREATE TABLE tab;
ALTER TABLE tab MATERIALIZE STATISTICS a, b;
INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000;
SELECT 'After materialize statistic';
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT count(*) FROM tab WHERE b < 10 and a < 10;
OPTIMIZE TABLE tab FINAL;
SELECT 'After merge';
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT count(*) FROM tab WHERE b < 10 and a < 10;
ALTER TABLE tab RENAME COLUMN b TO c;
SHOW CREATE TABLE tab;
SELECT 'After rename';
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT count(*) FROM tab WHERE c < 10 and a < 10;
DROP TABLE IF EXISTS tab;
DROP TABLE tab;

View File

@ -0,0 +1,12 @@
After insert
Prewhere info
Prewhere filter
Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed)
After merge
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed)
After truncate, insert, and materialize
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed)

View File

@ -0,0 +1,36 @@
-- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0').
-- (The concrete statistics type, column data type and predicate type don't matter)
-- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?)
DROP TABLE IF EXISTS tab;
SET allow_experimental_statistics = 1;
SET allow_statistics_optimize = 1;
SET enable_analyzer = 1;
SET materialize_statistics_on_insert = 0;
CREATE TABLE tab
(
a Int64 STATISTICS(tdigest),
b Int16 STATISTICS(tdigest),
) ENGINE = MergeTree() ORDER BY tuple()
SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics.
INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000;
SELECT 'After insert';
SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used)
OPTIMIZE TABLE tab FINAL;
SELECT 'After merge';
SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used)
TRUNCATE TABLE tab;
SET mutations_sync = 2;
INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000;
ALTER TABLE tab MATERIALIZE STATISTICS a, b;
SELECT 'After truncate, insert, and materialize';
SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used)
DROP TABLE tab;

View File

@ -1,55 +0,0 @@
-- Tests creating/dropping/materializing statistics produces the right exceptions.
DROP TABLE IF EXISTS tab;
-- Can't create statistics when allow_experimental_statistics = 0
CREATE TABLE tab
(
a Float64 STATISTICS(tdigest)
) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY }
SET allow_experimental_statistics = 1;
-- The same type of statistics can't exist more than once on a column
CREATE TABLE tab
(
a Float64 STATISTICS(tdigest, tdigest)
) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY }
-- Unknown statistics types are rejected
CREATE TABLE tab
(
a Float64 STATISTICS(no_statistics_type)
) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY }
-- tDigest statistics can only be created on numeric columns
CREATE TABLE tab
(
a String STATISTICS(tdigest),
) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE tab
(
a Float64,
b String
) Engine = MergeTree() ORDER BY tuple();
ALTER TABLE tab ADD STATISTICS a TYPE no_statistics_type; -- { serverError INCORRECT_QUERY }
ALTER TABLE tab ADD STATISTICS a TYPE tdigest;
ALTER TABLE tab ADD STATISTICS IF NOT EXISTS a TYPE tdigest;
ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest;
-- Statistics can be created only on integer columns
ALTER TABLE tab ADD STATISTICS b TYPE tdigest; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab DROP STATISTICS b; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab DROP STATISTICS a;
ALTER TABLE tab DROP STATISTICS IF EXISTS a;
ALTER TABLE tab CLEAR STATISTICS a; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab CLEAR STATISTICS IF EXISTS a;
ALTER TABLE tab MATERIALIZE STATISTICS b; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE tab ADD STATISTICS a TYPE tdigest;
ALTER TABLE tab MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH;
ALTER TABLE tab MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN }
DROP TABLE tab;

View File

@ -1,10 +0,0 @@
10
10
10
statistics not used Condition less(b, 10_UInt8) moved to PREWHERE
statistics not used Condition less(a, 10_UInt8) moved to PREWHERE
statistics used after merge Condition less(a, 10_UInt8) moved to PREWHERE
statistics used after merge Condition less(b, 10_UInt8) moved to PREWHERE
statistics used after materialize Condition less(a, 10_UInt8) moved to PREWHERE
statistics used after materialize Condition less(b, 10_UInt8) moved to PREWHERE
2 0

View File

@ -1,52 +0,0 @@
-- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0').
DROP TABLE IF EXISTS tab;
SET enable_analyzer = 1;
SET allow_experimental_statistics = 1;
SET allow_statistics_optimize = 1;
SET materialize_statistics_on_insert = 0;
CREATE TABLE tab
(
a Int64 STATISTICS(tdigest),
b Int16 STATISTICS(tdigest),
) ENGINE = MergeTree() ORDER BY tuple()
SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics.
INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000;
SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics not used';
OPTIMIZE TABLE tab FINAL;
SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after merge';
TRUNCATE TABLE tab;
SET mutations_sync = 2;
INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000;
ALTER TABLE tab MATERIALIZE STATISTICS a, b;
SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after materialize';
DROP TABLE tab;
SYSTEM FLUSH LOGS;
SELECT log_comment, message FROM system.text_log JOIN
(
SELECT Settings['log_comment'] AS log_comment, query_id FROM system.query_log
WHERE current_database = currentDatabase()
AND query LIKE 'SELECT count(*) FROM tab%'
AND type = 'QueryFinish'
) AS query_log USING (query_id)
WHERE message LIKE '%moved to PREWHERE%'
ORDER BY event_time_microseconds;
SELECT count(), sum(ProfileEvents['MergeTreeDataWriterStatisticsCalculationMicroseconds'])
FROM system.query_log
WHERE current_database = currentDatabase()
AND query LIKE 'INSERT INTO tab SELECT%'
AND type = 'QueryFinish';

View File

@ -0,0 +1,98 @@
u64 and =
10
10
10
10
0
0
0
0
10
10
10
10
u64 and <
70
70
70
70
80
80
80
80
70
70
70
70
f64 and =
10
10
10
10
0
0
0
0
10
10
10
10
0
0
0
0
f64 and <
70
70
70
70
80
80
80
80
70
70
70
70
80
80
80
80
dt and =
0
0
0
0
10
10
10
10
dt and <
10000
10000
10000
10000
70
70
70
70
b and =
5000
5000
5000
5000
5000
5000
5000
5000
5000
5000
5000
5000
0
0
0
0
s and =
10
10

View File

@ -0,0 +1,214 @@
-- Tags: no-fasttest
-- no-fasttest: 'count_min' sketches need a 3rd party library
-- Tests the cross product of all predicates with all right-hand sides on all data types and all statistics types.
SET allow_experimental_statistics = 1;
SET allow_statistics_optimize = 1;
DROP TABLE IF EXISTS tab;
CREATE TABLE tab
(
u64 UInt64,
u64_tdigest UInt64 STATISTICS(tdigest),
u64_count_min UInt64 STATISTICS(count_min),
u64_uniq UInt64 STATISTICS(uniq),
f64 Float64,
f64_tdigest Float64 STATISTICS(tdigest),
f64_count_min Float64 STATISTICS(count_min),
f64_uniq Float64 STATISTICS(uniq),
dt DateTime,
dt_tdigest DateTime STATISTICS(tdigest),
dt_count_min DateTime STATISTICS(count_min),
dt_uniq DateTime STATISTICS(uniq),
b Bool,
b_tdigest Bool STATISTICS(tdigest),
b_count_min Bool STATISTICS(count_min),
b_uniq Bool STATISTICS(uniq),
s String,
-- s_tdigest String STATISTICS(tdigest), -- not supported by tdigest
s_count_min String STATISTICS(count_min)
-- s_uniq String STATISTICS(uniq), -- not supported by uniq
) Engine = MergeTree() ORDER BY tuple()
SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO tab
-- SELECT number % 10000, number % 1000, -(number % 100) FROM system.numbers LIMIT 10000;
SELECT number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 1000,
number % 2,
number % 2,
number % 2,
number % 2,
toString(number % 1000),
toString(number % 1000)
FROM system.numbers LIMIT 10000;
-- u64 ----------------------------------------------------
SELECT 'u64 and =';
SELECT count(*) FROM tab WHERE u64 = 7;
SELECT count(*) FROM tab WHERE u64_tdigest = 7;
SELECT count(*) FROM tab WHERE u64_count_min = 7;
SELECT count(*) FROM tab WHERE u64_uniq = 7;
SELECT count(*) FROM tab WHERE u64 = 7.7;
SELECT count(*) FROM tab WHERE u64_tdigest = 7.7;
SELECT count(*) FROM tab WHERE u64_count_min = 7.7;
SELECT count(*) FROM tab WHERE u64_uniq = 7.7;
SELECT count(*) FROM tab WHERE u64 = '7';
SELECT count(*) FROM tab WHERE u64_tdigest = '7';
SELECT count(*) FROM tab WHERE u64_count_min = '7';
SELECT count(*) FROM tab WHERE u64_uniq = '7';
SELECT count(*) FROM tab WHERE u64 = '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_tdigest = '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_count_min = '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_uniq = '7.7'; -- { serverError TYPE_MISMATCH }
SELECT 'u64 and <';
SELECT count(*) FROM tab WHERE u64 < 7;
SELECT count(*) FROM tab WHERE u64_tdigest < 7;
SELECT count(*) FROM tab WHERE u64_count_min < 7;
SELECT count(*) FROM tab WHERE u64_uniq < 7;
SELECT count(*) FROM tab WHERE u64 < 7.7;
SELECT count(*) FROM tab WHERE u64_tdigest < 7.7;
SELECT count(*) FROM tab WHERE u64_count_min < 7.7;
SELECT count(*) FROM tab WHERE u64_uniq < 7.7;
SELECT count(*) FROM tab WHERE u64 < '7';
SELECT count(*) FROM tab WHERE u64_tdigest < '7';
SELECT count(*) FROM tab WHERE u64_count_min < '7';
SELECT count(*) FROM tab WHERE u64_uniq < '7';
SELECT count(*) FROM tab WHERE u64 < '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_tdigest < '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_count_min < '7.7'; -- { serverError TYPE_MISMATCH }
SELECT count(*) FROM tab WHERE u64_uniq < '7.7'; -- { serverError TYPE_MISMATCH }
-- f64 ----------------------------------------------------
SELECT 'f64 and =';
SELECT count(*) FROM tab WHERE f64 = 7;
SELECT count(*) FROM tab WHERE f64_tdigest = 7;
SELECT count(*) FROM tab WHERE f64_count_min = 7;
SELECT count(*) FROM tab WHERE f64_uniq = 7;
SELECT count(*) FROM tab WHERE f64 = 7.7;
SELECT count(*) FROM tab WHERE f64_tdigest = 7.7;
SELECT count(*) FROM tab WHERE f64_count_min = 7.7;
SELECT count(*) FROM tab WHERE f64_uniq = 7.7;
SELECT count(*) FROM tab WHERE f64 = '7';
SELECT count(*) FROM tab WHERE f64_tdigest = '7';
SELECT count(*) FROM tab WHERE f64_count_min = '7';
SELECT count(*) FROM tab WHERE f64_uniq = '7';
SELECT count(*) FROM tab WHERE f64 = '7.7';
SELECT count(*) FROM tab WHERE f64_tdigest = '7.7';
SELECT count(*) FROM tab WHERE f64_count_min = '7.7';
SELECT count(*) FROM tab WHERE f64_uniq = '7.7';
SELECT 'f64 and <';
SELECT count(*) FROM tab WHERE f64 < 7;
SELECT count(*) FROM tab WHERE f64_tdigest < 7;
SELECT count(*) FROM tab WHERE f64_count_min < 7;
SELECT count(*) FROM tab WHERE f64_uniq < 7;
SELECT count(*) FROM tab WHERE f64 < 7.7;
SELECT count(*) FROM tab WHERE f64_tdigest < 7.7;
SELECT count(*) FROM tab WHERE f64_count_min < 7.7;
SELECT count(*) FROM tab WHERE f64_uniq < 7.7;
SELECT count(*) FROM tab WHERE f64 < '7';
SELECT count(*) FROM tab WHERE f64_tdigest < '7';
SELECT count(*) FROM tab WHERE f64_count_min < '7';
SELECT count(*) FROM tab WHERE f64_uniq < '7';
SELECT count(*) FROM tab WHERE f64 < '7.7';
SELECT count(*) FROM tab WHERE f64_tdigest < '7.7';
SELECT count(*) FROM tab WHERE f64_count_min < '7.7';
SELECT count(*) FROM tab WHERE f64_uniq < '7.7';
-- dt ----------------------------------------------------
SELECT 'dt and =';
SELECT count(*) FROM tab WHERE dt = '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_tdigest = '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_count_min = '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_uniq = '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt = 7;
SELECT count(*) FROM tab WHERE dt_tdigest = 7;
SELECT count(*) FROM tab WHERE dt_count_min = 7;
SELECT count(*) FROM tab WHERE dt_uniq = 7;
SELECT 'dt and <';
SELECT count(*) FROM tab WHERE dt < '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_count_min < '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt_uniq < '2024-08-08 11:12:13';
SELECT count(*) FROM tab WHERE dt < 7;
SELECT count(*) FROM tab WHERE dt_tdigest < 7;
SELECT count(*) FROM tab WHERE dt_count_min < 7;
SELECT count(*) FROM tab WHERE dt_uniq < 7;
-- b ----------------------------------------------------
SELECT 'b and =';
SELECT count(*) FROM tab WHERE b = true;
SELECT count(*) FROM tab WHERE b_tdigest = true;
SELECT count(*) FROM tab WHERE b_count_min = true;
SELECT count(*) FROM tab WHERE b_uniq = true;
SELECT count(*) FROM tab WHERE b = 'true';
SELECT count(*) FROM tab WHERE b_tdigest = 'true';
SELECT count(*) FROM tab WHERE b_count_min = 'true';
SELECT count(*) FROM tab WHERE b_uniq = 'true';
SELECT count(*) FROM tab WHERE b = 1;
SELECT count(*) FROM tab WHERE b_tdigest = 1;
SELECT count(*) FROM tab WHERE b_count_min = 1;
SELECT count(*) FROM tab WHERE b_uniq = 1;
SELECT count(*) FROM tab WHERE b = 1.1;
SELECT count(*) FROM tab WHERE b_tdigest = 1.1;
SELECT count(*) FROM tab WHERE b_count_min = 1.1;
SELECT count(*) FROM tab WHERE b_uniq = 1.1;
-- s ----------------------------------------------------
SELECT 's and =';
SELECT count(*) FROM tab WHERE s = 7; -- { serverError NO_COMMON_TYPE }
-- SELECT count(*) FROM tab WHERE s_tdigest = 7; -- not supported
SELECT count(*) FROM tab WHERE s_count_min = 7; -- { serverError NO_COMMON_TYPE }
-- SELECT count(*) FROM tab WHERE s_uniq = 7; -- not supported
SELECT count(*) FROM tab WHERE s = '7';
-- SELECT count(*) FROM tab WHERE s_tdigest = '7'; -- not supported
SELECT count(*) FROM tab WHERE s_count_min = '7';
-- SELECT count(*) FROM tab WHERE s_uniq = '7'; -- not supported
DROP TABLE tab;

View File

@ -1,35 +0,0 @@
CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest, uniq),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After insert
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed)
After merge
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed)
After modify TDigest
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(c, -1), less(a, 10), less(b, 10)) (removed)
After drop
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed)

View File

@ -1,73 +0,0 @@
DROP TABLE IF EXISTS t1;
SET allow_experimental_statistics = 1;
SET allow_statistics_optimize = 1;
SET mutations_sync = 1;
CREATE TABLE t1
(
a Float64 STATISTICS(tdigest),
b Int64 STATISTICS(tdigest),
c Int64 STATISTICS(tdigest, uniq),
pk String,
) Engine = MergeTree() ORDER BY pk
SETTINGS min_bytes_for_wide_part = 0;
SHOW CREATE TABLE t1;
INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000;
INSERT INTO t1 select 0, 0, 11, generateUUIDv4();
SELECT 'After insert';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
OPTIMIZE TABLE t1 FINAL;
SELECT 'After merge';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT 'After modify TDigest';
ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest;
ALTER TABLE t1 MATERIALIZE STATISTICS c;
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
ALTER TABLE t1 DROP STATISTICS c;
SELECT 'After drop';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
SET allow_suspicious_low_cardinality_types=1;
CREATE TABLE t2
(
a Float64 STATISTICS(tdigest),
b Int64 STATISTICS(tdigest),
c LowCardinality(Int64) STATISTICS(tdigest, uniq),
pk String,
) Engine = MergeTree() ORDER BY pk
SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000;
DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t3;
CREATE TABLE t3
(
a Float64 STATISTICS(tdigest),
b Int64 STATISTICS(tdigest),
c Nullable(Int64) STATISTICS(tdigest, uniq),
pk String,
) Engine = MergeTree() ORDER BY pk
SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000;
DROP TABLE IF EXISTS t3;

View File

@ -0,0 +1,20 @@
After insert
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed)
After drop statistic
Prewhere info
Prewhere filter
Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed)
After add and materialize statistic
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed)
After merge
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed)
After rename
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10_UInt8), less(c, 10_UInt8)) (removed)

View File

@ -0,0 +1,42 @@
-- Test that the optimizer picks up column statistics
-- (The concrete statistics type, column data type and predicate type don't matter)
-- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?)
SET allow_experimental_statistics = 1;
SET allow_statistics_optimize = 1;
SET mutations_sync = 1;
SET enable_analyzer = 1;
DROP TABLE IF EXISTS tab;
CREATE TABLE tab
(
a Float64 STATISTICS(tdigest),
b Int64 STATISTICS(tdigest)
) Engine = MergeTree() ORDER BY tuple()
SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000;
SELECT 'After insert';
SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used)
ALTER TABLE tab DROP STATISTICS a, b;
SELECT 'After drop statistic';
SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used)
ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest;
ALTER TABLE tab MATERIALIZE STATISTICS a, b;
INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000;
SELECT 'After add and materialize statistic';
SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used)
OPTIMIZE TABLE tab FINAL;
SELECT 'After merge';
SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used)
ALTER TABLE tab RENAME COLUMN b TO c;
SELECT 'After rename';
SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then c (statistics used)
DROP TABLE IF EXISTS tab;

View File

@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -nm -q """
${CLICKHOUSE_CLIENT} -m -q """
DROP TABLE IF EXISTS with_lonely;
CREATE TABLE with_lonely
@ -23,7 +23,7 @@ ORDER BY (id);
"""
create_optimize_partition() {
${CLICKHOUSE_CLIENT} -nm -q """
${CLICKHOUSE_CLIENT} -m -q """
INSERT INTO with_lonely SELECT number, '$1', number*10, 0 FROM numbers(10);
INSERT INTO with_lonely SELECT number+500000, '$1', number*10, 1 FROM numbers(10);
"""
@ -39,7 +39,7 @@ create_optimize_partition "2022-10-29"
create_optimize_partition "2022-10-30"
create_optimize_partition "2022-10-31"
${CLICKHOUSE_CLIENT} -nm -q """
${CLICKHOUSE_CLIENT} -m -q """
SYSTEM STOP MERGES with_lonely;
INSERT INTO with_lonely SELECT number, '2022-11-01', number*10, 0 FROM numbers(10);

View File

@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# head by default print 10 rows, but it is not enough to query 11 rows, since
# we need to overflow the default pipe size, hence just 1 million of rows (it
# should be around 6 MiB in text representation, should be definitelly enough).
$CLICKHOUSE_CLIENT --ignore-error -nm --pager head -q "
$CLICKHOUSE_CLIENT --ignore-error -m --pager head -q "
select * from numbers(1e6); -- { clientError CANNOT_WRITE_TO_FILE_DESCRIPTOR }
select * from numbers(1e6); -- { clientError CANNOT_WRITE_TO_FILE_DESCRIPTOR }
"

View File

@ -9,7 +9,7 @@ $CLICKHOUSE_CLIENT -q "select 1; select 2;"
$CLICKHOUSE_LOCAL -q "select 1; select 2;"
# -n is a no-op
$CLICKHOUSE_CLIENT -n -q "select 1; select 2;"
$CLICKHOUSE_LOCAL -n -q "select 1; select 2;"
$CLICKHOUSE_CLIENT -q "select 1; select 2;"
$CLICKHOUSE_LOCAL -q "select 1; select 2;"
exit 0

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