Merge branch 'master' into parquet-fixed-binary

This commit is contained in:
Kruglov Pavel 2023-01-23 15:31:45 +01:00 committed by GitHub
commit 01ddf326ac
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
340 changed files with 10708 additions and 2454 deletions

View File

@ -8,4 +8,4 @@ jobs:
DebugInfo: DebugInfo:
runs-on: ubuntu-latest runs-on: ubuntu-latest
steps: steps:
- uses: hmarr/debug-action@1201a20fc9d278ddddd5f0f46922d06513892491 - uses: hmarr/debug-action@a701ed95a46e6f2fb0df25e1a558c16356fae35a

2
.gitignore vendored
View File

@ -154,6 +154,8 @@ website/package-lock.json
/programs/server/data /programs/server/data
/programs/server/metadata /programs/server/metadata
/programs/server/store /programs/server/store
/programs/server/uuid
/programs/server/coordination
# temporary test files # temporary test files
tests/queries/0_stateless/test_* tests/queries/0_stateless/test_*

2
contrib/poco vendored

@ -1 +1 @@
Subproject commit 799234226187c0ae0b8c90f23465b25ed7956e56 Subproject commit 4b1c8dd9913d2a16db62df0e509fa598da5c8219

View File

@ -58,7 +58,7 @@ echo 'SELECT version()' | curl 'http://localhost:18123/' --data-binary @-
22.6.3.35 22.6.3.35
``` ```
or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows archiving better network performance): or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows achieving better network performance):
```bash ```bash
docker run -d --network=host --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server docker run -d --network=host --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server

View File

@ -18,13 +18,25 @@ repo_dir=ch
BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-15_debug_none_unsplitted_disable_False_binary"} BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-15_debug_none_unsplitted_disable_False_binary"}
BINARY_URL_TO_DOWNLOAD=${BINARY_URL_TO_DOWNLOAD:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/$BINARY_TO_DOWNLOAD/clickhouse"} BINARY_URL_TO_DOWNLOAD=${BINARY_URL_TO_DOWNLOAD:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/$BINARY_TO_DOWNLOAD/clickhouse"}
function git_clone_with_retry
{
for _ in 1 2 3 4; do
if git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git -- "$1" 2>&1 | ts '%Y-%m-%d %H:%M:%S';then
return 0
else
sleep 0.5
fi
done
return 1
}
function clone function clone
{ {
# For local runs, start directly from the "fuzz" stage. # For local runs, start directly from the "fuzz" stage.
rm -rf "$repo_dir" ||: rm -rf "$repo_dir" ||:
mkdir "$repo_dir" ||: mkdir "$repo_dir" ||:
git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git -- "$repo_dir" 2>&1 | ts '%Y-%m-%d %H:%M:%S' git_clone_with_retry "$repo_dir"
( (
cd "$repo_dir" cd "$repo_dir"
if [ "$PR_TO_TEST" != "0" ]; then if [ "$PR_TO_TEST" != "0" ]; then

View File

@ -50,7 +50,7 @@ Action required for every item -- these are errors that must be fixed.
A query is supposed to run longer than 0.1 second. If your query runs faster, increase the amount of processed data to bring the run time above this threshold. You can use a bigger table (e.g. `hits_100m` instead of `hits_10m`), increase a `LIMIT`, make a query single-threaded, and so on. Queries that are too fast suffer from poor stability and precision. A query is supposed to run longer than 0.1 second. If your query runs faster, increase the amount of processed data to bring the run time above this threshold. You can use a bigger table (e.g. `hits_100m` instead of `hits_10m`), increase a `LIMIT`, make a query single-threaded, and so on. Queries that are too fast suffer from poor stability and precision.
#### Partial Queries #### Backward-incompatible Queries
Action required for the cells marked in red. Action required for the cells marked in red.
Shows the queries we are unable to run on an old server -- probably because they contain a new function. You should see this table when you add a new function and a performance test for it. Check that the run time and variance are acceptable (run time between 0.1 and 1 seconds, variance below 10%). If not, they will be highlighted in red. Shows the queries we are unable to run on an old server -- probably because they contain a new function. You should see this table when you add a new function and a performance test for it. Check that the run time and variance are acceptable (run time between 0.1 and 1 seconds, variance below 10%). If not, they will be highlighted in red.

View File

@ -399,7 +399,7 @@ clickhouse-local --query "
create view query_runs as select * from file('analyze/query-runs.tsv', TSV, create view query_runs as select * from file('analyze/query-runs.tsv', TSV,
'test text, query_index int, query_id text, version UInt8, time float'); 'test text, query_index int, query_id text, version UInt8, time float');
-- Separately process 'partial' queries which we could only run on the new server -- Separately process backward-incompatible ('partial') queries which we could only run on the new server
-- because they use new functions. We can't make normal stats for them, but still -- because they use new functions. We can't make normal stats for them, but still
-- have to show some stats so that the PR author can tweak them. -- have to show some stats so that the PR author can tweak them.
create view partial_queries as select test, query_index create view partial_queries as select test, query_index
@ -650,7 +650,7 @@ create view partial_query_times as select * from
'test text, query_index int, time_stddev float, time_median double') 'test text, query_index int, time_stddev float, time_median double')
; ;
-- Report for partial queries that we could only run on the new server (e.g. -- Report for backward-incompatible ('partial') queries that we could only run on the new server (e.g.
-- queries with new functions added in the tested PR). -- queries with new functions added in the tested PR).
create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv') create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv')
settings output_format_decimal_trailing_zeros = 1 settings output_format_decimal_trailing_zeros = 1
@ -829,7 +829,7 @@ create view query_runs as select * from file('analyze/query-runs.tsv', TSV,
-- Guess the number of query runs used for this test. The number is required to -- Guess the number of query runs used for this test. The number is required to
-- calculate and check the average query run time in the report. -- calculate and check the average query run time in the report.
-- We have to be careful, because we will encounter: -- We have to be careful, because we will encounter:
-- 1) partial queries which run only on one server -- 1) backward-incompatible ('partial') queries which run only on one server
-- 3) some errors that make query run for a different number of times on a -- 3) some errors that make query run for a different number of times on a
-- particular server. -- particular server.
-- --

View File

@ -30,7 +30,7 @@ faster_queries = 0
slower_queries = 0 slower_queries = 0
unstable_queries = 0 unstable_queries = 0
very_unstable_queries = 0 very_unstable_queries = 0
unstable_partial_queries = 0 unstable_backward_incompatible_queries = 0
# max seconds to run one query by itself, not counting preparation # max seconds to run one query by itself, not counting preparation
allowed_single_run_time = 2 allowed_single_run_time = 2
@ -378,13 +378,13 @@ if args.report == "main":
] ]
) )
def add_partial(): def add_backward_incompatible():
rows = tsvRows("report/partial-queries-report.tsv") rows = tsvRows("report/partial-queries-report.tsv")
if not rows: if not rows:
return return
global unstable_partial_queries, slow_average_tests, tables global unstable_backward_incompatible_queries, slow_average_tests, tables
text = tableStart("Partial Queries") text = tableStart("Backward-incompatible queries")
columns = ["Median time, s", "Relative time variance", "Test", "#", "Query"] columns = ["Median time, s", "Relative time variance", "Test", "#", "Query"]
text += tableHeader(columns) text += tableHeader(columns)
attrs = ["" for c in columns] attrs = ["" for c in columns]
@ -392,7 +392,7 @@ if args.report == "main":
anchor = f"{currentTableAnchor()}.{row[2]}.{row[3]}" anchor = f"{currentTableAnchor()}.{row[2]}.{row[3]}"
if float(row[1]) > 0.10: if float(row[1]) > 0.10:
attrs[1] = f'style="background: {color_bad}"' attrs[1] = f'style="background: {color_bad}"'
unstable_partial_queries += 1 unstable_backward_incompatible_queries += 1
errors_explained.append( errors_explained.append(
[ [
f"<a href=\"#{anchor}\">The query no. {row[3]} of test '{row[2]}' has excessive variance of run time. Keep it below 10%</a>" f"<a href=\"#{anchor}\">The query no. {row[3]} of test '{row[2]}' has excessive variance of run time. Keep it below 10%</a>"
@ -414,7 +414,7 @@ if args.report == "main":
text += tableEnd() text += tableEnd()
tables.append(text) tables.append(text)
add_partial() add_backward_incompatible()
def add_changes(): def add_changes():
rows = tsvRows("report/changed-perf.tsv") rows = tsvRows("report/changed-perf.tsv")
@ -630,8 +630,8 @@ if args.report == "main":
status = "failure" status = "failure"
message_array.append(str(slower_queries) + " slower") message_array.append(str(slower_queries) + " slower")
if unstable_partial_queries: if unstable_backward_incompatible_queries:
very_unstable_queries += unstable_partial_queries very_unstable_queries += unstable_backward_incompatible_queries
status = "failure" status = "failure"
# Don't show mildly unstable queries, only the very unstable ones we # Don't show mildly unstable queries, only the very unstable ones we

View File

@ -130,6 +130,8 @@ function run_tests()
ADDITIONAL_OPTIONS+=('--report-coverage') ADDITIONAL_OPTIONS+=('--report-coverage')
fi fi
ADDITIONAL_OPTIONS+=('--report-logs-stats')
set +e set +e
clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \
--test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \

View File

@ -289,6 +289,7 @@ if __name__ == "__main__":
"--database=system", "--database=system",
"--hung-check", "--hung-check",
"--stress", "--stress",
"--report-logs-stats",
"00001_select_1", "00001_select_1",
] ]
) )

View File

@ -182,6 +182,31 @@ No matter what pool is used for a job, at start `ThreadStatus` instance is creat
If thread is related to query execution, then the most important thing attached to `ThreadStatus` is query context `ContextPtr`. Every query has its master thread in the server pool. Master thread does the attachment by holding an `ThreadStatus::QueryScope query_scope(query_context)` object. Master thread also creates a thread group represented with `ThreadGroupStatus` object. Every additional thread that is allocated during this query execution is attached to its thread group by `CurrentThread::attachTo(thread_group)` call. Thread groups are used to aggregate profile event counters and track memory consumption by all threads dedicated to a single task (see `MemoryTracker` and `ProfileEvents::Counters` classes for more information). If thread is related to query execution, then the most important thing attached to `ThreadStatus` is query context `ContextPtr`. Every query has its master thread in the server pool. Master thread does the attachment by holding an `ThreadStatus::QueryScope query_scope(query_context)` object. Master thread also creates a thread group represented with `ThreadGroupStatus` object. Every additional thread that is allocated during this query execution is attached to its thread group by `CurrentThread::attachTo(thread_group)` call. Thread groups are used to aggregate profile event counters and track memory consumption by all threads dedicated to a single task (see `MemoryTracker` and `ProfileEvents::Counters` classes for more information).
## Concurrency control {#concurrency-control}
Query that can be parallelized uses `max_threads` setting to limit itself. Default value for this setting is selected in a way that allows single query to utilize all CPU cores in the best way. But what if there are multiple concurrent queries and each of them uses default `max_threads` setting value? Then queries will share CPU resources. OS will ensure fairness by constantly switching threads, which introduce some performance penalty. `ConcurrencyControl` helps to deal with this penalty and avoid allocating a lot of threads. Configuration setting `concurrent_threads_soft_limit_num` is used to limit how many concurrent thread can be allocated before applying some kind of CPU pressure.
:::note
`concurrent_threads_soft_limit_num` and `concurrent_threads_soft_limit_ratio_to_cores` are disabled (equal 0) by default. So this feature must be enabled before use.
:::
Notion of CPU `slot` is introduced. Slot is a unit of concurrency: to run a thread query has to acquire a slot in advance and release it when thread stops. The number of slots is globally limited in a server. Multiple concurrent queries are competing for CPU slots if the total demand exceeds the total number of slots. `ConcurrencyControl` is responsible to resolve this competition by doing CPU slot scheduling in a fair manner.
Each slot can be seen as an independent state machine with the following states:
* `free`: slot is available to be allocated by any query.
* `granted`: slot is `allocated` by specific query, but not yet acquired by any thread.
* `acquired`: slot is `allocated` by specific query and acquired by a thread.
Note that `allocated` slot can be in two different states: `granted` and `acquired`. The former is a transitional state, that actually should be short (from the instant when a slot is allocated to a query till the moment when the up-scaling procedure is run by any thread of that query).
![state diagram](@site/docs/en/development/images/concurrency.png)
API of `ConcurrencyControl` consists of the following functions:
1. Create a resource allocation for a query: `auto slots = ConcurrencyControl::instance().allocate(1, max_threads);`. It will allocate at least 1 and at most `max_threads` slots. Note that the first slot is granted immediately, but the remaining slots may be granted later. Thus limit is soft, because every query will obtain at least one thread.
2. For every thread a slot has to be acquired from an allocation: `while (auto slot = slots->tryAcquire()) spawnThread([slot = std::move(slot)] { ... });`.
3. Update the total amount of slots: `ConcurrencyControl::setMaxConcurrency(concurrent_threads_soft_limit_num)`. Can be done in runtime, w/o server restart.
This API allows queries to start with at least one thread (in presence of CPU pressure) and later scale up to `max_threads`.
## Distributed Query Execution {#distributed-query-execution} ## Distributed Query Execution {#distributed-query-execution}
Servers in a cluster setup are mostly independent. You can create a `Distributed` table on one or all servers in a cluster. The `Distributed` table does not store data itself it only provides a “view” to all local tables on multiple nodes of a cluster. When you SELECT from a `Distributed` table, it rewrites that query, chooses remote nodes according to load balancing settings, and sends the query to them. The `Distributed` table requests remote servers to process a query just up to a stage where intermediate results from different servers can be merged. Then it receives the intermediate results and merges them. The distributed table tries to distribute as much work as possible to remote servers and does not send much intermediate data over the network. Servers in a cluster setup are mostly independent. You can create a `Distributed` table on one or all servers in a cluster. The `Distributed` table does not store data itself it only provides a “view” to all local tables on multiple nodes of a cluster. When you SELECT from a `Distributed` table, it rewrites that query, chooses remote nodes according to load balancing settings, and sends the query to them. The `Distributed` table requests remote servers to process a query just up to a stage where intermediate results from different servers can be merged. Then it receives the intermediate results and merges them. The distributed table tries to distribute as much work as possible to remote servers and does not send much intermediate data over the network.

View File

@ -147,6 +147,14 @@ hash cmake
ClickHouse is available in pre-built binaries and packages. Binaries are portable and can be run on any Linux flavour. ClickHouse is available in pre-built binaries and packages. Binaries are portable and can be run on any Linux flavour.
Binaries are built for stable and LTS releases and also every commit to `master` for each pull request. The CI checks build the binaries on each commit to [ClickHouse](https://github.com/clickhouse/clickhouse/). To download them:
1. Open the [commits list](https://github.com/ClickHouse/ClickHouse/commits/master)
1. Choose a **Merge pull request** commit that includes the new feature, or was added after the new feature
1. Click the status symbol (yellow dot, red x, green check) to open the CI check list
1. Scroll through the list until you find **ClickHouse build check x/x artifact groups are OK**
1. Click **Details**
1. Find the type of package for your operating system that you need and download the files.
![build artifact check](images/find-build-artifact.png)
To find the freshest build from `master`, go to [commits page](https://github.com/ClickHouse/ClickHouse/commits/master), click on the first green check mark or red cross near commit, and click to the “Details” link right after “ClickHouse Build Check”.

Binary file not shown.

After

Width:  |  Height:  |  Size: 34 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 122 KiB

View File

@ -0,0 +1,66 @@
# Inverted indexes [experimental] {#table_engines-ANNIndex}
Inverted indexes are an experimental type of [secondary indexes](mergetree.md#available-types-of-indices) which provide fast text search
capabilities for [String](../../../sql-reference/data-types/string.md) or [FixedString](../../../sql-reference/data-types/fixedstring.md)
columns. The main idea of an inverted indexes is to store a mapping from "terms" to the rows which contains these terms. "Terms" are
tokenized cells of the string column. For example, string cell "I will be a little late" is by default tokenized into six terms "I", "will",
"be", "a", "little" and "late". Another kind of tokenizer are n-grams. For example, the result of 3-gram tokenization will be 21 terms "I w",
" wi", "wil", "ill", "ll ", "l b", " be" etc. The more fine-granular the input strings are tokenized, the bigger but also the more
useful the resulting inverted index will be.
:::warning
Inverted indexes are experimental and should not be used in production environment yet. They may change in future in backwards-incompatible
ways, for example with respect to their DDL/DQL syntax or performance/compression characteristics.
:::
## Usage
To use inverted indexes, first enable them in the configuration:
```sql
SET allow_experimental_inverted_index = true;
```
An inverted index can be defined on a string column using the following syntax
``` sql
CREATE TABLE tab (key UInt64, str String, INDEX inv_idx(s) TYPE inverted(N) GRANULARITY 1) Engine=MergeTree ORDER BY (k);
```
where `N` specifies the tokenizer:
- `inverted(0)` (or shorter: `inverted()`) set the tokenizer to "tokens", i.e. split strings along spaces,
- `inverted(N)` with `N` between 2 and 8 sets the tokenizer to "ngrams(N)"
Being a type of skipping indexes, inverted indexes can be dropped or added to a column after table creation:
``` sql
ALTER TABLE tbl DROP INDEX inv_idx;
ALTER TABLE tbl ADD INDEX inv_idx(s) TYPE inverted(2) GRANULARITY 1;
```
To use the index, no special functions or syntax are required. Typical string search predicates automatically leverage the index. As
examples, consider:
```sql
SELECT * from tab WHERE s == 'Hello World;;
SELECT * from tab WHERE s IN (Hello, World);
SELECT * from tab WHERE s LIKE %Hello%;
SELECT * from tab WHERE multiSearchAny(s, Hello, World);
SELECT * from tab WHERE hasToken(s, Hello);
SELECT * from tab WHERE multiSearchAll(s, [Hello, World])
```
The inverted index also works on columns of type `Array(String)`, `Array(FixedString)`, `Map(String)` and `Map(String)`.
Like for other secondary indices, each column part has its own inverted index. Furthermore, each inverted index is internally divided into
"segments". The existence and size of the segments is generally transparent to users but the segment size determines the memory consumption
during index construction (e.g. when two parts are merged). Configuration parameter "max_digestion_size_per_segment" (default: 256 MB)
controls the amount of data read consumed from the underlying column before a new segment is created. Incrementing the parameter raises the
intermediate memory consumption for index constuction but also improves lookup performance since fewer segments need to be checked on
average to evaluate a query.
Unlike other secondary indices, inverted indexes (for now) map to row numbers (row ids) instead of granule ids. The reason for this design
is performance. In practice, users often search for multiple terms at once. For example, filter predicate `WHERE s LIKE '%little%' OR s LIKE
'%big%'` can be evaluated directly using an inverted index by forming the union of the rowid lists for terms "little" and "big". This also
means that parameter `GRANULARITY` supplied to index creation has no meaning (it may be removed from the syntax in future).

View File

@ -40,8 +40,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2],
... ...
INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1, INDEX index_name1 expr1 TYPE type1(...) [GRANULARITY value1],
INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2, INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2],
... ...
PROJECTION projection_name_1 (SELECT <COLUMN LIST EXPR> [GROUP BY] [ORDER BY]), PROJECTION projection_name_1 (SELECT <COLUMN LIST EXPR> [GROUP BY] [ORDER BY]),
PROJECTION projection_name_2 (SELECT <COLUMN LIST EXPR> [GROUP BY] [ORDER BY]) PROJECTION projection_name_2 (SELECT <COLUMN LIST EXPR> [GROUP BY] [ORDER BY])
@ -359,13 +359,15 @@ ClickHouse uses this logic not only for days of the month sequences, but for any
The index declaration is in the columns section of the `CREATE` query. The index declaration is in the columns section of the `CREATE` query.
``` sql ``` sql
INDEX index_name expr TYPE type(...) GRANULARITY granularity_value INDEX index_name expr TYPE type(...) [GRANULARITY granularity_value]
``` ```
For tables from the `*MergeTree` family, data skipping indices can be specified. For tables from the `*MergeTree` family, data skipping indices can be specified.
These indices aggregate some information about the specified expression on blocks, which consist of `granularity_value` granules (the size of the granule is specified using the `index_granularity` setting in the table engine). Then these aggregates are used in `SELECT` queries for reducing the amount of data to read from the disk by skipping big blocks of data where the `where` query cannot be satisfied. These indices aggregate some information about the specified expression on blocks, which consist of `granularity_value` granules (the size of the granule is specified using the `index_granularity` setting in the table engine). Then these aggregates are used in `SELECT` queries for reducing the amount of data to read from the disk by skipping big blocks of data where the `where` query cannot be satisfied.
The `GRANULARITY` clause can be omitted, the default value of `granularity_value` is 1.
**Example** **Example**
``` sql ``` sql
@ -390,40 +392,47 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
### Available Types of Indices {#available-types-of-indices} ### Available Types of Indices {#available-types-of-indices}
#### `minmax` #### MinMax
Stores extremes of the specified expression (if the expression is `tuple`, then it stores extremes for each element of `tuple`), uses stored info for skipping blocks of data like the primary key. Stores extremes of the specified expression (if the expression is `tuple`, then it stores extremes for each element of `tuple`), uses stored info for skipping blocks of data like the primary key.
#### `set(max_rows)` Syntax: `minmax`
#### Set
Stores unique values of the specified expression (no more than `max_rows` rows, `max_rows=0` means “no limits”). Uses the values to check if the `WHERE` expression is not satisfiable on a block of data. Stores unique values of the specified expression (no more than `max_rows` rows, `max_rows=0` means “no limits”). Uses the values to check if the `WHERE` expression is not satisfiable on a block of data.
#### `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)` Syntax: `set(max_rows)`
Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) that contains all ngrams from a block of data. Works only with datatypes: [String](/docs/en/sql-reference/data-types/string.md), [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) and [Map](/docs/en/sql-reference/data-types/map.md). Can be used for optimization of `EQUALS`, `LIKE` and `IN` expressions. #### Bloom Filter
Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) for the specified columns. An optional `false_positive` parameter with possible values between 0 and 1 specifies the probability of receiving a false positive response from the filter. Default value: 0.025. Supported data types: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`, `UUID` and `Map`. For the `Map` data type, the client can specify if the index should be created for keys or values using [mapKeys](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapkeys) or [mapValues](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapvalues) function.
Syntax: `bloom_filter([false_positive])`
#### N-gram Bloom Filter
Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) that contains all n-grams from a block of data. Only works with datatypes: [String](/docs/en/sql-reference/data-types/string.md), [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) and [Map](/docs/en/sql-reference/data-types/map.md). Can be used for optimization of `EQUALS`, `LIKE` and `IN` expressions.
Syntax: `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
- `n` — ngram size, - `n` — ngram size,
- `size_of_bloom_filter_in_bytes` — Bloom filter size in bytes (you can use large values here, for example, 256 or 512, because it can be compressed well). - `size_of_bloom_filter_in_bytes` — Bloom filter size in bytes (you can use large values here, for example, 256 or 512, because it can be compressed well).
- `number_of_hash_functions` — The number of hash functions used in the Bloom filter. - `number_of_hash_functions` — The number of hash functions used in the Bloom filter.
- `random_seed` — The seed for Bloom filter hash functions. - `random_seed` — The seed for Bloom filter hash functions.
#### `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)` #### Token Bloom Filter
The same as `ngrambf_v1`, but stores tokens instead of ngrams. Tokens are sequences separated by non-alphanumeric characters. The same as `ngrambf_v1`, but stores tokens instead of ngrams. Tokens are sequences separated by non-alphanumeric characters.
#### `bloom_filter([false_positive])` — Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) for the specified columns. Syntax: `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
The optional `false_positive` parameter is the probability of receiving a false positive response from the filter. Possible values: (0, 1). Default value: 0.025. #### Special-purpose
Supported data types: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`, `UUID`, `Map`. - An experimental index to support approximate nearest neighbor (ANN) search. See [here](annindexes.md) for details.
- An experimental inverted index to support full-text search. See [here](invertedindexes.md) for details.
For `Map` data type client can specify if index should be created for keys or values using [mapKeys](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapkeys) or [mapValues](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapvalues) function. ## Example of index creation for Map data type
There are also special-purpose and experimental indexes to support approximate nearest neighbor (ANN) queries. See [here](annindexes.md) for details.
The following functions can use the filter: [equals](/docs/en/sql-reference/functions/comparison-functions.md), [notEquals](/docs/en/sql-reference/functions/comparison-functions.md), [in](/docs/en/sql-reference/functions/in-functions), [notIn](/docs/en/sql-reference/functions/in-functions), [has](/docs/en/sql-reference/functions/array-functions#hasarr-elem), [hasAny](/docs/en/sql-reference/functions/array-functions#hasany), [hasAll](/docs/en/sql-reference/functions/array-functions#hasall).
Example of index creation for `Map` data type
``` ```
INDEX map_key_index mapKeys(map_column) TYPE bloom_filter GRANULARITY 1 INDEX map_key_index mapKeys(map_column) TYPE bloom_filter GRANULARITY 1
@ -484,9 +493,6 @@ For example:
::: :::
## Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex}
In addition to skip indices, there are also [Approximate Nearest Neighbor Search Indexes](/docs/en/engines/table-engines/mergetree-family/annindexes.md).
## Projections {#projections} ## Projections {#projections}
Projections are like [materialized views](/docs/en/sql-reference/statements/create/view.md/#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. Projections are like [materialized views](/docs/en/sql-reference/statements/create/view.md/#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries.
@ -885,6 +891,10 @@ User can assign new big parts to different disks of a [JBOD](https://en.wikipedi
## Using S3 for Data Storage {#table_engine-mergetree-s3} ## Using S3 for Data Storage {#table_engine-mergetree-s3}
:::note
Google Cloud Storage (GCS) is also supported using the type `s3`. See [GCS backed MergeTree](/docs/en/integrations/data-ingestion/s3/gcs-merge-tree.md).
:::
`MergeTree` family table engines can store data to [S3](https://aws.amazon.com/s3/) using a disk with type `s3`. `MergeTree` family table engines can store data to [S3](https://aws.amazon.com/s3/) using a disk with type `s3`.
Configuration markup: Configuration markup:
@ -894,6 +904,7 @@ Configuration markup:
<disks> <disks>
<s3> <s3>
<type>s3</type> <type>s3</type>
<support_batch_delete>true</support_batch_delete>
<endpoint>https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/root-path/</endpoint> <endpoint>https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/root-path/</endpoint>
<access_key_id>your_access_key_id</access_key_id> <access_key_id>your_access_key_id</access_key_id>
<secret_access_key>your_secret_access_key</secret_access_key> <secret_access_key>your_secret_access_key</secret_access_key>
@ -927,6 +938,7 @@ Required parameters:
Optional parameters: Optional parameters:
- `region` — S3 region name. - `region` — S3 region name.
- `support_batch_delete` — This controls the check to see if batch deletes are supported. Set this to `false` when using Google Cloud Storage (GCS) as GCS does not support batch deletes and preventing the checks will prevent error messages in the logs.
- `use_environment_credentials` — Reads AWS credentials from the Environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY and AWS_SESSION_TOKEN if they exist. Default value is `false`. - `use_environment_credentials` — Reads AWS credentials from the Environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY and AWS_SESSION_TOKEN if they exist. Default value is `false`.
- `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Default value is `false`. - `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Default value is `false`.
- `proxy` — Proxy configuration for S3 endpoint. Each `uri` element inside `proxy` block should contain a proxy URL. - `proxy` — Proxy configuration for S3 endpoint. Each `uri` element inside `proxy` block should contain a proxy URL.

View File

@ -271,6 +271,9 @@ Youll need to create data and metadata folders manually and `chown` them for
On Gentoo, you can just use `emerge clickhouse` to install ClickHouse from sources. On Gentoo, you can just use `emerge clickhouse` to install ClickHouse from sources.
### From CI checks pre-built binaries
ClickHouse binaries are built for each [commit](/docs/en/development/build.md#you-dont-have-to-build-clickhouse).
## Launch {#launch} ## Launch {#launch}
To start the server as a daemon, run: To start the server as a daemon, run:

View File

@ -757,6 +757,10 @@ Possible values:
Default value: `0`. Default value: `0`.
**See Also**
- [Concurrency Control](/docs/en/development/architecture.md#concurrency-control)
## concurrent_threads_soft_limit_ratio_to_cores {#concurrent_threads_soft_limit_ratio_to_cores} ## concurrent_threads_soft_limit_ratio_to_cores {#concurrent_threads_soft_limit_ratio_to_cores}
The maximum number of query processing threads as multiple of number of logical cores. The maximum number of query processing threads as multiple of number of logical cores.
More details: [concurrent_threads_soft_limit_num](#concurrent-threads-soft-limit-num). More details: [concurrent_threads_soft_limit_num](#concurrent-threads-soft-limit-num).
@ -768,6 +772,12 @@ Possible values:
Default value: `0`. Default value: `0`.
**Example**
``` xml
<concurrent_threads_soft_limit_ratio_to_cores>3</concurrent_threads_soft_limit_ratio_to_cores>
```
## max_concurrent_queries {#max-concurrent-queries} ## max_concurrent_queries {#max-concurrent-queries}
The maximum number of simultaneously processed queries. The maximum number of simultaneously processed queries.
@ -1181,6 +1191,7 @@ Use the following parameters to configure logging:
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table. - `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
**Example** **Example**
@ -1244,6 +1255,7 @@ Use the following parameters to configure logging:
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table. - `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
If the table does not exist, ClickHouse will create it. If the structure of the query log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. If the table does not exist, ClickHouse will create it. If the structure of the query log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically.
@ -1271,6 +1283,7 @@ Use the following parameters to configure logging:
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table. - `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
If the table does not exist, ClickHouse will create it. If the structure of the query thread log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. If the table does not exist, ClickHouse will create it. If the structure of the query thread log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically.
@ -1298,6 +1311,7 @@ Use the following parameters to configure logging:
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table. - `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
If the table does not exist, ClickHouse will create it. If the structure of the query views log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. If the table does not exist, ClickHouse will create it. If the structure of the query views log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically.
@ -1324,6 +1338,7 @@ Parameters:
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. - `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
**Example** **Example**
```xml ```xml
@ -1351,6 +1366,7 @@ Parameters:
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. - `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
The default server configuration file `config.xml` contains the following settings section: The default server configuration file `config.xml` contains the following settings section:

View File

@ -176,6 +176,59 @@ Similar to [replicated_deduplication_window](#replicated-deduplication-window),
The time is relative to the time of the most recent record, not to the wall time. If it's the only record it will be stored forever. The time is relative to the time of the most recent record, not to the wall time. If it's the only record it will be stored forever.
## replicated_deduplication_window_for_async_inserts {#replicated-deduplication-window-for-async-inserts}
The number of most recently async inserted blocks for which ClickHouse Keeper stores hash sums to check for duplicates.
Possible values:
- Any positive integer.
- 0 (disable deduplication for async_inserts)
Default value: 10000.
The [Async Insert](./settings.md#async-insert) command will be cached in one or more blocks (parts). For [insert deduplication](../../engines/table-engines/mergetree-family/replication.md), when writing into replicated tables, ClickHouse writes the hash sums of each insert into ClickHouse Keeper. Hash sums are stored only for the most recent `replicated_deduplication_window_for_async_inserts` blocks. The oldest hash sums are removed from ClickHouse Keeper.
A large number of `replicated_deduplication_window_for_async_inserts` slows down `Async Inserts` because it needs to compare more entries.
The hash sum is calculated from the composition of the field names and types and the data of the insert (stream of bytes).
## replicated_deduplication_window_seconds_for_async_inserts {#replicated-deduplication-window-seconds-for-async_inserts}
The number of seconds after which the hash sums of the async inserts are removed from ClickHouse Keeper.
Possible values:
- Any positive integer.
Default value: 604800 (1 week).
Similar to [replicated_deduplication_window_for_async_inserts](#replicated-deduplication-window-for-async-inserts), `replicated_deduplication_window_seconds_for_async_inserts` specifies how long to store hash sums of blocks for async insert deduplication. Hash sums older than `replicated_deduplication_window_seconds_for_async_inserts` are removed from ClickHouse Keeper, even if they are less than ` replicated_deduplication_window_for_async_inserts`.
The time is relative to the time of the most recent record, not to the wall time. If it's the only record it will be stored forever.
## use_async_block_ids_cache {#use-async-block-ids-cache}
If true, we cache the hash sums of the async inserts.
Possible values:
- true, false
Default value: false.
A block bearing multiple async inserts will generate multiple hash sums. When some of the inserts are duplicated, keeper will only return one duplicated hash sum in one RPC, which will cause unnecessary RPC retries. This cache will watch the hash sums path in Keeper. If updates are watched in the Keeper, the cache will update as soon as possible, so that we are able to filter the duplicated inserts in the memory.
## async_block_ids_cache_min_update_interval_ms
The minimum interval (in milliseconds) to update the `use_async_block_ids_cache`
Possible values:
- Any positive integer.
Default value: 100.
Normally, the `use_async_block_ids_cache` updates as soon as there are updates in the watching keeper path. However, the cache updates might be too frequent and become a heavy burden. This minimum interval prevents the cache from updating too fast. Note that if we set this value too long, the block with duplicated inserts will have a longer retry time.
## max_replicated_logs_to_keep ## max_replicated_logs_to_keep
How many records may be in the ClickHouse Keeper log if there is inactive replica. An inactive replica becomes lost when when this number exceed. How many records may be in the ClickHouse Keeper log if there is inactive replica. An inactive replica becomes lost when when this number exceed.
@ -745,4 +798,4 @@ You can see which parts of `s` were stored using the sparse serialization:
│ id │ Default │ │ id │ Default │
│ s │ Sparse │ │ s │ Sparse │
└────────┴────────────────────┘ └────────┴────────────────────┘
``` ```

View File

@ -266,7 +266,7 @@ Default value: 0.
Limits the size in bytes of the hash table used when joining tables. Limits the size in bytes of the hash table used when joining tables.
This settings applies to [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operations and [Join table engine](../../engines/table-engines/special/join.md). This setting applies to [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operations and [Join table engine](../../engines/table-engines/special/join.md).
If the query contains joins, ClickHouse checks this setting for every intermediate result. If the query contains joins, ClickHouse checks this setting for every intermediate result.

View File

@ -402,40 +402,62 @@ Default value: `ALL`.
## join_algorithm {#settings-join_algorithm} ## join_algorithm {#settings-join_algorithm}
Specifies [JOIN](../../sql-reference/statements/select/join.md) algorithm. Specifies which [JOIN](../../sql-reference/statements/select/join.md) algorithm is used.
Several algorithms can be specified, and an available one would be chosen for a particular query based on kind/strictness and table engine. Several algorithms can be specified, and an available one would be chosen for a particular query based on kind/strictness and table engine.
Possible values: Possible values:
- `default` — `hash` or `direct`, if possible (same as `direct,hash`) - default
- `hash` — [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. This is the equivalent of `hash` or `direct`, if possible (same as `direct,hash`)
- `parallel_hash` - a variation of `hash` join that splits the data into buckets and builds several hashtables instead of one concurrently to speed up this process. - grace_hash
When using the `hash` algorithm, the right part of `JOIN` is uploaded into RAM. [Grace hash join](https://en.wikipedia.org/wiki/Hash_join#Grace_hash_join) is used. Grace hash provides an algorithm option that provides performant complex joins while limiting memory use.
- `partial_merge` — a variation of the [sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join), where only the right table is fully sorted. The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which dont belong to the current bucket are flushed and reassigned.
The `RIGHT JOIN` and `FULL JOIN` are supported only with `ALL` strictness (`SEMI`, `ANTI`, `ANY`, and `ASOF` are not supported). - hash
When using `partial_merge` algorithm, ClickHouse sorts the data and dumps it to the disk. The `partial_merge` algorithm in ClickHouse differs slightly from the classic realization. First, ClickHouse sorts the right table by joining keys in blocks and creates a min-max index for sorted blocks. Then it sorts parts of the left table by `join key` and joins them over the right table. The min-max index is also used to skip unneeded right table blocks. [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section.
- `direct` - can be applied when the right storage supports key-value requests. - parallel_hash
The `direct` algorithm performs a lookup in the right table using rows from the left table as keys. It's supported only by special storage such as [Dictionary](../../engines/table-engines/special/dictionary.md/#dictionary) or [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) and only the `LEFT` and `INNER` JOINs. A variation of `hash` join that splits the data into buckets and builds several hashtables instead of one concurrently to speed up this process.
- `auto` — try `hash` join and switch on the fly to another algorithm if the memory limit is violated. When using the `hash` algorithm, the right part of `JOIN` is uploaded into RAM.
- `full_sorting_merge` — [Sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join) with full sorting joined tables before joining. - partial_merge
- `prefer_partial_merge` — ClickHouse always tries to use `partial_merge` join if possible, otherwise, it uses `hash`. *Deprecated*, same as `partial_merge,hash`. A variation of the [sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join), where only the right table is fully sorted.
The `RIGHT JOIN` and `FULL JOIN` are supported only with `ALL` strictness (`SEMI`, `ANTI`, `ANY`, and `ASOF` are not supported).
When using the `partial_merge` algorithm, ClickHouse sorts the data and dumps it to the disk. The `partial_merge` algorithm in ClickHouse differs slightly from the classic realization. First, ClickHouse sorts the right table by joining keys in blocks and creates a min-max index for sorted blocks. Then it sorts parts of the left table by the `join key` and joins them over the right table. The min-max index is also used to skip unneeded right table blocks.
- direct
This algorithm can be applied when the storage for the right table supports key-value requests.
The `direct` algorithm performs a lookup in the right table using rows from the left table as keys. It's supported only by special storage such as [Dictionary](../../engines/table-engines/special/dictionary.md/#dictionary) or [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) and only the `LEFT` and `INNER` JOINs.
- auto
When set to `auto`, `hash` join is tried first, and the algorithm is switched on the fly to another algorithm if the memory limit is violated.
- full_sorting_merge
[Sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join) with full sorting joined tables before joining.
- prefer_partial_merge
ClickHouse always tries to use `partial_merge` join if possible, otherwise, it uses `hash`. *Deprecated*, same as `partial_merge,hash`.
## join_any_take_last_row {#settings-join_any_take_last_row} ## join_any_take_last_row {#settings-join_any_take_last_row}
Changes behaviour of join operations with `ANY` strictness. Changes the behaviour of join operations with `ANY` strictness.
:::warning :::warning
This setting applies only for `JOIN` operations with [Join](../../engines/table-engines/special/join.md) engine tables. This setting applies only for `JOIN` operations with [Join](../../engines/table-engines/special/join.md) engine tables.
@ -498,7 +520,7 @@ Default value: 65536.
Limits the number of files allowed for parallel sorting in MergeJoin operations when they are executed on disk. Limits the number of files allowed for parallel sorting in MergeJoin operations when they are executed on disk.
The bigger the value of the setting, the more RAM used and the less disk I/O needed. The bigger the value of the setting, the more RAM is used and the less disk I/O is needed.
Possible values: Possible values:
@ -514,12 +536,12 @@ Enables legacy ClickHouse server behaviour in `ANY INNER|LEFT JOIN` operations.
Use this setting only for backward compatibility if your use cases depend on legacy `JOIN` behaviour. Use this setting only for backward compatibility if your use cases depend on legacy `JOIN` behaviour.
::: :::
When the legacy behaviour enabled: When the legacy behaviour is enabled:
- Results of `t1 ANY LEFT JOIN t2` and `t2 ANY RIGHT JOIN t1` operations are not equal because ClickHouse uses the logic with many-to-one left-to-right table keys mapping. - Results of `t1 ANY LEFT JOIN t2` and `t2 ANY RIGHT JOIN t1` operations are not equal because ClickHouse uses the logic with many-to-one left-to-right table keys mapping.
- Results of `ANY INNER JOIN` operations contain all rows from the left table like the `SEMI LEFT JOIN` operations do. - Results of `ANY INNER JOIN` operations contain all rows from the left table like the `SEMI LEFT JOIN` operations do.
When the legacy behaviour disabled: When the legacy behaviour is disabled:
- Results of `t1 ANY LEFT JOIN t2` and `t2 ANY RIGHT JOIN t1` operations are equal because ClickHouse uses the logic which provides one-to-many keys mapping in `ANY RIGHT JOIN` operations. - Results of `t1 ANY LEFT JOIN t2` and `t2 ANY RIGHT JOIN t1` operations are equal because ClickHouse uses the logic which provides one-to-many keys mapping in `ANY RIGHT JOIN` operations.
- Results of `ANY INNER JOIN` operations contain one row per key from both the left and right tables. - Results of `ANY INNER JOIN` operations contain one row per key from both the left and right tables.
@ -572,7 +594,7 @@ Default value: `163840`.
## merge_tree_min_rows_for_concurrent_read_for_remote_filesystem {#merge-tree-min-rows-for-concurrent-read-for-remote-filesystem} ## merge_tree_min_rows_for_concurrent_read_for_remote_filesystem {#merge-tree-min-rows-for-concurrent-read-for-remote-filesystem}
The minimum number of lines to read from one file before [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can parallelize reading, when reading from remote filesystem. The minimum number of lines to read from one file before the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can parallelize reading, when reading from remote filesystem.
Possible values: Possible values:
@ -706,7 +728,7 @@ log_queries=1
## log_queries_min_query_duration_ms {#settings-log-queries-min-query-duration-ms} ## log_queries_min_query_duration_ms {#settings-log-queries-min-query-duration-ms}
If enabled (non-zero), queries faster then the value of this setting will not be logged (you can think about this as a `long_query_time` for [MySQL Slow Query Log](https://dev.mysql.com/doc/refman/5.7/en/slow-query-log.html)), and this basically means that you will not find them in the following tables: If enabled (non-zero), queries faster than the value of this setting will not be logged (you can think about this as a `long_query_time` for [MySQL Slow Query Log](https://dev.mysql.com/doc/refman/5.7/en/slow-query-log.html)), and this basically means that you will not find them in the following tables:
- `system.query_log` - `system.query_log`
- `system.query_thread_log` - `system.query_thread_log`
@ -741,7 +763,7 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING'
Setting up query threads logging. Setting up query threads logging.
Query threads log into [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting have effect only when [log_queries](#settings-log-queries) is true. Queries threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_thread_log) server configuration parameter. Query threads log into the [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting has effect only when [log_queries](#settings-log-queries) is true. Queries threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_thread_log) server configuration parameter.
Possible values: Possible values:
@ -760,7 +782,7 @@ log_query_threads=1
Setting up query views logging. Setting up query views logging.
When a query run by ClickHouse with this setup on has associated views (materialized or live views), they are logged in the [query_views_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_views_log) server configuration parameter. When a query run by ClickHouse with this setting enabled has associated views (materialized or live views), they are logged in the [query_views_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_views_log) server configuration parameter.
Example: Example:
@ -787,7 +809,7 @@ It can be used to improve the readability of server logs. Additionally, it helps
Possible values: Possible values:
- Any string no longer than [max_query_size](#settings-max_query_size). If length is exceeded, the server throws an exception. - Any string no longer than [max_query_size](#settings-max_query_size). If the max_query_size is exceeded, the server throws an exception.
Default value: empty string. Default value: empty string.
@ -821,11 +843,11 @@ The setting also does not have a purpose when using INSERT SELECT, since data is
Default value: 1,048,576. Default value: 1,048,576.
The default is slightly more than `max_block_size`. The reason for this is because certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allow sorting more data in RAM. The default is slightly more than `max_block_size`. The reason for this is that certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allow sorting more data in RAM.
## min_insert_block_size_rows {#min-insert-block-size-rows} ## min_insert_block_size_rows {#min-insert-block-size-rows}
Sets the minimum number of rows in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. Sets the minimum number of rows in the block that can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones.
Possible values: Possible values:
@ -891,7 +913,7 @@ Higher values will lead to higher memory usage.
## max_compress_block_size {#max-compress-block-size} ## max_compress_block_size {#max-compress-block-size}
The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). Specifying smaller block size generally leads to slightly reduced compression ratio, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced. The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). Specifying a smaller block size generally leads to slightly reduced compression ratio, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced.
:::warning :::warning
This is an expert-level setting, and you shouldn't change it if you're just getting started with ClickHouse. This is an expert-level setting, and you shouldn't change it if you're just getting started with ClickHouse.
@ -935,7 +957,7 @@ Default value: 1000.
## interactive_delay {#interactive-delay} ## interactive_delay {#interactive-delay}
The interval in microseconds for checking whether request execution has been cancelled and sending the progress. The interval in microseconds for checking whether request execution has been canceled and sending the progress.
Default value: 100,000 (checks for cancelling and sends the progress ten times per second). Default value: 100,000 (checks for cancelling and sends the progress ten times per second).
@ -1372,6 +1394,22 @@ By default, blocks inserted into replicated tables by the `INSERT` statement are
For the replicated tables by default the only 100 of the most recent blocks for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md/#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)). For the replicated tables by default the only 100 of the most recent blocks for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md/#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)).
For not replicated tables see [non_replicated_deduplication_window](merge-tree-settings.md/#non-replicated-deduplication-window). For not replicated tables see [non_replicated_deduplication_window](merge-tree-settings.md/#non-replicated-deduplication-window).
## async_insert_deduplicate {#settings-async-insert-deduplicate}
Enables or disables insert deduplication of `ASYNC INSERT` (for Replicated\* tables).
Possible values:
- 0 — Disabled.
- 1 — Enabled.
Default value: 1.
By default, async inserts are inserted into replicated tables by the `INSERT` statement enabling [async_isnert](#async-insert) are deduplicated (see [Data Replication](../../engines/table-engines/mergetree-family/replication.md)).
For the replicated tables, by default, only 10000 of the most recent inserts for each partition are deduplicated (see [replicated_deduplication_window_for_async_inserts](merge-tree-settings.md/#replicated-deduplication-window-async-inserts), [replicated_deduplication_window_seconds_for_async_inserts](merge-tree-settings.md/#replicated-deduplication-window-seconds-async-inserts)).
We recommend enabling the [async_block_ids_cache](merge-tree-settings.md/#use-async-block-ids-cache) to increase the efficiency of deduplication.
This function does not work for non-replicated tables.
## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} ## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views}
Enables or disables the deduplication check for materialized views that receive data from Replicated\* tables. Enables or disables the deduplication check for materialized views that receive data from Replicated\* tables.

View File

@ -4,9 +4,9 @@ sidebar_position: 43
sidebar_label: Boolean sidebar_label: Boolean
--- ---
# Boolean Values bool (boolean) # Bool
Type `bool` is stored as UInt8. Possible values `true` (1), `false` (0). Type `bool` is internally stored as UInt8. Possible values are `true` (1), `false` (0).
```sql ```sql

View File

@ -4,7 +4,7 @@ sidebar_position: 45
sidebar_label: FixedString(N) sidebar_label: FixedString(N)
--- ---
# Fixedstring # FixedString
A fixed-length string of `N` bytes (neither characters nor code points). A fixed-length string of `N` bytes (neither characters nor code points).

View File

@ -4,7 +4,7 @@ sidebar_position: 51
sidebar_label: LowCardinality sidebar_label: LowCardinality
--- ---
# LowCardinality Data Type # LowCardinality
Changes the internal representation of other data types to be dictionary-encoded. Changes the internal representation of other data types to be dictionary-encoded.

View File

@ -133,19 +133,39 @@ The dictionary is completely stored in memory in the form of a hash table. The d
The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type. The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type.
If `preallocate` is `true` (default is `false`) the hash table will be preallocated (this will make the dictionary load faster). But note that you should use it only if:
- The source support an approximate number of elements (for now it is supported only by the `ClickHouse` source).
- There are no duplicates in the data (otherwise it may increase memory usage for the hashtable).
All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety. All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety.
Configuration example: Configuration example:
``` xml
<layout>
<hashed />
</layout>
```
or
``` sql
LAYOUT(HASHED())
```
If `shards` greater then 1 (default is `1`) the dictionary will load data in parallel, useful if you have huge amount of elements in one dictionary.
Configuration example:
``` xml ``` xml
<layout> <layout>
<hashed> <hashed>
<preallocate>0</preallocate> <shards>10</shards>
<!-- Size of the backlog for blocks in parallel queue.
Since the bottleneck in parallel loading is rehash, and so to avoid
stalling because of thread is doing rehash, you need to have some
backlog.
10000 is good balance between memory and speed.
Even for 10e10 elements and can handle all the load without starvation. -->
<shard_load_queue_backlog>10000</shard_load_queue_backlog>
</hashed> </hashed>
</layout> </layout>
``` ```
@ -153,7 +173,7 @@ Configuration example:
or or
``` sql ``` sql
LAYOUT(HASHED(PREALLOCATE 0)) LAYOUT(HASHED(SHARDS 10 [SHARD_LOAD_QUEUE_BACKLOG 10000]))
``` ```
### sparse_hashed ### sparse_hashed
@ -162,8 +182,6 @@ Similar to `hashed`, but uses less memory in favor more CPU usage.
The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type. The dictionary key has the [UInt64](../../../sql-reference/data-types/int-uint.md) type.
It will be also preallocated so as `hashed` (with `preallocate` set to `true`), and note that it is even more significant for `sparse_hashed`.
Configuration example: Configuration example:
``` xml ``` xml
@ -175,9 +193,11 @@ Configuration example:
or or
``` sql ``` sql
LAYOUT(SPARSE_HASHED([PREALLOCATE 0])) LAYOUT(SPARSE_HASHED())
``` ```
It is also possible to use `shards` for this type of dictionary, and again it is more important for `sparse_hashed` then for `hashed`, since `sparse_hashed` is slower.
### complex_key_hashed ### complex_key_hashed
This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `hashed`. This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `hashed`.
@ -186,14 +206,17 @@ Configuration example:
``` xml ``` xml
<layout> <layout>
<complex_key_hashed /> <complex_key_hashed>
<shards>1</shards>
<!-- <shard_load_queue_backlog>10000</shard_load_queue_backlog> -->
</complex_key_hashed>
</layout> </layout>
``` ```
or or
``` sql ``` sql
LAYOUT(COMPLEX_KEY_HASHED()) LAYOUT(COMPLEX_KEY_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000]))
``` ```
### complex_key_sparse_hashed ### complex_key_sparse_hashed
@ -204,14 +227,16 @@ Configuration example:
``` xml ``` xml
<layout> <layout>
<complex_key_sparse_hashed /> <complex_key_sparse_hashed>
<shards>1</shards>
</complex_key_sparse_hashed>
</layout> </layout>
``` ```
or or
``` sql ``` sql
LAYOUT(COMPLEX_KEY_SPARSE_HASHED()) LAYOUT(COMPLEX_KEY_SPARSE_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000]))
``` ```
### hashed_array ### hashed_array

View File

@ -158,8 +158,6 @@ For examples of columns TTL modifying, see [Column TTL](/docs/en/engines/table-e
If the `IF EXISTS` clause is specified, the query wont return an error if the column does not exist. If the `IF EXISTS` clause is specified, the query wont return an error if the column does not exist.
The query also can change the order of the columns using `FIRST | AFTER` clause, see [ADD COLUMN](#alter_add-column) description.
When changing the type, values are converted as if the [toType](/docs/en/sql-reference/functions/type-conversion-functions.md) functions were applied to them. If only the default expression is changed, the query does not do anything complex, and is completed almost instantly. When changing the type, values are converted as if the [toType](/docs/en/sql-reference/functions/type-conversion-functions.md) functions were applied to them. If only the default expression is changed, the query does not do anything complex, and is completed almost instantly.
Example: Example:
@ -170,6 +168,40 @@ ALTER TABLE visits MODIFY COLUMN browser Array(String)
Changing the column type is the only complex action it changes the contents of files with data. For large tables, this may take a long time. Changing the column type is the only complex action it changes the contents of files with data. For large tables, this may take a long time.
The query also can change the order of the columns using `FIRST | AFTER` clause, see [ADD COLUMN](#alter_add-column) description, but column type is mandatory in this case.
Example:
```sql
CREATE TABLE users (
c1 Int16,
c2 String
) ENGINE = MergeTree
ORDER BY c1;
DESCRIBE users;
┌─name─┬─type───┬
│ c1 │ Int16 │
│ c2 │ String │
└──────┴────────┴
ALTER TABLE users MODIFY COLUMN c2 String FIRST;
DESCRIBE users;
┌─name─┬─type───┬
│ c2 │ String │
│ c1 │ Int16 │
└──────┴────────┴
ALTER TABLE users ALTER COLUMN c2 TYPE String AFTER c1;
DESCRIBE users;
┌─name─┬─type───┬
│ c1 │ Int16 │
│ c2 │ String │
└──────┴────────┴
```
The `ALTER` query is atomic. For MergeTree tables it is also lock-free. The `ALTER` query is atomic. For MergeTree tables it is also lock-free.
The `ALTER` query for changing columns is replicated. The instructions are saved in ZooKeeper, then each replica applies them. All `ALTER` queries are run in the same order. The query waits for the appropriate actions to be completed on the other replicas. However, a query to change columns in a replicated table can be interrupted, and all actions will be performed asynchronously. The `ALTER` query for changing columns is replicated. The instructions are saved in ZooKeeper, then each replica applies them. All `ALTER` queries are run in the same order. The query waits for the appropriate actions to be completed on the other replicas. However, a query to change columns in a replicated table can be interrupted, and all actions will be performed asynchronously.

View File

@ -131,27 +131,20 @@ LAYOUT(FLAT(INITIAL_ARRAY_SIZE 50000 MAX_ARRAY_SIZE 5000000))
Ключ словаря имеет тип [UInt64](../../../sql-reference/data-types/int-uint.md). Ключ словаря имеет тип [UInt64](../../../sql-reference/data-types/int-uint.md).
Если `preallocate` имеет значение `true` (по умолчанию `false`), хеш-таблица будет предварительно определена (это ускорит загрузку словаря). Используйте этот метод только в случае, если:
- Источник поддерживает произвольное количество элементов (пока поддерживается только источником `ClickHouse`).
- В данных нет дубликатов (иначе это может увеличить объем используемой памяти хеш-таблицы).
Поддерживаются все виды источников. При обновлении данные (из файла, из таблицы) читаются целиком. Поддерживаются все виды источников. При обновлении данные (из файла, из таблицы) читаются целиком.
Пример конфигурации: Пример конфигурации:
``` xml ``` xml
<layout> <layout>
<hashed> <hashed/>
<preallocate>0</preallocate>
</hashed>
</layout> </layout>
``` ```
или или
``` sql ``` sql
LAYOUT(HASHED(PREALLOCATE 0)) LAYOUT(HASHED())
``` ```
### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} ### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed}
@ -160,8 +153,6 @@ LAYOUT(HASHED(PREALLOCATE 0))
Ключ словаря имеет тип [UInt64](../../../sql-reference/data-types/int-uint.md). Ключ словаря имеет тип [UInt64](../../../sql-reference/data-types/int-uint.md).
Для этого типа размещения также можно задать `preallocate` в значении `true`. В данном случае это более важно, чем для типа `hashed`.
Пример конфигурации: Пример конфигурации:
``` xml ``` xml
@ -173,7 +164,7 @@ LAYOUT(HASHED(PREALLOCATE 0))
или или
``` sql ``` sql
LAYOUT(SPARSE_HASHED([PREALLOCATE 0])) LAYOUT(SPARSE_HASHED())
``` ```
### complex_key_hashed {#complex-key-hashed} ### complex_key_hashed {#complex-key-hashed}

View File

@ -1,97 +1 @@
## Generating ClickHouse documentation {#how-clickhouse-documentation-is-generated} See https://github.com/ClickHouse/clickhouse-docs/blob/main/contrib-writing-guide.md
ClickHouse documentation is built using [Docusaurus](https://docusaurus.io).
## Check the look of your documentation changes {#how-to-check-if-the-documentation-will-look-fine}
There are a few options that are all useful depending on how large or complex your edits are.
### Use the GitHub web interface to edit
Every page in the docs has an **Edit this page** link that opens the page in the GitHub editor. GitHub has Markdown support with a preview feature. The details of GitHub Markdown and the documentation Markdown are a bit different but generally this is close enough, and the person merging your PR will build the docs and check them.
### Install a Markdown editor or plugin for your IDE {#install-markdown-editor-or-plugin-for-your-ide}
Usually, these plugins provide a preview of how the markdown will render, and they catch basic errors like unclosed tags very early.
## Build the docs locally {#use-build-py}
You can build the docs locally. It takes a few minutes to set up, but once you have done it the first time, the process is very simple.
### Clone the repos
The documentation is in two repos, clone both of them:
- [ClickHouse/ClickHouse](https://github.com/ClickHouse/ClickHouse)
- [ClickHouse/ClickHouse-docs](https://github.com/ClickHouse/clickhouse-docs)
### Install Node.js
The documentation is built with Docusaurus, which requires Node.js. We recommend version 16. Install [Node.js](https://nodejs.org/en/download/).
### Copy files into place
Docusaurus expects all of the markdown files to be located in the directory tree `clickhouse-docs/docs/`. This is not the way our repos are set up, so some copying of files is needed to build the docs:
```bash
# from the parent directory of both the ClickHouse/ClickHouse and ClickHouse-clickhouse-docs repos:
cp -r ClickHouse/docs/en/development clickhouse-docs/docs/en/
cp -r ClickHouse/docs/en/engines clickhouse-docs/docs/en/
cp -r ClickHouse/docs/en/getting-started clickhouse-docs/docs/en/
cp -r ClickHouse/docs/en/interfaces clickhouse-docs/docs/en/
cp -r ClickHouse/docs/en/operations clickhouse-docs/docs/en/
cp -r ClickHouse/docs/en/sql-reference clickhouse-docs/docs/en/
cp -r ClickHouse/docs/ru/* clickhouse-docs/docs/ru/
cp -r ClickHouse/docs/zh clickhouse-docs/docs/
```
#### Note: Symlinks will not work.
### Setup Docusaurus
There are two commands that you may need to use with Docusaurus:
- `yarn install`
- `yarn start`
#### Install Docusaurus and its dependencies:
```bash
cd clickhouse-docs
yarn install
```
#### Start a development Docusaurus environment
This command will start Docusaurus in development mode, which means that as you edit source (for example, `.md` files) files the changes will be rendered into HTML files and served by the Docusaurus development server.
```bash
yarn start
```
### Make your changes to the markdown files
Edit your files. Remember that if you are editing files in the `ClickHouse/ClickHouse` repo then you should edit them
in that repo and then copy the edited file into the `ClickHouse/clickhouse-docs/` directory structure so that they are updated in your develoment environment.
`yarn start` probably opened a browser for you when you ran it; if not, open a browser to `http://localhost:3000/docs/en/intro` and navigate to the documentation that you are changing. If you have already made the changes, you can verify them here; if not, make them, and you will see the page update as you save the changes.
## How to change code highlighting? {#how-to-change-code-hl}
Code highlighting is based on the language chosen for your code blocks. Specify the language when you start the code block:
<pre lang="no-highlight"><code>```sql
SELECT firstname from imdb.actors;
```
</code></pre>
```sql
SELECT firstname from imdb.actors;
```
If you need a language supported then open an issue in [ClickHouse-docs](https://github.com/ClickHouse/clickhouse-docs/issues).
## How to subscribe on documentation changes? {#how-to-subscribe-on-documentation-changes}
At the moment theres no easy way to do just that, but you can consider:
- To hit the “Watch” button on top of GitHub web interface to know as early as possible, even during pull request. Alternative to this is `#github-activity` channel of [public ClickHouse Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-qfort0u8-TWqK4wIP0YSdoDE0btKa1w).
- Some search engines allow to subscribe on specific website changes via email and you can opt-in for that for https://clickhouse.com.

View File

@ -1073,6 +1073,9 @@
<!-- Interval of flushing data. --> <!-- Interval of flushing data. -->
<flush_interval_milliseconds>7500</flush_interval_milliseconds> <flush_interval_milliseconds>7500</flush_interval_milliseconds>
<!-- example of using a different storage policy for a system table -->
<!-- storage_policy>local_ssd</storage_policy -->
</query_log> </query_log>
<!-- Trace log. Stores stack traces collected by query profilers. <!-- Trace log. Stores stack traces collected by query profilers.

View File

@ -575,7 +575,9 @@ UUID AccessControl::authenticate(const Credentials & credentials, const Poco::Ne
/// We use the same message for all authentication failures because we don't want to give away any unnecessary information for security reasons, /// We use the same message for all authentication failures because we don't want to give away any unnecessary information for security reasons,
/// only the log will show the exact reason. /// only the log will show the exact reason.
throw Exception(message.str(), ErrorCodes::AUTHENTICATION_FAILED); throw Exception(PreformattedMessage{message.str(),
"{}: Authentication failed: password is incorrect, or there is no user with such name.{}"},
ErrorCodes::AUTHENTICATION_FAILED);
} }
} }

View File

@ -118,8 +118,7 @@ public:
const auto * y_arg = arguments.at(1).get(); const auto * y_arg = arguments.at(1).get();
if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber()) if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber())
throw Exception("Illegal types of arguments of aggregate function " + getName() + ", must have number representation.", throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal types of arguments of aggregate function {}, must have number representation.", getName());
ErrorCodes::BAD_ARGUMENTS);
} }
bool allocatesMemoryInArena() const override { return false; } bool allocatesMemoryInArena() const override { return false; }

View File

@ -226,7 +226,7 @@ public:
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{ {
if (!this->data(place).size_x || !this->data(place).size_y) if (!this->data(place).size_x || !this->data(place).size_y)
throw Exception("Aggregate function " + getName() + " require both samples to be non empty", ErrorCodes::BAD_ARGUMENTS); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} require both samples to be non empty", getName());
auto [u_statistic, p_value] = this->data(place).getResult(alternative, continuity_correction); auto [u_statistic, p_value] = this->data(place).getResult(alternative, continuity_correction);

View File

@ -48,7 +48,7 @@ void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state
if (getSourceExpression()) if (getSourceExpression())
{ {
buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION " << '\n'; buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION" << '\n';
getSourceExpression()->dumpTreeImpl(buffer, format_state, indent + 4); getSourceExpression()->dumpTreeImpl(buffer, format_state, indent + 4);
} }
} }

View File

@ -2,6 +2,7 @@
#include <Common/SipHash.h> #include <Common/SipHash.h>
#include <Common/FieldVisitorToString.h> #include <Common/FieldVisitorToString.h>
#include <DataTypes/IDataType.h>
#include <Analyzer/ConstantNode.h> #include <Analyzer/ConstantNode.h>
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
@ -31,6 +32,15 @@ FunctionNode::FunctionNode(String function_name_)
children[arguments_child_index] = std::make_shared<ListNode>(); children[arguments_child_index] = std::make_shared<ListNode>();
} }
const DataTypes & FunctionNode::getArgumentTypes() const
{
if (!function)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Function {} is not resolved",
function_name);
return function->getArgumentTypes();
}
ColumnsWithTypeAndName FunctionNode::getArgumentColumns() const ColumnsWithTypeAndName FunctionNode::getArgumentColumns() const
{ {
const auto & arguments = getArguments().getNodes(); const auto & arguments = getArguments().getNodes();

View File

@ -85,6 +85,7 @@ public:
/// Get arguments node /// Get arguments node
QueryTreeNodePtr & getArgumentsNode() { return children[arguments_child_index]; } QueryTreeNodePtr & getArgumentsNode() { return children[arguments_child_index]; }
const DataTypes & getArgumentTypes() const;
ColumnsWithTypeAndName getArgumentColumns() const; ColumnsWithTypeAndName getArgumentColumns() const;
/// Returns true if function node has window, false otherwise /// Returns true if function node has window, false otherwise
@ -144,6 +145,11 @@ public:
*/ */
void resolveAsFunction(FunctionBasePtr function_value); void resolveAsFunction(FunctionBasePtr function_value);
void resolveAsFunction(const FunctionOverloadResolverPtr & resolver)
{
resolveAsFunction(resolver->build(getArgumentColumns()));
}
/** Resolve function node as aggregate function. /** Resolve function node as aggregate function.
* It is important that function name is updated with resolved function name. * It is important that function name is updated with resolved function name.
* Main motivation for this is query tree optimizations. * Main motivation for this is query tree optimizations.

View File

@ -1,8 +1,13 @@
#pragma once #pragma once
#include <optional>
#include <utility>
#include <Common/SettingsChanges.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Core/Settings.h>
#include <Analyzer/IQueryTreeNode.h> #include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/QueryNode.h>
namespace DB namespace DB

View File

@ -16,6 +16,8 @@ using ListNodePtr = std::shared_ptr<ListNode>;
class ListNode final : public IQueryTreeNode class ListNode final : public IQueryTreeNode
{ {
public: public:
using iterator = QueryTreeNodes::iterator;
/// Initialize list node with empty nodes /// Initialize list node with empty nodes
ListNode(); ListNode();
@ -41,6 +43,9 @@ public:
void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override;
iterator begin() { return children.begin(); }
iterator end() { return children.end(); }
protected: protected:
bool isEqualImpl(const IQueryTreeNode & rhs) const override; bool isEqualImpl(const IQueryTreeNode & rhs) const override;

View File

@ -3,6 +3,7 @@
#include <AggregateFunctions/AggregateFunctionFactory.h> #include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/IAggregateFunction.h> #include <AggregateFunctions/IAggregateFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h> #include <Functions/IFunction.h>
#include <Analyzer/InDepthQueryTreeVisitor.h> #include <Analyzer/InDepthQueryTreeVisitor.h>
@ -47,19 +48,23 @@ Field zeroField(const Field & value)
class AggregateFunctionsArithmericOperationsVisitor : public InDepthQueryTreeVisitor<AggregateFunctionsArithmericOperationsVisitor> class AggregateFunctionsArithmericOperationsVisitor : public InDepthQueryTreeVisitor<AggregateFunctionsArithmericOperationsVisitor>
{ {
public: public:
explicit AggregateFunctionsArithmericOperationsVisitor(ContextPtr context_)
: context(std::move(context_))
{}
/// Traverse tree bottom to top /// Traverse tree bottom to top
static bool shouldTraverseTopToBottom() static bool shouldTraverseTopToBottom()
{ {
return false; return false;
} }
static void visitImpl(QueryTreeNodePtr & node) void visitImpl(QueryTreeNodePtr & node)
{ {
auto * aggregate_function_node = node->as<FunctionNode>(); auto * aggregate_function_node = node->as<FunctionNode>();
if (!aggregate_function_node || !aggregate_function_node->isAggregateFunction()) if (!aggregate_function_node || !aggregate_function_node->isAggregateFunction())
return; return;
static std::unordered_map<std::string_view, std::unordered_set<std::string_view>> supported_functions static std::unordered_map<std::string_view, std::unordered_set<std::string_view>> supported_aggregate_functions
= {{"sum", {"multiply", "divide"}}, = {{"sum", {"multiply", "divide"}},
{"min", {"multiply", "divide", "plus", "minus"}}, {"min", {"multiply", "divide", "plus", "minus"}},
{"max", {"multiply", "divide", "plus", "minus"}}, {"max", {"multiply", "divide", "plus", "minus"}},
@ -69,88 +74,111 @@ public:
if (aggregate_function_arguments_nodes.size() != 1) if (aggregate_function_arguments_nodes.size() != 1)
return; return;
auto * inner_function_node = aggregate_function_arguments_nodes[0]->as<FunctionNode>(); const auto & arithmetic_function_node = aggregate_function_arguments_nodes[0];
if (!inner_function_node) auto * arithmetic_function_node_typed = arithmetic_function_node->as<FunctionNode>();
if (!arithmetic_function_node_typed)
return; return;
const auto & inner_function_arguments_nodes = inner_function_node->getArguments().getNodes(); const auto & arithmetic_function_arguments_nodes = arithmetic_function_node_typed->getArguments().getNodes();
if (inner_function_arguments_nodes.size() != 2) if (arithmetic_function_arguments_nodes.size() != 2)
return; return;
/// Aggregate functions[sum|min|max|avg] is case-insensitive, so we use lower cases name /// Aggregate functions[sum|min|max|avg] is case-insensitive, so we use lower cases name
auto lower_function_name = Poco::toLower(aggregate_function_node->getFunctionName()); auto lower_aggregate_function_name = Poco::toLower(aggregate_function_node->getFunctionName());
auto supported_function_it = supported_functions.find(lower_function_name); auto supported_aggregate_function_it = supported_aggregate_functions.find(lower_aggregate_function_name);
if (supported_function_it == supported_functions.end()) if (supported_aggregate_function_it == supported_aggregate_functions.end())
return; return;
const auto & inner_function_name = inner_function_node->getFunctionName(); const auto & arithmetic_function_name = arithmetic_function_node_typed->getFunctionName();
if (!supported_aggregate_function_it->second.contains(arithmetic_function_name))
if (!supported_function_it->second.contains(inner_function_name))
return; return;
const auto * left_argument_constant_node = inner_function_arguments_nodes[0]->as<ConstantNode>(); const auto * left_argument_constant_node = arithmetic_function_arguments_nodes[0]->as<ConstantNode>();
const auto * right_argument_constant_node = inner_function_arguments_nodes[1]->as<ConstantNode>(); const auto * right_argument_constant_node = arithmetic_function_arguments_nodes[1]->as<ConstantNode>();
/** If we extract negative constant, aggregate function name must be updated. /** If we extract negative constant, aggregate function name must be updated.
* *
* Example: SELECT min(-1 * id); * Example: SELECT min(-1 * id);
* Result: SELECT -1 * max(id); * Result: SELECT -1 * max(id);
*/ */
std::string function_name_if_constant_is_negative; std::string aggregate_function_name_if_constant_is_negative;
if (inner_function_name == "multiply" || inner_function_name == "divide") if (arithmetic_function_name == "multiply" || arithmetic_function_name == "divide")
{ {
if (lower_function_name == "min") if (lower_aggregate_function_name == "min")
function_name_if_constant_is_negative = "max"; aggregate_function_name_if_constant_is_negative = "max";
else if (lower_function_name == "max") else if (lower_aggregate_function_name == "max")
function_name_if_constant_is_negative = "min"; aggregate_function_name_if_constant_is_negative = "min";
} }
size_t arithmetic_function_argument_index = 0;
if (left_argument_constant_node && !right_argument_constant_node) if (left_argument_constant_node && !right_argument_constant_node)
{ {
/// Do not rewrite `sum(1/n)` with `sum(1) * div(1/n)` because of lose accuracy /// Do not rewrite `sum(1/n)` with `sum(1) * div(1/n)` because of lose accuracy
if (inner_function_name == "divide") if (arithmetic_function_name == "divide")
return; return;
/// Rewrite `aggregate_function(inner_function(constant, argument))` into `inner_function(constant, aggregate_function(argument))` /// Rewrite `aggregate_function(inner_function(constant, argument))` into `inner_function(constant, aggregate_function(argument))`
const auto & left_argument_constant_value_literal = left_argument_constant_node->getValue(); const auto & left_argument_constant_value_literal = left_argument_constant_node->getValue();
if (!function_name_if_constant_is_negative.empty() && if (!aggregate_function_name_if_constant_is_negative.empty() &&
left_argument_constant_value_literal < zeroField(left_argument_constant_value_literal)) left_argument_constant_value_literal < zeroField(left_argument_constant_value_literal))
{ {
lower_function_name = function_name_if_constant_is_negative; lower_aggregate_function_name = aggregate_function_name_if_constant_is_negative;
} }
auto inner_function_clone = inner_function_node->clone(); arithmetic_function_argument_index = 1;
auto & inner_function_clone_arguments = inner_function_clone->as<FunctionNode &>().getArguments();
auto & inner_function_clone_arguments_nodes = inner_function_clone_arguments.getNodes();
auto inner_function_clone_right_argument = inner_function_clone_arguments_nodes[1];
aggregate_function_arguments_nodes = {inner_function_clone_right_argument};
resolveAggregateFunctionNode(*aggregate_function_node, inner_function_clone_right_argument, lower_function_name);
inner_function_clone_arguments_nodes[1] = node;
node = std::move(inner_function_clone);
} }
else if (right_argument_constant_node) else if (right_argument_constant_node)
{ {
/// Rewrite `aggregate_function(inner_function(argument, constant))` into `inner_function(aggregate_function(argument), constant)` /// Rewrite `aggregate_function(inner_function(argument, constant))` into `inner_function(aggregate_function(argument), constant)`
const auto & right_argument_constant_value_literal = right_argument_constant_node->getValue(); const auto & right_argument_constant_value_literal = right_argument_constant_node->getValue();
if (!function_name_if_constant_is_negative.empty() && if (!aggregate_function_name_if_constant_is_negative.empty() &&
right_argument_constant_value_literal < zeroField(right_argument_constant_value_literal)) right_argument_constant_value_literal < zeroField(right_argument_constant_value_literal))
{ {
lower_function_name = function_name_if_constant_is_negative; lower_aggregate_function_name = aggregate_function_name_if_constant_is_negative;
} }
auto inner_function_clone = inner_function_node->clone(); arithmetic_function_argument_index = 0;
auto & inner_function_clone_arguments = inner_function_clone->as<FunctionNode &>().getArguments();
auto & inner_function_clone_arguments_nodes = inner_function_clone_arguments.getNodes();
auto inner_function_clone_left_argument = inner_function_clone_arguments_nodes[0];
aggregate_function_arguments_nodes = {inner_function_clone_left_argument};
resolveAggregateFunctionNode(*aggregate_function_node, inner_function_clone_left_argument, lower_function_name);
inner_function_clone_arguments_nodes[0] = node;
node = std::move(inner_function_clone);
} }
auto optimized_function_node = cloneArithmeticFunctionAndWrapArgumentIntoAggregateFunction(arithmetic_function_node,
arithmetic_function_argument_index,
node,
lower_aggregate_function_name);
if (optimized_function_node->getResultType()->equals(*node->getResultType()))
node = std::move(optimized_function_node);
} }
private: private:
QueryTreeNodePtr cloneArithmeticFunctionAndWrapArgumentIntoAggregateFunction(
const QueryTreeNodePtr & arithmetic_function,
size_t arithmetic_function_argument_index,
const QueryTreeNodePtr & aggregate_function,
const std::string & result_aggregate_function_name)
{
auto arithmetic_function_clone = arithmetic_function->clone();
auto & arithmetic_function_clone_typed = arithmetic_function_clone->as<FunctionNode &>();
auto & arithmetic_function_clone_arguments_nodes = arithmetic_function_clone_typed.getArguments().getNodes();
auto & arithmetic_function_clone_argument = arithmetic_function_clone_arguments_nodes[arithmetic_function_argument_index];
auto aggregate_function_clone = aggregate_function->clone();
auto & aggregate_function_clone_typed = aggregate_function_clone->as<FunctionNode &>();
aggregate_function_clone_typed.getArguments().getNodes() = { arithmetic_function_clone_argument };
resolveAggregateFunctionNode(aggregate_function_clone_typed, arithmetic_function_clone_argument, result_aggregate_function_name);
arithmetic_function_clone_arguments_nodes[arithmetic_function_argument_index] = std::move(aggregate_function_clone);
resolveOrdinaryFunctionNode(arithmetic_function_clone_typed, arithmetic_function_clone_typed.getFunctionName());
return arithmetic_function_clone;
}
inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const
{
auto function = FunctionFactory::instance().get(function_name, context);
function_node.resolveAsFunction(function->build(function_node.getArgumentColumns()));
}
static inline void resolveAggregateFunctionNode(FunctionNode & function_node, const QueryTreeNodePtr & argument, const String & aggregate_function_name) static inline void resolveAggregateFunctionNode(FunctionNode & function_node, const QueryTreeNodePtr & argument, const String & aggregate_function_name)
{ {
auto function_aggregate_function = function_node.getAggregateFunction(); auto function_aggregate_function = function_node.getAggregateFunction();
@ -163,13 +191,15 @@ private:
function_node.resolveAsAggregateFunction(std::move(aggregate_function)); function_node.resolveAsAggregateFunction(std::move(aggregate_function));
} }
ContextPtr context;
}; };
} }
void AggregateFunctionsArithmericOperationsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) void AggregateFunctionsArithmericOperationsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{ {
AggregateFunctionsArithmericOperationsVisitor visitor; AggregateFunctionsArithmericOperationsVisitor visitor(std::move(context));
visitor.visit(query_tree_node); visitor.visit(query_tree_node);
} }

View File

@ -0,0 +1,134 @@
#include <memory>
#include <unordered_map>
#include <vector>
#include <Analyzer/Passes/ConvertOrLikeChainPass.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/UnionNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/HashUtils.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Core/Field.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/likePatternToRegexp.h>
#include <Interpreters/Context.h>
namespace DB
{
namespace
{
class ConvertOrLikeChainVisitor : public InDepthQueryTreeVisitor<ConvertOrLikeChainVisitor>
{
using FunctionNodes = std::vector<std::shared_ptr<FunctionNode>>;
const FunctionOverloadResolverPtr match_function_ref;
const FunctionOverloadResolverPtr or_function_resolver;
public:
explicit ConvertOrLikeChainVisitor(ContextPtr context)
: InDepthQueryTreeVisitor<ConvertOrLikeChainVisitor>()
, match_function_ref(FunctionFactory::instance().get("multiMatchAny", context))
, or_function_resolver(FunctionFactory::instance().get("or", context))
{}
static bool needChildVisit(VisitQueryTreeNodeType & parent, VisitQueryTreeNodeType &)
{
ContextPtr context;
if (auto * query = parent->as<QueryNode>())
context = query->getContext();
else if (auto * union_node = parent->as<UnionNode>())
context = union_node->getContext();
if (context)
{
const auto & settings = context->getSettingsRef();
return settings.optimize_or_like_chain
&& settings.allow_hyperscan
&& settings.max_hyperscan_regexp_length == 0
&& settings.max_hyperscan_regexp_total_length == 0;
}
return true;
}
void visitImpl(QueryTreeNodePtr & node)
{
auto * function_node = node->as<FunctionNode>();
if (!function_node || function_node->getFunctionName() != "or")
return;
QueryTreeNodes unique_elems;
QueryTreeNodePtrWithHashMap<Array> node_to_patterns;
FunctionNodes match_functions;
for (auto & arg : function_node->getArguments())
{
unique_elems.push_back(arg);
auto * arg_func = arg->as<FunctionNode>();
if (!arg_func)
continue;
const bool is_like = arg_func->getFunctionName() == "like";
const bool is_ilike = arg_func->getFunctionName() == "ilike";
/// Not {i}like -> bail out.
if (!is_like && !is_ilike)
continue;
const auto & like_arguments = arg_func->getArguments().getNodes();
if (like_arguments.size() != 2)
continue;
auto identifier = like_arguments[0];
auto * pattern = like_arguments[1]->as<ConstantNode>();
if (!pattern || !isString(pattern->getResultType()))
continue;
auto regexp = likePatternToRegexp(pattern->getValue().get<String>());
/// Case insensitive. Works with UTF-8 as well.
if (is_ilike)
regexp = "(?i)" + regexp;
unique_elems.pop_back();
auto it = node_to_patterns.find(identifier);
if (it == node_to_patterns.end())
{
it = node_to_patterns.insert({identifier, Array{}}).first;
/// The second argument will be added when all patterns are known.
auto match_function = std::make_shared<FunctionNode>("multiMatchAny");
match_function->getArguments().getNodes().push_back(identifier);
match_functions.push_back(match_function);
unique_elems.push_back(std::move(match_function));
}
it->second.push_back(regexp);
}
/// Add all the patterns into the function arguments lists.
for (auto & match_function : match_functions)
{
auto & arguments = match_function->getArguments().getNodes();
auto & patterns = node_to_patterns.at(arguments[0]);
arguments.push_back(std::make_shared<ConstantNode>(Field{std::move(patterns)}));
match_function->resolveAsFunction(match_function_ref);
}
/// OR must have at least two arguments.
if (unique_elems.size() == 1)
unique_elems.push_back(std::make_shared<ConstantNode>(false));
function_node->getArguments().getNodes() = std::move(unique_elems);
function_node->resolveAsFunction(or_function_resolver);
}
};
}
void ConvertOrLikeChainPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
ConvertOrLikeChainVisitor visitor(context);
visitor.visit(query_tree_node);
}
}

View File

@ -0,0 +1,20 @@
#pragma once
#include <Analyzer/IQueryTreePass.h>
namespace DB
{
/** Replaces all the "or"'s with {i}like to multiMatchAny
*/
class ConvertOrLikeChainPass final : public IQueryTreePass
{
public:
String getName() override { return "ConvertOrLikeChain"; }
String getDescription() override { return "Replaces all the 'or's with {i}like to multiMatchAny"; }
void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
};
}

View File

@ -0,0 +1,253 @@
#include <Analyzer/Passes/GroupingFunctionsResolvePass.h>
#include <Core/ColumnNumbers.h>
#include <Functions/grouping.h>
#include <Interpreters/Context.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/HashUtils.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/ColumnNode.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
namespace
{
enum class GroupByKind
{
ORDINARY,
ROLLUP,
CUBE,
GROUPING_SETS
};
class GroupingFunctionResolveVisitor : public InDepthQueryTreeVisitor<GroupingFunctionResolveVisitor>
{
public:
GroupingFunctionResolveVisitor(GroupByKind group_by_kind_,
QueryTreeNodePtrWithHashMap<size_t> aggregation_key_to_index_,
ColumnNumbersList grouping_sets_keys_indices_,
ContextPtr context_)
: group_by_kind(group_by_kind_)
, aggregation_key_to_index(std::move(aggregation_key_to_index_))
, grouping_sets_keys_indexes(std::move(grouping_sets_keys_indices_))
, context(std::move(context_))
{
}
void visitImpl(const QueryTreeNodePtr & node)
{
auto * function_node = node->as<FunctionNode>();
if (!function_node || function_node->getFunctionName() != "grouping")
return;
auto & function_arguments = function_node->getArguments().getNodes();
ColumnNumbers arguments_indexes;
arguments_indexes.reserve(function_arguments.size());
for (const auto & argument : function_arguments)
{
auto it = aggregation_key_to_index.find(argument);
if (it == aggregation_key_to_index.end())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Argument {} of GROUPING function is not a part of GROUP BY clause",
argument->formatASTForErrorMessage());
arguments_indexes.push_back(it->second);
}
FunctionOverloadResolverPtr grouping_function_resolver;
bool add_grouping_set_column = false;
bool force_grouping_standard_compatibility = context->getSettingsRef().force_grouping_standard_compatibility;
size_t aggregation_keys_size = aggregation_key_to_index.size();
switch (group_by_kind)
{
case GroupByKind::ORDINARY:
{
auto grouping_ordinary_function = std::make_shared<FunctionGroupingOrdinary>(arguments_indexes,
force_grouping_standard_compatibility);
grouping_function_resolver = std::make_shared<FunctionToOverloadResolverAdaptor>(std::move(grouping_ordinary_function));
break;
}
case GroupByKind::ROLLUP:
{
auto grouping_rollup_function = std::make_shared<FunctionGroupingForRollup>(arguments_indexes,
aggregation_keys_size,
force_grouping_standard_compatibility);
grouping_function_resolver = std::make_shared<FunctionToOverloadResolverAdaptor>(std::move(grouping_rollup_function));
add_grouping_set_column = true;
break;
}
case GroupByKind::CUBE:
{
auto grouping_cube_function = std::make_shared<FunctionGroupingForCube>(arguments_indexes,
aggregation_keys_size,
force_grouping_standard_compatibility);
grouping_function_resolver = std::make_shared<FunctionToOverloadResolverAdaptor>(std::move(grouping_cube_function));
add_grouping_set_column = true;
break;
}
case GroupByKind::GROUPING_SETS:
{
auto grouping_grouping_sets_function = std::make_shared<FunctionGroupingForGroupingSets>(arguments_indexes,
grouping_sets_keys_indexes,
force_grouping_standard_compatibility);
grouping_function_resolver = std::make_shared<FunctionToOverloadResolverAdaptor>(std::move(grouping_grouping_sets_function));
add_grouping_set_column = true;
break;
}
}
if (add_grouping_set_column)
{
QueryTreeNodeWeakPtr column_source;
auto grouping_set_column = NameAndTypePair{"__grouping_set", std::make_shared<DataTypeUInt64>()};
auto grouping_set_argument_column = std::make_shared<ColumnNode>(std::move(grouping_set_column), std::move(column_source));
function_arguments.insert(function_arguments.begin(), std::move(grouping_set_argument_column));
}
function_node->resolveAsFunction(grouping_function_resolver->build(function_node->getArgumentColumns()));
}
static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node)
{
return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION);
}
private:
GroupByKind group_by_kind;
QueryTreeNodePtrWithHashMap<size_t> aggregation_key_to_index;
ColumnNumbersList grouping_sets_keys_indexes;
ContextPtr context;
};
void resolveGroupingFunctions(QueryTreeNodePtr & query_node, ContextPtr context)
{
auto & query_node_typed = query_node->as<QueryNode &>();
size_t aggregation_node_index = 0;
QueryTreeNodePtrWithHashMap<size_t> aggregation_key_to_index;
std::vector<QueryTreeNodes> grouping_sets_used_aggregation_keys_list;
if (query_node_typed.hasGroupBy())
{
/// It is expected by execution layer that if there are only 1 grouping set it will be removed
if (query_node_typed.isGroupByWithGroupingSets() && query_node_typed.getGroupBy().getNodes().size() == 1)
{
auto & grouping_set_list_node = query_node_typed.getGroupBy().getNodes().front()->as<ListNode &>();
query_node_typed.getGroupBy().getNodes() = std::move(grouping_set_list_node.getNodes());
query_node_typed.setIsGroupByWithGroupingSets(false);
}
if (query_node_typed.isGroupByWithGroupingSets())
{
for (const auto & grouping_set_keys_list_node : query_node_typed.getGroupBy().getNodes())
{
auto & grouping_set_keys_list_node_typed = grouping_set_keys_list_node->as<ListNode &>();
grouping_sets_used_aggregation_keys_list.emplace_back();
auto & grouping_sets_used_aggregation_keys = grouping_sets_used_aggregation_keys_list.back();
for (auto & grouping_set_key_node : grouping_set_keys_list_node_typed.getNodes())
{
if (aggregation_key_to_index.contains(grouping_set_key_node))
continue;
grouping_sets_used_aggregation_keys.push_back(grouping_set_key_node);
aggregation_key_to_index.emplace(grouping_set_key_node, aggregation_node_index);
++aggregation_node_index;
}
}
}
else
{
for (auto & group_by_key_node : query_node_typed.getGroupBy().getNodes())
{
if (aggregation_key_to_index.contains(group_by_key_node))
continue;
aggregation_key_to_index.emplace(group_by_key_node, aggregation_node_index);
++aggregation_node_index;
}
}
}
/// Indexes of aggregation keys used in each grouping set (only for GROUP BY GROUPING SETS)
ColumnNumbersList grouping_sets_keys_indexes;
for (const auto & grouping_set_used_aggregation_keys : grouping_sets_used_aggregation_keys_list)
{
grouping_sets_keys_indexes.emplace_back();
auto & grouping_set_keys_indexes = grouping_sets_keys_indexes.back();
for (const auto & used_aggregation_key : grouping_set_used_aggregation_keys)
{
auto aggregation_node_index_it = aggregation_key_to_index.find(used_aggregation_key);
if (aggregation_node_index_it == aggregation_key_to_index.end())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Aggregation key {} in GROUPING SETS is not found in GROUP BY keys",
used_aggregation_key->formatASTForErrorMessage());
grouping_set_keys_indexes.push_back(aggregation_node_index_it->second);
}
}
GroupByKind group_by_kind = GroupByKind::ORDINARY;
if (query_node_typed.isGroupByWithRollup())
group_by_kind = GroupByKind::ROLLUP;
else if (query_node_typed.isGroupByWithCube())
group_by_kind = GroupByKind::CUBE;
else if (query_node_typed.isGroupByWithGroupingSets())
group_by_kind = GroupByKind::GROUPING_SETS;
GroupingFunctionResolveVisitor visitor(group_by_kind,
std::move(aggregation_key_to_index),
std::move(grouping_sets_keys_indexes),
std::move(context));
visitor.visit(query_node);
}
class GroupingFunctionsResolveVisitor : public InDepthQueryTreeVisitor<GroupingFunctionsResolveVisitor>
{
public:
explicit GroupingFunctionsResolveVisitor(ContextPtr context_)
: context(std::move(context_))
{}
void visitImpl(QueryTreeNodePtr & node)
{
if (node->getNodeType() != QueryTreeNodeType::QUERY)
return;
resolveGroupingFunctions(node, context);
}
private:
ContextPtr context;
};
}
void GroupingFunctionsResolvePass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
GroupingFunctionsResolveVisitor visitor(std::move(context));
visitor.visit(query_tree_node);
}
}

View File

@ -0,0 +1,31 @@
#pragma once
#include <Analyzer/IQueryTreePass.h>
namespace DB
{
/** Resolve GROUPING functions in query node.
* GROUPING function is replaced with specialized GROUPING function based on GROUP BY modifiers.
* For ROLLUP, CUBE, GROUPING SETS specialized GROUPING function take special __grouping_set column as argument
* and previous GROUPING function arguments.
*
* Example: SELECT grouping(id) FROM test_table GROUP BY id;
* Result: SELECT groupingOrdinary(id) FROM test_table GROUP BY id;
*
* Example: SELECT grouping(id), grouping(value) FROM test_table GROUP BY GROUPING SETS ((id), (value));
* Result: SELECT groupingForGroupingSets(__grouping_set, id), groupingForGroupingSets(__grouping_set, value)
* FROM test_table GROUP BY GROUPING SETS ((id), (value));
*/
class GroupingFunctionsResolvePass final : public IQueryTreePass
{
public:
String getName() override { return "GroupingFunctionsResolvePass"; }
String getDescription() override { return "Resolve GROUPING functions based on GROUP BY modifiers"; }
void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
};
}

View File

@ -0,0 +1,113 @@
#include <Analyzer/Passes/OptimizeGroupByFunctionKeysPass.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/HashUtils.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/QueryNode.h>
#include <algorithm>
#include <queue>
namespace DB
{
class OptimizeGroupByFunctionKeysVisitor : public InDepthQueryTreeVisitor<OptimizeGroupByFunctionKeysVisitor>
{
public:
static bool needChildVisit(QueryTreeNodePtr & /*parent*/, QueryTreeNodePtr & child)
{
return !child->as<FunctionNode>();
}
static void visitImpl(QueryTreeNodePtr & node)
{
auto * query = node->as<QueryNode>();
if (!query)
return;
if (!query->hasGroupBy())
return;
auto & group_by = query->getGroupBy().getNodes();
if (query->isGroupByWithGroupingSets())
{
for (auto & set : group_by)
{
auto & grouping_set = set->as<ListNode>()->getNodes();
optimizeGroupingSet(grouping_set);
}
}
else
optimizeGroupingSet(group_by);
}
private:
static bool canBeEliminated(QueryTreeNodePtr & node, const QueryTreeNodePtrWithHashSet & group_by_keys)
{
auto * function = node->as<FunctionNode>();
if (!function || function->getArguments().getNodes().empty())
return false;
QueryTreeNodes candidates;
auto & function_arguments = function->getArguments().getNodes();
for (auto it = function_arguments.rbegin(); it != function_arguments.rend(); ++it)
candidates.push_back(*it);
// Using DFS we traverse function tree and try to find if it uses other keys as function arguments.
// TODO: Also process CONSTANT here. We can simplify GROUP BY x, x + 1 to GROUP BY x.
while (!candidates.empty())
{
auto candidate = candidates.back();
candidates.pop_back();
bool found = group_by_keys.contains(candidate);
switch (candidate->getNodeType())
{
case QueryTreeNodeType::FUNCTION:
{
auto * func = candidate->as<FunctionNode>();
auto & arguments = func->getArguments().getNodes();
if (arguments.empty())
return false;
if (!found)
{
for (auto it = arguments.rbegin(); it != arguments.rend(); ++it)
candidates.push_back(*it);
}
break;
}
case QueryTreeNodeType::COLUMN:
if (!found)
return false;
break;
default:
return false;
}
}
return true;
}
static void optimizeGroupingSet(QueryTreeNodes & grouping_set)
{
QueryTreeNodePtrWithHashSet group_by_keys(grouping_set.begin(), grouping_set.end());
QueryTreeNodes new_group_by_keys;
new_group_by_keys.reserve(grouping_set.size());
for (auto & group_by_elem : grouping_set)
{
if (!canBeEliminated(group_by_elem, group_by_keys))
new_group_by_keys.push_back(group_by_elem);
}
grouping_set = std::move(new_group_by_keys);
}
};
void OptimizeGroupByFunctionKeysPass::run(QueryTreeNodePtr query_tree_node, ContextPtr /*context*/)
{
OptimizeGroupByFunctionKeysVisitor().visit(query_tree_node);
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include <Analyzer/IQueryTreePass.h>
namespace DB
{
/* Eliminates functions of other keys in GROUP BY section.
* Ex.: GROUP BY x, f(x)
* Output: GROUP BY x
*/
class OptimizeGroupByFunctionKeysPass final : public IQueryTreePass
{
public:
String getName() override { return "OptimizeGroupByFunctionKeys"; }
String getDescription() override { return "Eliminates functions of other keys in GROUP BY section."; }
void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
};
}

View File

@ -1,6 +1,7 @@
#include <Analyzer/Passes/QueryAnalysisPass.h> #include <Analyzer/Passes/QueryAnalysisPass.h>
#include <Common/NamePrompter.h> #include <Common/NamePrompter.h>
#include <Common/ProfileEvents.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
@ -66,6 +67,14 @@
#include <Analyzer/UnionNode.h> #include <Analyzer/UnionNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h> #include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/QueryTreeBuilder.h> #include <Analyzer/QueryTreeBuilder.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/HashUtils.h>
namespace ProfileEvents
{
extern const Event ScalarSubqueriesGlobalCacheHit;
extern const Event ScalarSubqueriesCacheMiss;
}
#include <Common/checkStackSize.h> #include <Common/checkStackSize.h>
@ -1049,6 +1058,8 @@ private:
static bool isTableExpressionNodeType(QueryTreeNodeType node_type); static bool isTableExpressionNodeType(QueryTreeNodeType node_type);
static DataTypePtr getExpressionNodeResultTypeOrNull(const QueryTreeNodePtr & query_tree_node);
static ProjectionName calculateFunctionProjectionName(const QueryTreeNodePtr & function_node, static ProjectionName calculateFunctionProjectionName(const QueryTreeNodePtr & function_node,
const ProjectionNames & parameters_projection_names, const ProjectionNames & parameters_projection_names,
const ProjectionNames & arguments_projection_names); const ProjectionNames & arguments_projection_names);
@ -1097,7 +1108,7 @@ private:
static QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunctions(const std::string & function_name, ContextPtr context); static QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunctions(const std::string & function_name, ContextPtr context);
static void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, size_t subquery_depth, ContextPtr context); void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, size_t subquery_depth, ContextPtr context);
static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope); static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope);
@ -1207,6 +1218,9 @@ private:
/// Global resolve expression node to projection names map /// Global resolve expression node to projection names map
std::unordered_map<QueryTreeNodePtr, ProjectionNames> resolved_expressions; std::unordered_map<QueryTreeNodePtr, ProjectionNames> resolved_expressions;
/// Results of scalar sub queries
std::unordered_map<QueryTreeNodeConstRawPtrWithHash, std::shared_ptr<ConstantValue>> scalars;
}; };
/// Utility functions implementation /// Utility functions implementation
@ -1229,6 +1243,34 @@ bool QueryAnalyzer::isTableExpressionNodeType(QueryTreeNodeType node_type)
node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION; node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION;
} }
DataTypePtr QueryAnalyzer::getExpressionNodeResultTypeOrNull(const QueryTreeNodePtr & query_tree_node)
{
auto node_type = query_tree_node->getNodeType();
switch (node_type)
{
case QueryTreeNodeType::CONSTANT:
[[fallthrough]];
case QueryTreeNodeType::COLUMN:
{
return query_tree_node->getResultType();
}
case QueryTreeNodeType::FUNCTION:
{
auto & function_node = query_tree_node->as<FunctionNode &>();
if (function_node.isResolved())
return function_node.getResultType();
break;
}
default:
{
break;
}
}
return nullptr;
}
ProjectionName QueryAnalyzer::calculateFunctionProjectionName(const QueryTreeNodePtr & function_node, const ProjectionNames & parameters_projection_names, ProjectionName QueryAnalyzer::calculateFunctionProjectionName(const QueryTreeNodePtr & function_node, const ProjectionNames & parameters_projection_names,
const ProjectionNames & arguments_projection_names) const ProjectionNames & arguments_projection_names)
{ {
@ -1534,12 +1576,12 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection(
auto expression_identifier = Identifier(name); auto expression_identifier = Identifier(name);
valid_identifiers_result.insert(expression_identifier); valid_identifiers_result.insert(expression_identifier);
auto expression_node_type = expression->getNodeType(); auto result_type = getExpressionNodeResultTypeOrNull(expression);
if (identifier_is_compound && isExpressionNodeType(expression_node_type)) if (identifier_is_compound && result_type)
{ {
collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier,
expression->getResultType(), result_type,
expression_identifier, expression_identifier,
valid_identifiers_result); valid_identifiers_result);
} }
@ -1571,21 +1613,23 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection(
for (const auto & [argument_name, expression] : scope.expression_argument_name_to_node) for (const auto & [argument_name, expression] : scope.expression_argument_name_to_node)
{ {
assert(expression);
auto expression_node_type = expression->getNodeType(); auto expression_node_type = expression->getNodeType();
if (allow_expression_identifiers && isExpressionNodeType(expression_node_type)) if (allow_expression_identifiers && isExpressionNodeType(expression_node_type))
{ {
auto expression_identifier = Identifier(argument_name); auto expression_identifier = Identifier(argument_name);
valid_identifiers_result.insert(expression_identifier);
if (identifier_is_compound) auto result_type = getExpressionNodeResultTypeOrNull(expression);
if (identifier_is_compound && result_type)
{ {
collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier,
expression->getResultType(), result_type,
expression_identifier, expression_identifier,
valid_identifiers_result); valid_identifiers_result);
} }
valid_identifiers_result.insert(expression_identifier);
} }
else if (identifier_is_short && allow_function_identifiers && isFunctionExpressionNodeType(expression_node_type)) else if (identifier_is_short && allow_function_identifiers && isFunctionExpressionNodeType(expression_node_type))
{ {
@ -1687,6 +1731,16 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size
node->getNodeTypeName(), node->getNodeTypeName(),
node->formatASTForErrorMessage()); node->formatASTForErrorMessage());
auto scalars_iterator = scalars.find(node.get());
if (scalars_iterator != scalars.end())
{
ProfileEvents::increment(ProfileEvents::ScalarSubqueriesGlobalCacheHit);
node = std::make_shared<ConstantNode>(scalars_iterator->second, node);
return;
}
ProfileEvents::increment(ProfileEvents::ScalarSubqueriesCacheMiss);
auto subquery_context = Context::createCopy(context); auto subquery_context = Context::createCopy(context);
Settings subquery_settings = context->getSettings(); Settings subquery_settings = context->getSettings();
@ -1699,10 +1753,11 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size
auto io = interpreter->execute(); auto io = interpreter->execute();
Block block;
PullingAsyncPipelineExecutor executor(io.pipeline); PullingAsyncPipelineExecutor executor(io.pipeline);
io.pipeline.setProgressCallback(context->getProgressCallback()); io.pipeline.setProgressCallback(context->getProgressCallback());
Block block;
while (block.rows() == 0 && executor.pull(block)) while (block.rows() == 0 && executor.pull(block))
{ {
} }
@ -1743,7 +1798,6 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size
block = materializeBlock(block); block = materializeBlock(block);
size_t columns = block.columns(); size_t columns = block.columns();
// Block scalar;
Field scalar_value; Field scalar_value;
DataTypePtr scalar_type; DataTypePtr scalar_type;
@ -1770,6 +1824,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size
} }
auto constant_value = std::make_shared<ConstantValue>(std::move(scalar_value), std::move(scalar_type)); auto constant_value = std::make_shared<ConstantValue>(std::move(scalar_value), std::move(scalar_type));
scalars[node.get()] = constant_value;
node = std::make_shared<ConstantNode>(std::move(constant_value), node); node = std::make_shared<ConstantNode>(std::move(constant_value), node);
} }
@ -4297,7 +4352,8 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
bool force_grouping_standard_compatibility = scope.context->getSettingsRef().force_grouping_standard_compatibility; bool force_grouping_standard_compatibility = scope.context->getSettingsRef().force_grouping_standard_compatibility;
auto grouping_function = std::make_shared<FunctionGrouping>(force_grouping_standard_compatibility); auto grouping_function = std::make_shared<FunctionGrouping>(force_grouping_standard_compatibility);
auto grouping_function_adaptor = std::make_shared<FunctionToOverloadResolverAdaptor>(std::move(grouping_function)); auto grouping_function_adaptor = std::make_shared<FunctionToOverloadResolverAdaptor>(std::move(grouping_function));
function_node.resolveAsFunction(grouping_function_adaptor->build({})); function_node.resolveAsFunction(grouping_function_adaptor->build(argument_columns));
return result_projection_names; return result_projection_names;
} }
} }

View File

@ -17,6 +17,7 @@
#include <Parsers/ASTSetQuery.h> #include <Parsers/ASTSetQuery.h>
#include <Analyzer/Utils.h> #include <Analyzer/Utils.h>
#include <fmt/core.h>
namespace DB namespace DB
{ {
@ -179,6 +180,16 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s
buffer << '\n' << std::string(indent + 2, ' ') << "OFFSET\n"; buffer << '\n' << std::string(indent + 2, ' ') << "OFFSET\n";
getOffset()->dumpTreeImpl(buffer, format_state, indent + 4); getOffset()->dumpTreeImpl(buffer, format_state, indent + 4);
} }
if (hasSettingsChanges())
{
buffer << '\n' << std::string(indent + 2, ' ') << "SETTINGS";
for (const auto & change : settings_changes)
{
buffer << fmt::format(" {}={}", change.name, toString(change.value));
}
buffer << '\n';
}
} }
bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const

View File

@ -1,5 +1,19 @@
#include <memory>
#include <Analyzer/QueryTreePassManager.h> #include <Analyzer/QueryTreePassManager.h>
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <DataTypes/IDataType.h>
#include <Interpreters/Context.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/Utils.h>
#include <Analyzer/Passes/QueryAnalysisPass.h> #include <Analyzer/Passes/QueryAnalysisPass.h>
#include <Analyzer/Passes/CountDistinctPass.h> #include <Analyzer/Passes/CountDistinctPass.h>
#include <Analyzer/Passes/FunctionToSubcolumnsPass.h> #include <Analyzer/Passes/FunctionToSubcolumnsPass.h>
@ -14,17 +28,11 @@
#include <Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h> #include <Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h>
#include <Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h> #include <Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h>
#include <Analyzer/Passes/FuseFunctionsPass.h> #include <Analyzer/Passes/FuseFunctionsPass.h>
#include <Analyzer/Passes/OptimizeGroupByFunctionKeysPass.h>
#include <Analyzer/Passes/IfTransformStringsToEnumPass.h> #include <Analyzer/Passes/IfTransformStringsToEnumPass.h>
#include <Analyzer/Passes/ConvertOrLikeChainPass.h>
#include <Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.h> #include <Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.h>
#include <Analyzer/Passes/GroupingFunctionsResolvePass.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Interpreters/Context.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Common/Exception.h>
namespace DB namespace DB
{ {
@ -45,24 +53,6 @@ namespace
*/ */
class ValidationChecker : public InDepthQueryTreeVisitor<ValidationChecker> class ValidationChecker : public InDepthQueryTreeVisitor<ValidationChecker>
{ {
String pass_name;
void visitColumn(ColumnNode * column) const
{
if (column->getColumnSourceOrNull() == nullptr)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Column {} {} query tree node does not have valid source node after running {} pass",
column->getColumnName(), column->getColumnType(), pass_name);
}
void visitFunction(FunctionNode * function) const
{
if (!function->isResolved())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Function {} is not resolved after running {} pass",
function->toAST()->formatForErrorMessage(), pass_name);
}
public: public:
explicit ValidationChecker(String pass_name_) explicit ValidationChecker(String pass_name_)
: pass_name(std::move(pass_name_)) : pass_name(std::move(pass_name_))
@ -75,6 +65,57 @@ public:
else if (auto * function = node->as<FunctionNode>()) else if (auto * function = node->as<FunctionNode>())
return visitFunction(function); return visitFunction(function);
} }
private:
void visitColumn(ColumnNode * column) const
{
if (column->getColumnSourceOrNull() == nullptr && column->getColumnName() != "__grouping_set")
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Column {} {} query tree node does not have valid source node after running {} pass",
column->getColumnName(), column->getColumnType(), pass_name);
}
void visitFunction(FunctionNode * function) const
{
if (!function->isResolved())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Function {} is not resolved after running {} pass",
function->toAST()->formatForErrorMessage(), pass_name);
if (isNameOfInFunction(function->getFunctionName()))
return;
const auto & expected_argument_types = function->getArgumentTypes();
size_t expected_argument_types_size = expected_argument_types.size();
auto actual_argument_columns = function->getArgumentColumns();
if (expected_argument_types_size != actual_argument_columns.size())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Function {} expects {} arguments but has {} after running {} pass",
function->toAST()->formatForErrorMessage(),
expected_argument_types_size,
actual_argument_columns.size(),
pass_name);
for (size_t i = 0; i < expected_argument_types_size; ++i)
{
// Skip lambdas
if (WhichDataType(expected_argument_types[i]).isFunction())
continue;
if (!expected_argument_types[i]->equals(*actual_argument_columns[i].type))
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Function {} expects {} argument to have {} type but receives {} after running {} pass",
function->toAST()->formatForErrorMessage(),
i + 1,
expected_argument_types[i]->getName(),
actual_argument_columns[i].type->getName(),
pass_name);
}
}
}
String pass_name;
}; };
#endif #endif
@ -88,7 +129,6 @@ public:
* TODO: Support setting optimize_using_constraints. * TODO: Support setting optimize_using_constraints.
* TODO: Support setting optimize_substitute_columns. * TODO: Support setting optimize_substitute_columns.
* TODO: Support GROUP BY injective function elimination. * TODO: Support GROUP BY injective function elimination.
* TODO: Support GROUP BY functions of other keys elimination.
* TODO: Support setting optimize_move_functions_out_of_any. * TODO: Support setting optimize_move_functions_out_of_any.
* TODO: Support setting optimize_aggregators_of_group_by_keys. * TODO: Support setting optimize_aggregators_of_group_by_keys.
* TODO: Support setting optimize_duplicate_order_by_and_distinct. * TODO: Support setting optimize_duplicate_order_by_and_distinct.
@ -195,6 +235,9 @@ void addQueryTreePasses(QueryTreePassManager & manager)
if (settings.optimize_injective_functions_inside_uniq) if (settings.optimize_injective_functions_inside_uniq)
manager.addPass(std::make_unique<UniqInjectiveFunctionsEliminationPass>()); manager.addPass(std::make_unique<UniqInjectiveFunctionsEliminationPass>());
if (settings.optimize_group_by_function_keys)
manager.addPass(std::make_unique<OptimizeGroupByFunctionKeysPass>());
if (settings.optimize_multiif_to_if) if (settings.optimize_multiif_to_if)
manager.addPass(std::make_unique<MultiIfToIfPass>()); manager.addPass(std::make_unique<MultiIfToIfPass>());
@ -214,6 +257,10 @@ void addQueryTreePasses(QueryTreePassManager & manager)
if (settings.optimize_if_transform_strings_to_enum) if (settings.optimize_if_transform_strings_to_enum)
manager.addPass(std::make_unique<IfTransformStringsToEnumPass>()); manager.addPass(std::make_unique<IfTransformStringsToEnumPass>());
manager.addPass(std::make_unique<ConvertOrLikeChainPass>());
manager.addPass(std::make_unique<GroupingFunctionsResolvePass>());
} }
} }

View File

@ -130,7 +130,7 @@ BackupEntries BackupEntriesCollector::run()
Strings BackupEntriesCollector::setStage(const String & new_stage, const String & message) Strings BackupEntriesCollector::setStage(const String & new_stage, const String & message)
{ {
LOG_TRACE(log, "{}", toUpperFirst(new_stage)); LOG_TRACE(log, fmt::runtime(toUpperFirst(new_stage)));
current_stage = new_stage; current_stage = new_stage;
backup_coordination->setStage(backup_settings.host_id, new_stage, message); backup_coordination->setStage(backup_settings.host_id, new_stage, message);
@ -215,7 +215,7 @@ void BackupEntriesCollector::gatherMetadataAndCheckConsistency()
if (std::chrono::steady_clock::now() > consistent_metadata_snapshot_end_time) if (std::chrono::steady_clock::now() > consistent_metadata_snapshot_end_time)
inconsistency_error->rethrow(); inconsistency_error->rethrow();
else else
LOG_WARNING(log, "{}", inconsistency_error->displayText()); LOG_WARNING(log, getExceptionMessageAndPattern(*inconsistency_error, /* with_stacktrace */ false));
} }
auto sleep_time = getSleepTimeAfterInconsistencyError(pass); auto sleep_time = getSleepTimeAfterInconsistencyError(pass);

View File

@ -7,6 +7,7 @@
#include <Storages/IStorage_fwd.h> #include <Storages/IStorage_fwd.h>
#include <Storages/TableLockHolder.h> #include <Storages/TableLockHolder.h>
#include <filesystem> #include <filesystem>
#include <queue>
namespace DB namespace DB

View File

@ -19,7 +19,7 @@ BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile(
std::unique_ptr<SeekableReadBuffer> BackupEntryFromAppendOnlyFile::getReadBuffer() const std::unique_ptr<SeekableReadBuffer> BackupEntryFromAppendOnlyFile::getReadBuffer() const
{ {
auto buf = BackupEntryFromImmutableFile::getReadBuffer(); auto buf = BackupEntryFromImmutableFile::getReadBuffer();
return std::make_unique<LimitSeekableReadBuffer>(std::move(buf), limit); return std::make_unique<LimitSeekableReadBuffer>(std::move(buf), 0, limit);
} }
} }

View File

@ -12,16 +12,19 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED; extern const int NOT_IMPLEMENTED;
} }
void IBackupWriter::copyFileThroughBuffer(std::unique_ptr<SeekableReadBuffer> && source, const String & file_name) void IBackupWriter::copyDataToFile(const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name)
{ {
auto write_buffer = writeFile(file_name); auto read_buffer = create_read_buffer();
copyData(*source, *write_buffer); if (offset)
read_buffer->seek(offset, SEEK_SET);
auto write_buffer = writeFile(dest_file_name);
copyData(*read_buffer, *write_buffer, size);
write_buffer->finalize(); write_buffer->finalize();
} }
void IBackupWriter::copyFileNative(DiskPtr /* from_disk */, const String & /* file_name_from */, const String & /* file_name_to */) void IBackupWriter::copyFileNative(
DiskPtr /* src_disk */, const String & /* src_file_name */, UInt64 /* src_offset */, UInt64 /* src_size */, const String & /* dest_file_name */)
{ {
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Native copy not implemented for backup writer"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Native copy not implemented for backup writer");
} }
} }

View File

@ -24,6 +24,8 @@ public:
class IBackupWriter /// BackupWriterFile, BackupWriterDisk class IBackupWriter /// BackupWriterFile, BackupWriterDisk
{ {
public: public:
using CreateReadBufferFunction = std::function<std::unique_ptr<SeekableReadBuffer>()>;
virtual ~IBackupWriter() = default; virtual ~IBackupWriter() = default;
virtual bool fileExists(const String & file_name) = 0; virtual bool fileExists(const String & file_name) = 0;
virtual UInt64 getFileSize(const String & file_name) = 0; virtual UInt64 getFileSize(const String & file_name) = 0;
@ -32,14 +34,9 @@ public:
virtual void removeFile(const String & file_name) = 0; virtual void removeFile(const String & file_name) = 0;
virtual void removeFiles(const Strings & file_names) = 0; virtual void removeFiles(const Strings & file_names) = 0;
virtual DataSourceDescription getDataSourceDescription() const = 0; virtual DataSourceDescription getDataSourceDescription() const = 0;
virtual void copyFileThroughBuffer(std::unique_ptr<SeekableReadBuffer> && source, const String & file_name); virtual void copyDataToFile(const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name);
virtual bool supportNativeCopy(DataSourceDescription /* data_source_description */) const { return false; }
virtual bool supportNativeCopy(DataSourceDescription /* data_source_description */) const virtual void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name);
{
return false;
}
virtual void copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to);
}; };
} }

View File

@ -105,13 +105,21 @@ bool BackupWriterDisk::supportNativeCopy(DataSourceDescription data_source_descr
return data_source_description == disk->getDataSourceDescription(); return data_source_description == disk->getDataSourceDescription();
} }
void BackupWriterDisk::copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to) void BackupWriterDisk::copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name)
{ {
if (!from_disk) if (!src_disk)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot natively copy data to disk without source disk"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot natively copy data to disk without source disk");
auto file_path = path / file_name_to;
if ((src_offset != 0) || (src_size != src_disk->getFileSize(src_file_name)))
{
auto create_read_buffer = [src_disk, src_file_name] { return src_disk->readFile(src_file_name); };
copyDataToFile(create_read_buffer, src_offset, src_size, dest_file_name);
return;
}
auto file_path = path / dest_file_name;
disk->createDirectories(file_path.parent_path()); disk->createDirectories(file_path.parent_path());
from_disk->copyFile(file_name_from, *disk, file_path); src_disk->copyFile(src_file_name, *disk, file_path);
} }
} }

View File

@ -39,8 +39,8 @@ public:
DataSourceDescription getDataSourceDescription() const override; DataSourceDescription getDataSourceDescription() const override;
bool supportNativeCopy(DataSourceDescription data_source_description) const override; bool supportNativeCopy(DataSourceDescription data_source_description) const override;
void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override;
void copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to) override;
private: private:
DiskPtr disk; DiskPtr disk;
std::filesystem::path path; std::filesystem::path path;

View File

@ -125,17 +125,24 @@ bool BackupWriterFile::supportNativeCopy(DataSourceDescription data_source_descr
return data_source_description == getDataSourceDescription(); return data_source_description == getDataSourceDescription();
} }
void BackupWriterFile::copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to) void BackupWriterFile::copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name)
{ {
auto file_path = path / file_name_to;
fs::create_directories(file_path.parent_path());
std::string abs_source_path; std::string abs_source_path;
if (from_disk) if (src_disk)
abs_source_path = fullPath(from_disk, file_name_from); abs_source_path = fullPath(src_disk, src_file_name);
else else
abs_source_path = fs::absolute(file_name_from); abs_source_path = fs::absolute(src_file_name);
fs::copy(abs_source_path, file_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); if ((src_offset != 0) || (src_size != fs::file_size(abs_source_path)))
{
auto create_read_buffer = [abs_source_path] { return createReadBufferFromFileBase(abs_source_path, {}); };
copyDataToFile(create_read_buffer, src_offset, src_size, dest_file_name);
return;
}
auto file_path = path / dest_file_name;
fs::create_directories(file_path.parent_path());
fs::copy(abs_source_path, file_path, fs::copy_options::overwrite_existing);
} }
} }

View File

@ -35,8 +35,7 @@ public:
void removeFiles(const Strings & file_names) override; void removeFiles(const Strings & file_names) override;
DataSourceDescription getDataSourceDescription() const override; DataSourceDescription getDataSourceDescription() const override;
bool supportNativeCopy(DataSourceDescription data_source_description) const override; bool supportNativeCopy(DataSourceDescription data_source_description) const override;
void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override;
void copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to) override;
private: private:
std::filesystem::path path; std::filesystem::path path;

View File

@ -4,17 +4,19 @@
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Interpreters/threadPoolCallbackRunner.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Storages/StorageS3Settings.h>
#include <IO/IOThreadPool.h> #include <IO/IOThreadPool.h>
#include <IO/ReadBufferFromS3.h> #include <IO/ReadBufferFromS3.h>
#include <IO/WriteBufferFromS3.h> #include <IO/WriteBufferFromS3.h>
#include <IO/HTTPHeaderEntries.h> #include <IO/HTTPHeaderEntries.h>
#include <IO/S3/copyDataToS3.h>
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include <aws/core/auth/AWSCredentials.h> #include <aws/core/auth/AWSCredentials.h>
#include <aws/s3/S3Client.h> #include <aws/s3/S3Client.h>
#include <filesystem> #include <aws/s3/model/DeleteObjectRequest.h>
#include <aws/s3/model/DeleteObjectsRequest.h>
#include <aws/s3/model/ListObjectsRequest.h> #include <aws/s3/model/ListObjectsRequest.h>
#include <filesystem>
namespace fs = std::filesystem; namespace fs = std::filesystem;
@ -24,7 +26,6 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int S3_ERROR; extern const int S3_ERROR;
extern const int INVALID_CONFIG_PARAMETER;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
@ -150,179 +151,33 @@ bool BackupWriterS3::supportNativeCopy(DataSourceDescription data_source_descrip
return getDataSourceDescription() == data_source_description; return getDataSourceDescription() == data_source_description;
} }
void BackupWriterS3::copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name)
void BackupWriterS3::copyObjectImpl(
const String & src_bucket,
const String & src_key,
const String & dst_bucket,
const String & dst_key,
const Aws::S3::Model::HeadObjectResult & head,
const std::optional<ObjectAttributes> & metadata) const
{ {
size_t size = head.GetContentLength(); if (!src_disk)
LOG_TRACE(log, "Copying {} bytes using single-operation copy", size);
Aws::S3::Model::CopyObjectRequest request;
request.SetCopySource(src_bucket + "/" + src_key);
request.SetBucket(dst_bucket);
request.SetKey(dst_key);
if (metadata)
{
request.SetMetadata(*metadata);
request.SetMetadataDirective(Aws::S3::Model::MetadataDirective::REPLACE);
}
auto outcome = client->CopyObject(request);
if (!outcome.IsSuccess() && (outcome.GetError().GetExceptionName() == "EntityTooLarge"
|| outcome.GetError().GetExceptionName() == "InvalidRequest"))
{ // Can't come here with MinIO, MinIO allows single part upload for large objects.
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata);
return;
}
if (!outcome.IsSuccess())
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
}
void BackupWriterS3::copyObjectMultipartImpl(
const String & src_bucket,
const String & src_key,
const String & dst_bucket,
const String & dst_key,
const Aws::S3::Model::HeadObjectResult & head,
const std::optional<ObjectAttributes> & metadata) const
{
size_t size = head.GetContentLength();
LOG_TRACE(log, "Copying {} bytes using multipart upload copy", size);
String multipart_upload_id;
{
Aws::S3::Model::CreateMultipartUploadRequest request;
request.SetBucket(dst_bucket);
request.SetKey(dst_key);
if (metadata)
request.SetMetadata(*metadata);
auto outcome = client->CreateMultipartUpload(request);
if (!outcome.IsSuccess())
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
multipart_upload_id = outcome.GetResult().GetUploadId();
}
std::vector<String> part_tags;
size_t position = 0;
const auto & settings = request_settings.getUploadSettings();
size_t upload_part_size = settings.min_upload_part_size;
for (size_t part_number = 1; position < size; ++part_number)
{
/// Check that part number is not too big.
if (part_number > settings.max_part_number)
{
throw Exception(
ErrorCodes::INVALID_CONFIG_PARAMETER,
"Part number exceeded {} while writing {} bytes to S3. Check min_upload_part_size = {}, max_upload_part_size = {}, "
"upload_part_size_multiply_factor = {}, upload_part_size_multiply_parts_count_threshold = {}, max_single_operation_copy_size = {}",
settings.max_part_number, size, settings.min_upload_part_size, settings.max_upload_part_size,
settings.upload_part_size_multiply_factor, settings.upload_part_size_multiply_parts_count_threshold,
settings.max_single_operation_copy_size);
}
size_t next_position = std::min(position + upload_part_size, size);
/// Make a copy request to copy a part.
Aws::S3::Model::UploadPartCopyRequest part_request;
part_request.SetCopySource(src_bucket + "/" + src_key);
part_request.SetBucket(dst_bucket);
part_request.SetKey(dst_key);
part_request.SetUploadId(multipart_upload_id);
part_request.SetPartNumber(static_cast<int>(part_number));
part_request.SetCopySourceRange(fmt::format("bytes={}-{}", position, next_position - 1));
auto outcome = client->UploadPartCopy(part_request);
if (!outcome.IsSuccess())
{
Aws::S3::Model::AbortMultipartUploadRequest abort_request;
abort_request.SetBucket(dst_bucket);
abort_request.SetKey(dst_key);
abort_request.SetUploadId(multipart_upload_id);
client->AbortMultipartUpload(abort_request);
// In error case we throw exception later with first error from UploadPartCopy
}
if (!outcome.IsSuccess())
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
auto etag = outcome.GetResult().GetCopyPartResult().GetETag();
part_tags.push_back(etag);
position = next_position;
/// Maybe increase `upload_part_size` (we need to increase it sometimes to keep `part_number` less or equal than `max_part_number`).
if (part_number % settings.upload_part_size_multiply_parts_count_threshold == 0)
{
upload_part_size *= settings.upload_part_size_multiply_factor;
upload_part_size = std::min(upload_part_size, settings.max_upload_part_size);
}
}
{
Aws::S3::Model::CompleteMultipartUploadRequest req;
req.SetBucket(dst_bucket);
req.SetKey(dst_key);
req.SetUploadId(multipart_upload_id);
Aws::S3::Model::CompletedMultipartUpload multipart_upload;
for (size_t i = 0; i < part_tags.size(); ++i)
{
Aws::S3::Model::CompletedPart part;
multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(static_cast<int>(i) + 1));
}
req.SetMultipartUpload(multipart_upload);
auto outcome = client->CompleteMultipartUpload(req);
if (!outcome.IsSuccess())
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
}
}
void BackupWriterS3::copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to)
{
if (!from_disk)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot natively copy data to disk without source disk"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot natively copy data to disk without source disk");
auto objects = from_disk->getStorageObjects(file_name_from); auto objects = src_disk->getStorageObjects(src_file_name);
if (objects.size() > 1) if (objects.size() > 1)
{ {
copyFileThroughBuffer(from_disk->readFile(file_name_from), file_name_to); auto create_read_buffer = [src_disk, src_file_name] { return src_disk->readFile(src_file_name); };
copyDataToFile(create_read_buffer, src_offset, src_size, dest_file_name);
} }
else else
{ {
auto object_storage = from_disk->getObjectStorage(); auto object_storage = src_disk->getObjectStorage();
std::string source_bucket = object_storage->getObjectsNamespace(); std::string src_bucket = object_storage->getObjectsNamespace();
auto file_path = fs::path(s3_uri.key) / file_name_to; auto file_path = fs::path(s3_uri.key) / dest_file_name;
copyFileS3ToS3(client, src_bucket, objects[0].absolute_path, src_offset, src_size, s3_uri.bucket, file_path, request_settings, {},
auto head = S3::headObject(*client, source_bucket, objects[0].absolute_path).GetResult(); threadPoolCallbackRunner<void>(IOThreadPool::get(), "BackupWriterS3"));
if (static_cast<size_t>(head.GetContentLength()) < request_settings.getUploadSettings().max_single_operation_copy_size)
{
copyObjectImpl(
source_bucket, objects[0].absolute_path, s3_uri.bucket, file_path, head);
}
else
{
copyObjectMultipartImpl(
source_bucket, objects[0].absolute_path, s3_uri.bucket, file_path, head);
}
} }
} }
void BackupWriterS3::copyDataToFile(
const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name)
{
copyDataToS3(create_read_buffer, offset, size, client, s3_uri.bucket, fs::path(s3_uri.key) / dest_file_name, request_settings, {},
threadPoolCallbackRunner<void>(IOThreadPool::get(), "BackupWriterS3"));
}
BackupWriterS3::~BackupWriterS3() = default; BackupWriterS3::~BackupWriterS3() = default;

View File

@ -4,22 +4,11 @@
#if USE_AWS_S3 #if USE_AWS_S3
#include <Backups/BackupIO.h> #include <Backups/BackupIO.h>
#include <IO/S3Common.h>
#include <IO/ReadSettings.h> #include <IO/ReadSettings.h>
#include <IO/S3Common.h>
#include <Storages/StorageS3Settings.h> #include <Storages/StorageS3Settings.h>
#include <aws/s3/S3Client.h> #include <aws/s3/S3Client.h>
#include <aws/s3/model/CopyObjectRequest.h>
#include <aws/s3/model/ListObjectsV2Request.h>
#include <aws/s3/model/HeadObjectRequest.h>
#include <aws/s3/model/DeleteObjectRequest.h>
#include <aws/s3/model/DeleteObjectsRequest.h>
#include <aws/s3/model/CreateMultipartUploadRequest.h>
#include <aws/s3/model/CompleteMultipartUploadRequest.h>
#include <aws/s3/model/UploadPartCopyRequest.h>
#include <aws/s3/model/AbortMultipartUploadRequest.h>
#include <aws/s3/model/HeadObjectResult.h>
#include <aws/s3/model/ListObjectsV2Result.h>
namespace DB namespace DB
{ {
@ -54,12 +43,15 @@ public:
UInt64 getFileSize(const String & file_name) override; UInt64 getFileSize(const String & file_name) override;
bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override;
std::unique_ptr<WriteBuffer> writeFile(const String & file_name) override; std::unique_ptr<WriteBuffer> writeFile(const String & file_name) override;
void copyDataToFile(const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name) override;
void removeFile(const String & file_name) override; void removeFile(const String & file_name) override;
void removeFiles(const Strings & file_names) override; void removeFiles(const Strings & file_names) override;
DataSourceDescription getDataSourceDescription() const override; DataSourceDescription getDataSourceDescription() const override;
bool supportNativeCopy(DataSourceDescription data_source_description) const override; bool supportNativeCopy(DataSourceDescription data_source_description) const override;
void copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to) override; void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override;
private: private:
void copyObjectImpl( void copyObjectImpl(
@ -67,7 +59,7 @@ private:
const String & src_key, const String & src_key,
const String & dst_bucket, const String & dst_bucket,
const String & dst_key, const String & dst_key,
const Aws::S3::Model::HeadObjectResult & head, size_t size,
const std::optional<ObjectAttributes> & metadata = std::nullopt) const; const std::optional<ObjectAttributes> & metadata = std::nullopt) const;
void copyObjectMultipartImpl( void copyObjectMultipartImpl(
@ -75,7 +67,7 @@ private:
const String & src_key, const String & src_key,
const String & dst_bucket, const String & dst_bucket,
const String & dst_key, const String & dst_key,
const Aws::S3::Model::HeadObjectResult & head, size_t size,
const std::optional<ObjectAttributes> & metadata = std::nullopt) const; const std::optional<ObjectAttributes> & metadata = std::nullopt) const;
void removeFilesBatch(const Strings & file_names); void removeFilesBatch(const Strings & file_names);

View File

@ -874,23 +874,18 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
/// We need to copy whole file without archive, we can do it faster /// We need to copy whole file without archive, we can do it faster
/// if source and destination are compatible /// if source and destination are compatible
if (!use_archives && info.base_size == 0 && writer->supportNativeCopy(reader_description)) if (!use_archives && writer->supportNativeCopy(reader_description))
{ {
/// Should be much faster than writing data through server. /// Should be much faster than writing data through server.
LOG_TRACE(log, "Will copy file {} using native copy", adjusted_path); LOG_TRACE(log, "Will copy file {} using native copy", adjusted_path);
/// NOTE: `mutex` must be unlocked here otherwise writing will be in one thread maximum and hence slow. /// NOTE: `mutex` must be unlocked here otherwise writing will be in one thread maximum and hence slow.
writer->copyFileNative(entry->tryGetDiskIfExists(), entry->getFilePath(), info.data_file_name); writer->copyFileNative(entry->tryGetDiskIfExists(), entry->getFilePath(), info.base_size, info.size - info.base_size, info.data_file_name);
} }
else else
{ {
LOG_TRACE(log, "Will copy file {} through memory buffers", adjusted_path); LOG_TRACE(log, "Will copy file {}", adjusted_path);
auto read_buffer = entry->getReadBuffer();
/// If we have prefix in base we will seek to the start of the suffix which differs
if (info.base_size != 0)
read_buffer->seek(info.base_size, SEEK_SET);
if (!num_files_written) if (!num_files_written)
checkLockFile(true); checkLockFile(true);
@ -919,13 +914,18 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
coordination->updateFileInfo(info); coordination->updateFileInfo(info);
} }
auto out = getArchiveWriter(current_archive_suffix)->writeFile(info.data_file_name); auto out = getArchiveWriter(current_archive_suffix)->writeFile(info.data_file_name);
auto read_buffer = entry->getReadBuffer();
if (info.base_size != 0)
read_buffer->seek(info.base_size, SEEK_SET);
copyData(*read_buffer, *out); copyData(*read_buffer, *out);
out->finalize(); out->finalize();
} }
else else
{ {
auto create_read_buffer = [entry] { return entry->getReadBuffer(); };
/// NOTE: `mutex` must be unlocked here otherwise writing will be in one thread maximum and hence slow. /// NOTE: `mutex` must be unlocked here otherwise writing will be in one thread maximum and hence slow.
writer->copyFileThroughBuffer(std::move(read_buffer), info.data_file_name); writer->copyDataToFile(create_read_buffer, info.base_size, info.size - info.base_size, info.data_file_name);
} }
} }

View File

@ -6,7 +6,7 @@
#include <Parsers/ASTSetQuery.h> #include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <Backups/SettingsFieldOptionalUUID.h>
namespace DB namespace DB
{ {
@ -16,48 +16,6 @@ namespace ErrorCodes
extern const int WRONG_BACKUP_SETTINGS; extern const int WRONG_BACKUP_SETTINGS;
} }
namespace
{
struct SettingFieldOptionalUUID
{
std::optional<UUID> value;
explicit SettingFieldOptionalUUID(const std::optional<UUID> & value_) : value(value_) {}
explicit SettingFieldOptionalUUID(const Field & field)
{
if (field.getType() == Field::Types::Null)
{
value = std::nullopt;
return;
}
if (field.getType() == Field::Types::String)
{
const String & str = field.get<const String &>();
if (str.empty())
{
value = std::nullopt;
return;
}
UUID id;
if (tryParse(id, str))
{
value = id;
return;
}
}
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Cannot parse uuid from {}", field);
}
explicit operator Field() const { return Field(value ? toString(*value) : ""); }
};
}
/// List of backup settings except base_backup_name and cluster_host_ids. /// List of backup settings except base_backup_name and cluster_host_ids.
#define LIST_OF_BACKUP_SETTINGS(M) \ #define LIST_OF_BACKUP_SETTINGS(M) \
M(String, id) \ M(String, id) \

View File

@ -30,6 +30,7 @@ namespace ErrorCodes
{ {
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int CONCURRENT_ACCESS_NOT_SUPPORTED;
} }
using OperationID = BackupsWorker::OperationID; using OperationID = BackupsWorker::OperationID;
@ -121,10 +122,12 @@ namespace
} }
BackupsWorker::BackupsWorker(size_t num_backup_threads, size_t num_restore_threads) BackupsWorker::BackupsWorker(size_t num_backup_threads, size_t num_restore_threads, bool allow_concurrent_backups_, bool allow_concurrent_restores_)
: backups_thread_pool(num_backup_threads, /* max_free_threads = */ 0, num_backup_threads) : backups_thread_pool(num_backup_threads, /* max_free_threads = */ 0, num_backup_threads)
, restores_thread_pool(num_restore_threads, /* max_free_threads = */ 0, num_restore_threads) , restores_thread_pool(num_restore_threads, /* max_free_threads = */ 0, num_restore_threads)
, log(&Poco::Logger::get("BackupsWorker")) , log(&Poco::Logger::get("BackupsWorker"))
, allow_concurrent_backups(allow_concurrent_backups_)
, allow_concurrent_restores(allow_concurrent_restores_)
{ {
/// We set max_free_threads = 0 because we don't want to keep any threads if there is no BACKUP or RESTORE query running right now. /// We set max_free_threads = 0 because we don't want to keep any threads if there is no BACKUP or RESTORE query running right now.
} }
@ -157,6 +160,16 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
else else
backup_id = toString(*backup_settings.backup_uuid); backup_id = toString(*backup_settings.backup_uuid);
/// Check if there are no concurrent backups
if (num_active_backups && !allow_concurrent_backups)
{
/// If its an internal backup and we currently have 1 active backup, it could be the original query, validate using backup_uuid
if (!(num_active_backups == 1 && backup_settings.internal && getAllActiveBackupInfos().at(0).id == toString(*backup_settings.backup_uuid)))
{
throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent backups not supported, turn on setting 'allow_concurrent_backups'");
}
}
std::shared_ptr<IBackupCoordination> backup_coordination; std::shared_ptr<IBackupCoordination> backup_coordination;
if (backup_settings.internal) if (backup_settings.internal)
{ {
@ -370,6 +383,9 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
auto restore_query = std::static_pointer_cast<ASTBackupQuery>(query->clone()); auto restore_query = std::static_pointer_cast<ASTBackupQuery>(query->clone());
auto restore_settings = RestoreSettings::fromRestoreQuery(*restore_query); auto restore_settings = RestoreSettings::fromRestoreQuery(*restore_query);
if (!restore_settings.backup_uuid)
restore_settings.backup_uuid = UUIDHelpers::generateV4();
/// `restore_id` will be used as a key to the `infos` map, so it should be unique. /// `restore_id` will be used as a key to the `infos` map, so it should be unique.
OperationID restore_id; OperationID restore_id;
if (restore_settings.internal) if (restore_settings.internal)
@ -377,7 +393,17 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
else if (!restore_settings.id.empty()) else if (!restore_settings.id.empty())
restore_id = restore_settings.id; restore_id = restore_settings.id;
else else
restore_id = toString(UUIDHelpers::generateV4()); restore_id = toString(*restore_settings.backup_uuid);
/// Check if there are no concurrent restores
if (num_active_restores && !allow_concurrent_restores)
{
/// If its an internal restore and we currently have 1 active restore, it could be the original query, validate using iz
if (!(num_active_restores == 1 && restore_settings.internal && getAllActiveRestoreInfos().at(0).id == toString(*restore_settings.backup_uuid)))
{
throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'");
}
}
std::shared_ptr<IRestoreCoordination> restore_coordination; std::shared_ptr<IRestoreCoordination> restore_coordination;
if (restore_settings.internal) if (restore_settings.internal)
@ -471,6 +497,7 @@ void BackupsWorker::doRestore(
backup_open_params.context = context; backup_open_params.context = context;
backup_open_params.backup_info = backup_info; backup_open_params.backup_info = backup_info;
backup_open_params.base_backup_info = restore_settings.base_backup_info; backup_open_params.base_backup_info = restore_settings.base_backup_info;
backup_open_params.backup_uuid = restore_settings.backup_uuid;
backup_open_params.password = restore_settings.password; backup_open_params.password = restore_settings.password;
BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params);
@ -687,6 +714,30 @@ std::vector<BackupsWorker::Info> BackupsWorker::getAllInfos() const
return res_infos; return res_infos;
} }
std::vector<BackupsWorker::Info> BackupsWorker::getAllActiveBackupInfos() const
{
std::vector<Info> res_infos;
std::lock_guard lock{infos_mutex};
for (const auto & info : infos | boost::adaptors::map_values)
{
if (info.status==BackupStatus::CREATING_BACKUP)
res_infos.push_back(info);
}
return res_infos;
}
std::vector<BackupsWorker::Info> BackupsWorker::getAllActiveRestoreInfos() const
{
std::vector<Info> res_infos;
std::lock_guard lock{infos_mutex};
for (const auto & info : infos | boost::adaptors::map_values)
{
if (info.status==BackupStatus::RESTORING)
res_infos.push_back(info);
}
return res_infos;
}
void BackupsWorker::shutdown() void BackupsWorker::shutdown()
{ {
bool has_active_backups_and_restores = (num_active_backups || num_active_restores); bool has_active_backups_and_restores = (num_active_backups || num_active_restores);

View File

@ -23,7 +23,7 @@ class IRestoreCoordination;
class BackupsWorker class BackupsWorker
{ {
public: public:
BackupsWorker(size_t num_backup_threads, size_t num_restore_threads); BackupsWorker(size_t num_backup_threads, size_t num_restore_threads, bool allow_concurrent_backups_, bool allow_concurrent_restores_);
/// Waits until all tasks have been completed. /// Waits until all tasks have been completed.
void shutdown(); void shutdown();
@ -103,6 +103,8 @@ private:
void setStatus(const OperationID & id, BackupStatus status, bool throw_if_error = true); void setStatus(const OperationID & id, BackupStatus status, bool throw_if_error = true);
void setStatusSafe(const String & id, BackupStatus status) { setStatus(id, status, false); } void setStatusSafe(const String & id, BackupStatus status) { setStatus(id, status, false); }
void setNumFilesAndSize(const OperationID & id, size_t num_files, UInt64 uncompressed_size, UInt64 compressed_size); void setNumFilesAndSize(const OperationID & id, size_t num_files, UInt64 uncompressed_size, UInt64 compressed_size);
std::vector<Info> getAllActiveBackupInfos() const;
std::vector<Info> getAllActiveRestoreInfos() const;
ThreadPool backups_thread_pool; ThreadPool backups_thread_pool;
ThreadPool restores_thread_pool; ThreadPool restores_thread_pool;
@ -113,6 +115,8 @@ private:
std::atomic<size_t> num_active_restores = 0; std::atomic<size_t> num_active_restores = 0;
mutable std::mutex infos_mutex; mutable std::mutex infos_mutex;
Poco::Logger * log; Poco::Logger * log;
const bool allow_concurrent_backups;
const bool allow_concurrent_restores;
}; };
} }

View File

@ -7,6 +7,7 @@
#include <Parsers/ASTSetQuery.h> #include <Parsers/ASTSetQuery.h>
#include <boost/algorithm/string/predicate.hpp> #include <boost/algorithm/string/predicate.hpp>
#include <Common/FieldVisitorConvertToNumber.h> #include <Common/FieldVisitorConvertToNumber.h>
#include <Backups/SettingsFieldOptionalUUID.h>
namespace DB namespace DB
@ -162,7 +163,9 @@ namespace
M(RestoreUDFCreationMode, create_function) \ M(RestoreUDFCreationMode, create_function) \
M(Bool, internal) \ M(Bool, internal) \
M(String, host_id) \ M(String, host_id) \
M(String, coordination_zk_path) M(String, coordination_zk_path) \
M(OptionalUUID, backup_uuid)
RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query)
{ {

View File

@ -122,6 +122,11 @@ struct RestoreSettings
/// Path in Zookeeper used to coordinate restoring process while executing by RESTORE ON CLUSTER. /// Path in Zookeeper used to coordinate restoring process while executing by RESTORE ON CLUSTER.
String coordination_zk_path; String coordination_zk_path;
/// Internal, should not be specified by user.
/// UUID of the backup. If it's not set it will be generated randomly.
/// This is used to validate internal restores when allow_concurrent_restores is turned off
std::optional<UUID> backup_uuid;
static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query);
void copySettingsToQuery(ASTBackupQuery & query) const; void copySettingsToQuery(ASTBackupQuery & query) const;
}; };

View File

@ -145,7 +145,7 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode)
void RestorerFromBackup::setStage(const String & new_stage, const String & message) void RestorerFromBackup::setStage(const String & new_stage, const String & message)
{ {
LOG_TRACE(log, "{}", toUpperFirst(new_stage)); LOG_TRACE(log, fmt::runtime(toUpperFirst(new_stage)));
current_stage = new_stage; current_stage = new_stage;
if (restore_coordination) if (restore_coordination)

View File

@ -0,0 +1,43 @@
#include <Backups/SettingsFieldOptionalUUID.h>
#include <Common/ErrorCodes.h>
#include <Core/SettingsFields.h>
#include <IO/ReadHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_PARSE_BACKUP_SETTINGS;
}
SettingFieldOptionalUUID::SettingFieldOptionalUUID(const Field & field)
{
if (field.getType() == Field::Types::Null)
{
value = std::nullopt;
return;
}
if (field.getType() == Field::Types::String)
{
const String & str = field.get<const String &>();
if (str.empty())
{
value = std::nullopt;
return;
}
UUID id;
if (tryParse(id, str))
{
value = id;
return;
}
}
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Cannot parse uuid from {}", field);
}
}

View File

@ -0,0 +1,18 @@
#pragma once
#include <optional>
#include <Core/SettingsFields.h>
namespace DB
{
struct SettingFieldOptionalUUID
{
std::optional<UUID> value;
explicit SettingFieldOptionalUUID(const std::optional<UUID> & value_) : value(value_) {}
explicit SettingFieldOptionalUUID(const Field & field);
explicit operator Field() const { return Field(value ? toString(*value) : ""); }
};
}

View File

@ -58,9 +58,8 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std::
auto table_status_it = status_response.table_states_by_id.find(*table_to_check); auto table_status_it = status_response.table_states_by_id.find(*table_to_check);
if (table_status_it == status_response.table_states_by_id.end()) if (table_status_it == status_response.table_states_by_id.end())
{ {
fail_message = fmt::format("There is no table {}.{} on server: {}", LOG_WARNING(LogToStr(fail_message, log), "There is no table {}.{} on server: {}",
backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription()); backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription());
LOG_WARNING(log, fmt::runtime(fail_message));
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
return; return;
} }

View File

@ -5,6 +5,7 @@
#include <Processors/Executors/PushingPipelineExecutor.h> #include <Processors/Executors/PushingPipelineExecutor.h>
#include <Processors/Executors/PushingAsyncPipelineExecutor.h> #include <Processors/Executors/PushingAsyncPipelineExecutor.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Core/Protocol.h> #include <Core/Protocol.h>

View File

@ -100,6 +100,7 @@
M(CacheDetachedFileSegments, "Number of existing detached cache file segments") \ M(CacheDetachedFileSegments, "Number of existing detached cache file segments") \
M(FilesystemCacheSize, "Filesystem cache size in bytes") \ M(FilesystemCacheSize, "Filesystem cache size in bytes") \
M(FilesystemCacheElements, "Filesystem cache elements (file segments)") \ M(FilesystemCacheElements, "Filesystem cache elements (file segments)") \
M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \
M(S3Requests, "S3 requests") \ M(S3Requests, "S3 requests") \
M(KeeperAliveConnections, "Number of alive connections") \ M(KeeperAliveConnections, "Number of alive connections") \
M(KeeperOutstandingRequets, "Number of outstanding requests") \ M(KeeperOutstandingRequets, "Number of outstanding requests") \

View File

@ -71,6 +71,13 @@ Exception::MessageMasked::MessageMasked(const std::string & msg_)
masker->wipeSensitiveData(msg); masker->wipeSensitiveData(msg);
} }
Exception::MessageMasked::MessageMasked(std::string && msg_)
: msg(std::move(msg_))
{
if (auto * masker = SensitiveDataMasker::getInstance())
masker->wipeSensitiveData(msg);
}
Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_) Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_)
: Poco::Exception(msg_masked.msg, code) : Poco::Exception(msg_masked.msg, code)
, remote(remote_) , remote(remote_)
@ -78,6 +85,13 @@ Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_)
handle_error_code(msg_masked.msg, code, remote, getStackFramePointers()); handle_error_code(msg_masked.msg, code, remote, getStackFramePointers());
} }
Exception::Exception(MessageMasked && msg_masked, int code, bool remote_)
: Poco::Exception(msg_masked.msg, code)
, remote(remote_)
{
handle_error_code(message(), code, remote, getStackFramePointers());
}
Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc) Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc)
: Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION) : Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION)
{ {
@ -172,10 +186,11 @@ static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string
{ {
try try
{ {
if (start_of_message.empty()) PreformattedMessage message = getCurrentExceptionMessageAndPattern(true);
LOG_ERROR(logger, "{}", getCurrentExceptionMessage(true)); if (!start_of_message.empty())
else message.message = fmt::format("{}: {}", start_of_message, message.message);
LOG_ERROR(logger, "{}: {}", start_of_message, getCurrentExceptionMessage(true));
LOG_ERROR(logger, message);
} }
catch (...) catch (...)
{ {
@ -323,8 +338,14 @@ std::string getExtraExceptionInfo(const std::exception & e)
} }
std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace /*= false*/, bool with_extra_info /*= true*/) std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace /*= false*/, bool with_extra_info /*= true*/)
{
return getCurrentExceptionMessageAndPattern(with_stacktrace, check_embedded_stacktrace, with_extra_info).message;
}
PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, bool check_embedded_stacktrace /*= false*/, bool with_extra_info /*= true*/)
{ {
WriteBufferFromOwnString stream; WriteBufferFromOwnString stream;
std::string_view message_format_string;
try try
{ {
@ -335,6 +356,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
stream << getExceptionMessage(e, with_stacktrace, check_embedded_stacktrace) stream << getExceptionMessage(e, with_stacktrace, check_embedded_stacktrace)
<< (with_extra_info ? getExtraExceptionInfo(e) : "") << (with_extra_info ? getExtraExceptionInfo(e) : "")
<< " (version " << VERSION_STRING << VERSION_OFFICIAL << ")"; << " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
message_format_string = e.tryGetMessageFormatString();
} }
catch (const Poco::Exception & e) catch (const Poco::Exception & e)
{ {
@ -380,7 +402,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
catch (...) {} catch (...) {}
} }
return stream.str(); return PreformattedMessage{stream.str(), message_format_string};
} }
@ -433,14 +455,6 @@ int getExceptionErrorCode(std::exception_ptr e)
} }
void rethrowFirstException(const Exceptions & exceptions)
{
for (const auto & exception : exceptions)
if (exception)
std::rethrow_exception(exception);
}
void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message) void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message)
{ {
try try
@ -466,6 +480,11 @@ void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::str
} }
std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace) std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace)
{
return getExceptionMessageAndPattern(e, with_stacktrace, check_embedded_stacktrace).message;
}
PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace)
{ {
WriteBufferFromOwnString stream; WriteBufferFromOwnString stream;
@ -497,7 +516,7 @@ std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool
} }
catch (...) {} catch (...) {}
return stream.str(); return PreformattedMessage{stream.str(), e.tryGetMessageFormatString()};
} }
std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace) std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace)

View File

@ -9,12 +9,32 @@
#include <base/defines.h> #include <base/defines.h>
#include <Common/StackTrace.h> #include <Common/StackTrace.h>
#include <Common/LoggingFormatStringHelpers.h>
#include <fmt/format.h> #include <fmt/format.h>
namespace Poco { class Logger; } namespace Poco { class Logger; }
/// Extract format string from a string literal and constructs consteval fmt::format_string
template <typename... Args>
struct FormatStringHelperImpl
{
std::string_view message_format_string;
fmt::format_string<Args...> fmt_str;
template<typename T>
consteval FormatStringHelperImpl(T && str) : message_format_string(tryGetStaticFormatString(str)), fmt_str(std::forward<T>(str)) {}
template<typename T>
FormatStringHelperImpl(fmt::basic_runtime<T> && str) : message_format_string(), fmt_str(std::forward<fmt::basic_runtime<T>>(str)) {}
PreformattedMessage format(Args && ...args) const
{
return PreformattedMessage{fmt::format(fmt_str, std::forward<Args...>(args)...), message_format_string};
}
};
template <typename... Args>
using FormatStringHelper = FormatStringHelperImpl<std::type_identity_t<Args>...>;
namespace DB namespace DB
{ {
@ -33,22 +53,37 @@ public:
{ {
std::string msg; std::string msg;
MessageMasked(const std::string & msg_); MessageMasked(const std::string & msg_);
MessageMasked(std::string && msg_);
}; };
Exception(const MessageMasked & msg_masked, int code, bool remote_); Exception(const MessageMasked & msg_masked, int code, bool remote_);
Exception(MessageMasked && msg_masked, int code, bool remote_);
// delegating constructor to mask sensitive information from the message // delegating constructor to mask sensitive information from the message
Exception(const std::string & msg, int code, bool remote_ = false): Exception(MessageMasked(msg), code, remote_) Exception(const std::string & msg, int code, bool remote_ = false): Exception(MessageMasked(msg), code, remote_) {}
{} Exception(std::string && msg, int code, bool remote_ = false): Exception(MessageMasked(std::move(msg)), code, remote_) {}
Exception(PreformattedMessage && msg, int code): Exception(std::move(msg.message), code)
{
message_format_string = msg.format_string;
}
Exception(int code, const std::string & message) template<typename T, typename = std::enable_if_t<std::is_convertible_v<T, String>>>
Exception(int code, T && message)
: Exception(message, code) : Exception(message, code)
{} {
message_format_string = tryGetStaticFormatString(message);
}
template<> Exception(int code, const String & message) : Exception(message, code) {}
template<> Exception(int code, String & message) : Exception(message, code) {}
template<> Exception(int code, String && message) : Exception(std::move(message), code) {}
// Format message with fmt::format, like the logging functions. // Format message with fmt::format, like the logging functions.
template <typename... Args> template <typename... Args>
Exception(int code, fmt::format_string<Args...> fmt, Args &&... args) : Exception(fmt::format(fmt, std::forward<Args>(args)...), code) Exception(int code, FormatStringHelper<Args...> fmt, Args &&... args)
: Exception(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code)
{ {
message_format_string = fmt.message_format_string;
} }
struct CreateFromPocoTag {}; struct CreateFromPocoTag {};
@ -87,6 +122,8 @@ public:
/// Used for system.errors /// Used for system.errors
FramePointers getStackFramePointers() const; FramePointers getStackFramePointers() const;
std::string_view tryGetMessageFormatString() const { return message_format_string; }
private: private:
#ifndef STD_EXCEPTION_HAS_STACK_TRACE #ifndef STD_EXCEPTION_HAS_STACK_TRACE
StackTrace trace; StackTrace trace;
@ -94,6 +131,9 @@ private:
bool remote = false; bool remote = false;
const char * className() const noexcept override { return "DB::Exception"; } const char * className() const noexcept override { return "DB::Exception"; }
protected:
std::string_view message_format_string;
}; };
@ -131,14 +171,15 @@ public:
ParsingException(); ParsingException();
ParsingException(const std::string & msg, int code); ParsingException(const std::string & msg, int code);
ParsingException(int code, const std::string & message); ParsingException(int code, const std::string & message);
ParsingException(int code, std::string && message) : Exception(message, code) {}
// Format message with fmt::format, like the logging functions. // Format message with fmt::format, like the logging functions.
template <typename... Args> template <typename... Args>
ParsingException(int code, fmt::format_string<Args...> fmt, Args &&... args) : Exception(code, fmt, std::forward<Args>(args)...) ParsingException(int code, FormatStringHelper<Args...> fmt, Args &&... args) : Exception(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code)
{ {
message_format_string = fmt.message_format_string;
} }
std::string displayText() const override; std::string displayText() const override;
ssize_t getLineNumber() const { return line_number; } ssize_t getLineNumber() const { return line_number; }
@ -184,6 +225,8 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_
*/ */
std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace = false, std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace = false,
bool with_extra_info = true); bool with_extra_info = true);
PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, bool check_embedded_stacktrace = false,
bool with_extra_info = true);
/// Returns error code from ErrorCodes /// Returns error code from ErrorCodes
int getCurrentExceptionCode(); int getCurrentExceptionCode();
@ -219,12 +262,10 @@ void tryLogException(std::exception_ptr e, const char * log_name, const std::str
void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::string & start_of_message = ""); void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::string & start_of_message = "");
std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false); std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false);
PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false);
std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace); std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace);
void rethrowFirstException(const Exceptions & exceptions);
template <typename T> template <typename T>
requires std::is_pointer_v<T> requires std::is_pointer_v<T>
T exception_cast(std::exception_ptr e) T exception_cast(std::exception_ptr e)

480
src/Common/FST.cpp Normal file
View File

@ -0,0 +1,480 @@
#include "FST.h"
#include <algorithm>
#include <cassert>
#include <iostream>
#include <memory>
#include <vector>
#include <Common/Exception.h>
#include <city.h>
/// "paper" in the comments in this file refers to:
/// [Direct Construction of Minimal Acyclic Subsequential Transduers] by Stoyan Mihov and Denis Maurel, University of Tours, France
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
};
namespace FST
{
UInt64 Arc::serialize(WriteBuffer& write_buffer) const
{
UInt64 written_bytes = 0;
bool has_output = output != 0;
/// First UInt64 is target_index << 1 + has_output
assert(target != nullptr);
UInt64 first = ((target->state_index) << 1) + has_output;
writeVarUInt(first, write_buffer);
written_bytes += getLengthOfVarUInt(first);
/// Second UInt64 is output (optional based on whether has_output is not zero)
if (has_output)
{
writeVarUInt(output, write_buffer);
written_bytes += getLengthOfVarUInt(output);
}
return written_bytes;
}
bool operator==(const Arc & arc1, const Arc & arc2)
{
assert(arc1.target != nullptr && arc2.target != nullptr);
return (arc1.output == arc2.output && arc1.target->id == arc2.target->id);
}
void LabelsAsBitmap::addLabel(char label)
{
UInt8 index = label;
UInt256 bit_label = 1;
bit_label <<= index;
data |= bit_label;
}
UInt64 LabelsAsBitmap::getIndex(char label) const
{
UInt64 bit_count = 0;
UInt8 index = label;
int which_int64 = 0;
while (true)
{
if (index < 64)
{
UInt64 mask = index == 63 ? (-1) : (1ULL << (index + 1)) - 1;
bit_count += std::popcount(mask & data.items[which_int64]);
break;
}
index -= 64;
bit_count += std::popcount(data.items[which_int64]);
which_int64++;
}
return bit_count;
}
UInt64 LabelsAsBitmap::serialize(WriteBuffer& write_buffer)
{
writeVarUInt(data.items[0], write_buffer);
writeVarUInt(data.items[1], write_buffer);
writeVarUInt(data.items[2], write_buffer);
writeVarUInt(data.items[3], write_buffer);
return getLengthOfVarUInt(data.items[0])
+ getLengthOfVarUInt(data.items[1])
+ getLengthOfVarUInt(data.items[2])
+ getLengthOfVarUInt(data.items[3]);
}
bool LabelsAsBitmap::hasLabel(char label) const
{
UInt8 index = label;
UInt256 bit_label = 1;
bit_label <<= index;
return ((data & bit_label) != 0);
}
Arc* State::getArc(char label) const
{
auto it = arcs.find(label);
if (it == arcs.cend())
return nullptr;
return const_cast<Arc *>(&it->second);
}
void State::addArc(char label, Output output, StatePtr target)
{
arcs[label] = Arc(output, target);
}
void State::clear()
{
id = 0;
state_index = 0;
flag = 0;
arcs.clear();
}
UInt64 State::hash() const
{
std::vector<char> values;
values.reserve(arcs.size() * (sizeof(Output) + sizeof(UInt64) + 1));
for (const auto & [label, arc] : arcs)
{
values.push_back(label);
const auto * ptr = reinterpret_cast<const char*>(&arc.output);
std::copy(ptr, ptr + sizeof(Output), std::back_inserter(values));
ptr = reinterpret_cast<const char*>(&arc.target->id);
std::copy(ptr, ptr + sizeof(UInt64), std::back_inserter(values));
}
return CityHash_v1_0_2::CityHash64(values.data(), values.size());
}
bool operator== (const State & state1, const State & state2)
{
if (state1.arcs.size() != state2.arcs.size())
return false;
for (const auto & [label, arc] : state1.arcs)
{
const auto it = state2.arcs.find(label);
if (it == state2.arcs.cend())
return false;
if (it->second != arc)
return false;
}
return true;
}
UInt64 State::serialize(WriteBuffer& write_buffer)
{
UInt64 written_bytes = 0;
/// Serialize flag
write_buffer.write(flag);
written_bytes += 1;
if (getEncodingMethod() == EncodingMethod::Sequential)
{
/// Serialize all labels
std::vector<char> labels;
labels.reserve(arcs.size());
for (auto& [label, state] : arcs)
{
labels.push_back(label);
}
UInt8 label_size = labels.size();
write_buffer.write(label_size);
written_bytes += 1;
write_buffer.write(labels.data(), labels.size());
written_bytes += labels.size();
/// Serialize all arcs
for (char label : labels)
{
Arc* arc = getArc(label);
assert(arc != nullptr);
written_bytes += arc->serialize(write_buffer);
}
}
else
{
/// Serialize bitmap
LabelsAsBitmap bmp;
for (auto & [label, state] : arcs)
{
bmp.addLabel(label);
}
written_bytes += bmp.serialize(write_buffer);
/// Serialize all arcs
for (auto & [label, state] : arcs)
{
Arc* arc = getArc(label);
assert(arc != nullptr);
written_bytes += arc->serialize(write_buffer);
}
}
return written_bytes;
}
FSTBuilder::FSTBuilder(WriteBuffer& write_buffer_) : write_buffer(write_buffer_)
{
for (auto & temp_state : temp_states)
{
temp_state = std::make_shared<State>();
}
}
/// See FindMinimized in the paper pseudo code l11-l21.
StatePtr FSTBuilder::findMinimized(const State & state, bool & found)
{
found = false;
auto hash = state.hash();
/// MEMBER: in the paper pseudo code l15
auto it = minimized_states.find(hash);
if (it != minimized_states.cend() && *it->second == state)
{
found = true;
return it->second;
}
/// COPY_STATE: in the paper pseudo code l17
StatePtr p = std::make_shared<State>(state);
/// INSERT: in the paper pseudo code l18
minimized_states[hash] = p;
return p;
}
/// See the paper pseudo code l33-34.
size_t FSTBuilder::getCommonPrefixLength(const String & word1, const String & word2)
{
size_t i = 0;
while (i < word1.size() && i < word2.size() && word1[i] == word2[i])
i++;
return i;
}
/// See the paper pseudo code l33-39 and l70-72(when down_to is 0).
void FSTBuilder::minimizePreviousWordSuffix(Int64 down_to)
{
for (Int64 i = static_cast<Int64>(previous_word.size()); i >= down_to; --i)
{
bool found = false;
auto minimized_state = findMinimized(*temp_states[i], found);
if (i != 0)
{
Output output = 0;
Arc* arc = temp_states[i - 1]->getArc(previous_word[i - 1]);
if (arc)
output = arc->output;
/// SET_TRANSITION
temp_states[i - 1]->addArc(previous_word[i - 1], output, minimized_state);
}
if (minimized_state->id == 0)
minimized_state->id = next_id++;
if (i > 0 && temp_states[i - 1]->id == 0)
temp_states[i - 1]->id = next_id++;
if (!found)
{
minimized_state->state_index = previous_state_index;
previous_written_bytes = minimized_state->serialize(write_buffer);
previous_state_index += previous_written_bytes;
}
}
}
void FSTBuilder::add(const std::string & current_word, Output current_output)
{
/// We assume word size is no greater than MAX_TERM_LENGTH(256).
/// FSTs without word size limitation would be inefficient and easy to cause memory bloat
/// Note that when using "split" tokenizer, if a granule has tokens which are longer than
/// MAX_TERM_LENGTH, the granule cannot be dropped and will be fully-scanned. It doesn't affect "ngram" tokenizers.
/// Another limitation is that if the query string has tokens which exceed this length
/// it will fallback to default searching when using "split" tokenizers.
auto current_word_len = current_word.size();
if (current_word_len > MAX_TERM_LENGTH)
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Too long term ({}) passed to FST builder.", current_word_len);
size_t prefix_length_plus1 = getCommonPrefixLength(current_word, previous_word) + 1;
minimizePreviousWordSuffix(prefix_length_plus1);
/// Initialize the tail state, see paper pseudo code l39-43
for (size_t i = prefix_length_plus1; i <= current_word.size(); ++i)
{
/// CLEAR_STATE: l41
temp_states[i]->clear();
/// SET_TRANSITION: l42
temp_states[i - 1]->addArc(current_word[i - 1], 0, temp_states[i]);
}
/// We assume the current word is different with previous word
/// See paper pseudo code l44-47
temp_states[current_word_len]->setFinal(true);
/// Adjust outputs on the arcs
/// See paper pseudo code l48-63
for (size_t i = 1; i <= prefix_length_plus1 - 1; ++i)
{
Arc * arc_ptr = temp_states[i - 1]->getArc(current_word[i - 1]);
assert(arc_ptr != nullptr);
Output common_prefix = std::min(arc_ptr->output, current_output);
Output word_suffix = arc_ptr->output - common_prefix;
arc_ptr->output = common_prefix;
/// For each arc, adjust its output
if (word_suffix != 0)
{
for (auto & [label, arc] : temp_states[i]->arcs)
{
arc.output += word_suffix;
}
}
/// Reduce current_output
current_output -= common_prefix;
}
/// Set last temp state's output
/// paper pseudo code l66-67 (assuming CurrentWord != PreviousWorld)
Arc * arc = temp_states[prefix_length_plus1 - 1]->getArc(current_word[prefix_length_plus1 - 1]);
assert(arc != nullptr);
arc->output = current_output;
previous_word = current_word;
}
UInt64 FSTBuilder::build()
{
minimizePreviousWordSuffix(0);
/// Save initial state index
previous_state_index -= previous_written_bytes;
UInt8 length = getLengthOfVarUInt(previous_state_index);
writeVarUInt(previous_state_index, write_buffer);
write_buffer.write(length);
return previous_state_index + previous_written_bytes + length + 1;
}
FiniteStateTransducer::FiniteStateTransducer(std::vector<UInt8> data_) : data(std::move(data_))
{
}
void FiniteStateTransducer::clear()
{
data.clear();
}
std::pair<UInt64, bool> FiniteStateTransducer::getOutput(const String & term)
{
std::pair<UInt64, bool> result{ 0, false };
/// Read index of initial state
ReadBufferFromMemory read_buffer(data.data(), data.size());
read_buffer.seek(data.size()-1, SEEK_SET);
UInt8 length{ 0 };
read_buffer.readStrict(reinterpret_cast<char&>(length));
/// FST contains no terms
if (length == 0)
return { 0, false };
read_buffer.seek(data.size() - 1 - length, SEEK_SET);
UInt64 state_index{ 0 };
readVarUInt(state_index, read_buffer);
for (size_t i = 0; i <= term.size(); ++i)
{
UInt64 arc_output{ 0 };
/// Read flag
State temp_state;
read_buffer.seek(state_index, SEEK_SET);
temp_state.readFlag(read_buffer);
if (i == term.size())
{
result.second = temp_state.isFinal();
break;
}
UInt8 label = term[i];
if (temp_state.getEncodingMethod() == State::EncodingMethod::Sequential)
{
/// Read number of labels
UInt8 label_num{ 0 };
read_buffer.readStrict(reinterpret_cast<char&>(label_num));
if (label_num == 0)
return { 0, false };
auto labels_position = read_buffer.getPosition();
/// Find the index of the label from "labels" bytes
auto begin_it{ data.begin() + labels_position };
auto end_it{ data.begin() + labels_position + label_num };
auto pos = std::find(begin_it, end_it, label);
if (pos == end_it)
return { 0, false };
/// Read the arc for the label
UInt64 arc_index = (pos - begin_it);
auto arcs_start_postion = labels_position + label_num;
read_buffer.seek(arcs_start_postion, SEEK_SET);
for (size_t j = 0; j <= arc_index; j++)
{
state_index = 0;
arc_output = 0;
readVarUInt(state_index, read_buffer);
if (state_index & 0x1) // output is followed
{
readVarUInt(arc_output, read_buffer);
}
state_index >>= 1;
}
}
else
{
LabelsAsBitmap bmp;
readVarUInt(bmp.data.items[0], read_buffer);
readVarUInt(bmp.data.items[1], read_buffer);
readVarUInt(bmp.data.items[2], read_buffer);
readVarUInt(bmp.data.items[3], read_buffer);
if (!bmp.hasLabel(label))
return { 0, false };
/// Read the arc for the label
size_t arc_index = bmp.getIndex(label);
for (size_t j = 0; j < arc_index; j++)
{
state_index = 0;
arc_output = 0;
readVarUInt(state_index, read_buffer);
if (state_index & 0x1) // output is followed
{
readVarUInt(arc_output, read_buffer);
}
state_index >>= 1;
}
}
/// Accumulate the output value
result.first += arc_output;
}
return result;
}
}
}

182
src/Common/FST.h Normal file
View File

@ -0,0 +1,182 @@
#pragma once
#include <array>
#include <map>
#include <memory>
#include <string>
#include <unordered_map>
#include <unordered_set>
#include <vector>
#include <Core/Types.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBuffer.h>
#include <base/types.h>
namespace DB
{
/// Finite State Transducer is an efficient way to represent term dictionary.
/// It can be viewed as a map of <term, output> where output is an integer.
/// Detailed explanation can be found in the following paper
/// [Direct Construction of Minimal Acyclic Subsequential Transduers] by Stoyan Mihov and Denis Maurel, University of Tours, France
namespace FST
{
using Output = UInt64;
class State;
using StatePtr = std::shared_ptr<State>;
/// Arc represents a transition from one state to another
/// It includes the target state to which the arc points and the arc's output.
struct Arc
{
Arc() = default;
explicit Arc(Output output_, const StatePtr & target_) : output{output_}, target{target_} { }
/// 0 means the arc has no output
Output output = 0;
StatePtr target;
UInt64 serialize(WriteBuffer & write_buffer) const;
};
bool operator==(const Arc & arc1, const Arc & arc2);
/// LabelsAsBitmap implements a 256-bit bitmap for all labels of a state. Each bit represents
/// a label's presence and the index value of the bit represents the corresponding label
class LabelsAsBitmap
{
public:
void addLabel(char label);
bool hasLabel(char label) const;
/// computes the rank
UInt64 getIndex(char label) const;
UInt64 serialize(WriteBuffer& write_buffer);
private:
friend class State;
friend class FiniteStateTransducer;
/// data holds a 256-bit bitmap for all labels of a state. Its 256 bits correspond to 256
/// possible label values.
UInt256 data{ 0 };
};
/// State implements the State in Finite State Transducer
/// Each state contains all its arcs and a flag indicating if it is final state
class State
{
public:
static constexpr size_t MAX_ARCS_IN_SEQUENTIAL_METHOD = 32;
enum class EncodingMethod
{
/// Serialize arcs sequentially
Sequential = 0,
/// Serialize arcs by using bitmap
/// Note this is NOT enabled for now since it is experimental
Bitmap,
};
State() = default;
State(const State & state) = default;
UInt64 hash() const;
Arc * getArc(char label) const;
void addArc(char label, Output output, StatePtr target);
void clear();
UInt64 serialize(WriteBuffer & write_buffer);
bool isFinal() const
{
return flag_values.is_final == 1;
}
void setFinal(bool value)
{
flag_values.is_final = value;
}
EncodingMethod getEncodingMethod() const
{
return flag_values.encoding_method;
}
void readFlag(ReadBuffer & read_buffer)
{
read_buffer.readStrict(reinterpret_cast<char&>(flag));
}
/// Transient ID of the state which is used for building FST. It won't be serialized
UInt64 id = 0;
/// State index which indicates location of state in FST
UInt64 state_index = 0;
/// Arcs which are started from state, the 'char' is the label on the arc
std::unordered_map<char, Arc> arcs;
private:
struct FlagValues
{
unsigned int is_final : 1;
EncodingMethod encoding_method : 3;
};
union
{
FlagValues flag_values;
uint8_t flag = 0;
};
};
bool operator==(const State & state1, const State & state2);
inline constexpr size_t MAX_TERM_LENGTH = 256;
/// FSTBuilder is used to build Finite State Transducer by adding words incrementally.
/// Note that all the words have to be added in sorted order in order to achieve minimized result.
/// In the end, the caller should call build() to serialize minimized FST to WriteBuffer
class FSTBuilder
{
public:
explicit FSTBuilder(WriteBuffer & write_buffer_);
void add(const std::string & word, Output output);
UInt64 build();
private:
StatePtr findMinimized(const State & s, bool & found);
void minimizePreviousWordSuffix(Int64 down_to);
static size_t getCommonPrefixLength(const String & word1, const String & word2);
std::array<StatePtr, MAX_TERM_LENGTH + 1> temp_states;
String previous_word;
StatePtr initial_state;
/// map of (state_hash, StatePtr)
std::unordered_map<UInt64, StatePtr> minimized_states;
/// Next available ID of state
UInt64 next_id = 1;
WriteBuffer & write_buffer;
UInt64 previous_written_bytes = 0;
UInt64 previous_state_index = 0;
};
//FiniteStateTransducer is constructed by using minimized FST blob(which is loaded from index storage)
// It is used to retrieve output by given term
class FiniteStateTransducer
{
public:
FiniteStateTransducer() = default;
explicit FiniteStateTransducer(std::vector<UInt8> data_);
std::pair<UInt64, bool> getOutput(const String & term);
void clear();
std::vector<UInt8> & getData() { return data; }
private:
std::vector<UInt8> data;
};
}
}

View File

@ -0,0 +1,55 @@
#pragma once
#include <base/defines.h>
#include <fmt/format.h>
/// Saves a format string for already formatted message
struct PreformattedMessage
{
String message;
std::string_view format_string;
operator const String & () const { return message; }
operator String () && { return std::move(message); }
operator fmt::format_string<> () const { UNREACHABLE(); }
};
template<typename T> struct is_fmt_runtime : std::false_type {};
template<typename T> struct is_fmt_runtime<fmt::basic_runtime<T>> : std::true_type {};
template <typename T> constexpr std::string_view tryGetStaticFormatString(T && x)
{
/// Failure of this asserting indicates that something went wrong during type deduction.
/// For example, a string literal was implicitly converted to std::string. It should not happen.
static_assert(!std::is_same_v<std::string, std::decay_t<T>>);
if constexpr (is_fmt_runtime<std::decay_t<T>>::value)
{
/// It definitely was fmt::runtime(something).
/// We are not sure about a lifetime of the string, so return empty view.
/// Also it can be arbitrary string, not a formatting pattern.
/// So returning empty pattern will not pollute the set of patterns.
return std::string_view();
}
else
{
if constexpr (std::is_same_v<PreformattedMessage, std::decay_t<T>>)
{
return x.format_string;
}
else
{
/// Most likely it was a string literal.
/// Unfortunately, there's no good way to check if something is a string literal.
/// But fmtlib requires a format string to be compile-time constant unless fmt::runtime is used.
static_assert(std::is_nothrow_convertible<T, const char * const>::value);
static_assert(!std::is_pointer<T>::value);
return std::string_view(x);
}
}
}
template <typename... Ts> constexpr size_t numArgs(Ts &&...) { return sizeof...(Ts); }
template <typename T, typename... Ts> constexpr auto firstArg(T && x, Ts &&...) { return std::forward<T>(x); }
/// For implicit conversion of fmt::basic_runtime<> to char* for std::string ctor
template <typename T, typename... Ts> constexpr auto firstArg(fmt::basic_runtime<T> && data, Ts &&...) { return data.str.data(); }

View File

@ -10,6 +10,7 @@
M(InsertQuery, "Same as Query, but only for INSERT queries.") \ M(InsertQuery, "Same as Query, but only for INSERT queries.") \
M(AsyncInsertQuery, "Same as InsertQuery, but only for asynchronous INSERT queries.") \ M(AsyncInsertQuery, "Same as InsertQuery, but only for asynchronous INSERT queries.") \
M(AsyncInsertBytes, "Data size in bytes of asynchronous INSERT queries.") \ M(AsyncInsertBytes, "Data size in bytes of asynchronous INSERT queries.") \
M(AsyncInsertCacheHits, "Number of times a duplicate hash id has been found in asynchronous INSERT hash id cache.") \
M(FailedQuery, "Number of failed queries.") \ M(FailedQuery, "Number of failed queries.") \
M(FailedSelectQuery, "Same as FailedQuery, but only for SELECT queries.") \ M(FailedSelectQuery, "Same as FailedQuery, but only for SELECT queries.") \
M(FailedInsertQuery, "Same as FailedQuery, but only for INSERT queries.") \ M(FailedInsertQuery, "Same as FailedQuery, but only for INSERT queries.") \
@ -63,7 +64,20 @@
M(NetworkSendElapsedMicroseconds, "Total time spent waiting for data to send to network or sending data to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries..") \ M(NetworkSendElapsedMicroseconds, "Total time spent waiting for data to send to network or sending data to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries..") \
M(NetworkReceiveBytes, "Total number of bytes received from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \ M(NetworkReceiveBytes, "Total number of bytes received from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
M(NetworkSendBytes, "Total number of bytes send to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \ M(NetworkSendBytes, "Total number of bytes send to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
M(ThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_network_bandwidth' and other throttling settings.") \ \
M(DiskS3GetRequestThrottlerCount, "Number of DiskS3 GET and SELECT requests passed through throttler.") \
M(DiskS3GetRequestThrottlerSleepMicroseconds, "Total time a query was sleeping to conform DiskS3 GET and SELECT request throttling.") \
M(DiskS3PutRequestThrottlerCount, "Number of DiskS3 PUT, COPY, POST and LIST requests passed through throttler.") \
M(DiskS3PutRequestThrottlerSleepMicroseconds, "Total time a query was sleeping to conform DiskS3 PUT, COPY, POST and LIST request throttling.") \
M(S3GetRequestThrottlerCount, "Number of S3 GET and SELECT requests passed through throttler.") \
M(S3GetRequestThrottlerSleepMicroseconds, "Total time a query was sleeping to conform S3 GET and SELECT request throttling.") \
M(S3PutRequestThrottlerCount, "Number of S3 PUT, COPY, POST and LIST requests passed through throttler.") \
M(S3PutRequestThrottlerSleepMicroseconds, "Total time a query was sleeping to conform S3 PUT, COPY, POST and LIST request throttling.") \
M(RemoteReadThrottlerBytes, "Bytes passed through 'max_remote_read_network_bandwidth_for_server' throttler.") \
M(RemoteReadThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_remote_read_network_bandwidth_for_server' throttling.") \
M(RemoteWriteThrottlerBytes, "Bytes passed through 'max_remote_write_network_bandwidth_for_server' throttler.") \
M(RemoteWriteThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_remote_write_network_bandwidth_for_server' throttling.") \
M(ThrottlerSleepMicroseconds, "Total time a query was sleeping to conform all throttling settings.") \
\ \
M(QueryMaskingRulesMatch, "Number of times query masking rules was successfully matched.") \ M(QueryMaskingRulesMatch, "Number of times query masking rules was successfully matched.") \
\ \
@ -309,6 +323,8 @@ The server successfully detected this situation and will download merged part fr
M(S3CopyObject, "Number of S3 API CopyObject calls.") \ M(S3CopyObject, "Number of S3 API CopyObject calls.") \
M(S3ListObjects, "Number of S3 API ListObjects calls.") \ M(S3ListObjects, "Number of S3 API ListObjects calls.") \
M(S3HeadObject, "Number of S3 API HeadObject calls.") \ M(S3HeadObject, "Number of S3 API HeadObject calls.") \
M(S3GetObjectAttributes, "Number of S3 API GetObjectAttributes calls.") \
M(S3GetObjectMetadata, "Number of S3 API GetObject calls for getting metadata.") \
M(S3CreateMultipartUpload, "Number of S3 API CreateMultipartUpload calls.") \ M(S3CreateMultipartUpload, "Number of S3 API CreateMultipartUpload calls.") \
M(S3UploadPartCopy, "Number of S3 API UploadPartCopy calls.") \ M(S3UploadPartCopy, "Number of S3 API UploadPartCopy calls.") \
M(S3UploadPart, "Number of S3 API UploadPart calls.") \ M(S3UploadPart, "Number of S3 API UploadPart calls.") \
@ -321,6 +337,8 @@ The server successfully detected this situation and will download merged part fr
M(DiskS3CopyObject, "Number of DiskS3 API CopyObject calls.") \ M(DiskS3CopyObject, "Number of DiskS3 API CopyObject calls.") \
M(DiskS3ListObjects, "Number of DiskS3 API ListObjects calls.") \ M(DiskS3ListObjects, "Number of DiskS3 API ListObjects calls.") \
M(DiskS3HeadObject, "Number of DiskS3 API HeadObject calls.") \ M(DiskS3HeadObject, "Number of DiskS3 API HeadObject calls.") \
M(DiskS3GetObjectAttributes, "Number of DiskS3 API GetObjectAttributes calls.") \
M(DiskS3GetObjectMetadata, "Number of DiskS3 API GetObject calls for getting metadata.") \
M(DiskS3CreateMultipartUpload, "Number of DiskS3 API CreateMultipartUpload calls.") \ M(DiskS3CreateMultipartUpload, "Number of DiskS3 API CreateMultipartUpload calls.") \
M(DiskS3UploadPartCopy, "Number of DiskS3 API UploadPartCopy calls.") \ M(DiskS3UploadPartCopy, "Number of DiskS3 API UploadPartCopy calls.") \
M(DiskS3UploadPart, "Number of DiskS3 API UploadPart calls.") \ M(DiskS3UploadPart, "Number of DiskS3 API UploadPart calls.") \

View File

@ -212,7 +212,7 @@ bool checkPermissionsImpl()
{ {
/// This error happens all the time when running inside Docker - consider it ok, /// This error happens all the time when running inside Docker - consider it ok,
/// don't create noise with this error. /// don't create noise with this error.
LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "{}", getCurrentExceptionMessage(false)); LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false));
} }
else else
{ {

View File

@ -1,5 +1,6 @@
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/ThreadProfileEvents.h> #include <Common/ThreadProfileEvents.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/QueryProfiler.h> #include <Common/QueryProfiler.h>
#include <Common/ThreadStatus.h> #include <Common/ThreadStatus.h>
#include <base/errnoToString.h> #include <base/errnoToString.h>

View File

@ -6,7 +6,6 @@
#include <Common/MemoryTracker.h> #include <Common/MemoryTracker.h>
#include <Common/ProfileEvents.h> #include <Common/ProfileEvents.h>
#include <base/StringRef.h> #include <base/StringRef.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <boost/noncopyable.hpp> #include <boost/noncopyable.hpp>
@ -24,6 +23,9 @@ namespace Poco
} }
template <class T>
class ConcurrentBoundedQueue;
namespace DB namespace DB
{ {

View File

@ -38,7 +38,7 @@ Throttler::Throttler(size_t max_speed_, size_t limit_, const char * limit_exceed
, parent(parent_) , parent(parent_)
{} {}
void Throttler::add(size_t amount) UInt64 Throttler::add(size_t amount)
{ {
// Values obtained under lock to be checked after release // Values obtained under lock to be checked after release
size_t count_value; size_t count_value;
@ -61,9 +61,10 @@ void Throttler::add(size_t amount)
throw Exception(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED); throw Exception(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED);
/// Wait unless there is positive amount of tokens - throttling /// Wait unless there is positive amount of tokens - throttling
Int64 sleep_time = 0;
if (max_speed && tokens_value < 0) if (max_speed && tokens_value < 0)
{ {
int64_t sleep_time = static_cast<int64_t>(-tokens_value / max_speed * NS); sleep_time = static_cast<Int64>(-tokens_value / max_speed * NS);
accumulated_sleep += sleep_time; accumulated_sleep += sleep_time;
sleepForNanoseconds(sleep_time); sleepForNanoseconds(sleep_time);
accumulated_sleep -= sleep_time; accumulated_sleep -= sleep_time;
@ -71,7 +72,9 @@ void Throttler::add(size_t amount)
} }
if (parent) if (parent)
parent->add(amount); sleep_time += parent->add(amount);
return static_cast<UInt64>(sleep_time);
} }
void Throttler::reset() void Throttler::reset()

View File

@ -1,10 +1,12 @@
#pragma once #pragma once
#include <Common/Throttler_fwd.h> #include <Common/Throttler_fwd.h>
#include <Common/ProfileEvents.h>
#include <mutex> #include <mutex>
#include <memory> #include <memory>
#include <base/sleep.h> #include <base/sleep.h>
#include <base/types.h>
#include <atomic> #include <atomic>
namespace DB namespace DB
@ -32,7 +34,16 @@ public:
const std::shared_ptr<Throttler> & parent_ = nullptr); const std::shared_ptr<Throttler> & parent_ = nullptr);
/// Use `amount` tokens, sleeps if required or throws exception on limit overflow. /// Use `amount` tokens, sleeps if required or throws exception on limit overflow.
void add(size_t amount); /// Returns duration of sleep in microseconds (to distinguish sleeping on different kinds of throttlers for metrics)
UInt64 add(size_t amount);
UInt64 add(size_t amount, ProfileEvents::Event event_amount, ProfileEvents::Event event_sleep_us)
{
UInt64 sleep_us = add(amount);
ProfileEvents::increment(event_amount, amount);
ProfileEvents::increment(event_sleep_us, sleep_us);
return sleep_us;
}
/// Not thread safe /// Not thread safe
void setParent(const std::shared_ptr<Throttler> & parent_) void setParent(const std::shared_ptr<Throttler> & parent_)
@ -50,12 +61,12 @@ private:
size_t count{0}; size_t count{0};
const size_t max_speed{0}; /// in tokens per second. const size_t max_speed{0}; /// in tokens per second.
const size_t max_burst{0}; /// in tokens. const size_t max_burst{0}; /// in tokens.
const uint64_t limit{0}; /// 0 - not limited. const UInt64 limit{0}; /// 0 - not limited.
const char * limit_exceeded_exception_message = nullptr; const char * limit_exceeded_exception_message = nullptr;
std::mutex mutex; std::mutex mutex;
std::atomic<uint64_t> accumulated_sleep{0}; // Accumulated sleep time over all waiting threads std::atomic<UInt64> accumulated_sleep{0}; // Accumulated sleep time over all waiting threads
double tokens{0}; /// Amount of tokens available in token bucket. Updated in `add` method. double tokens{0}; /// Amount of tokens available in token bucket. Updated in `add` method.
uint64_t prev_ns{0}; /// Previous `add` call time (in nanoseconds). UInt64 prev_ns{0}; /// Previous `add` call time (in nanoseconds).
/// Used to implement a hierarchy of throttlers /// Used to implement a hierarchy of throttlers
std::shared_ptr<Throttler> parent; std::shared_ptr<Throttler> parent;

View File

@ -6,19 +6,43 @@
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <Poco/Message.h> #include <Poco/Message.h>
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
#include <Common/LoggingFormatStringHelpers.h>
namespace Poco { class Logger; }
/// This wrapper is useful to save formatted message into a String before sending it to a logger
class LogToStrImpl
{
String & out_str;
Poco::Logger * logger;
bool propagate_to_actual_log = true;
public:
LogToStrImpl(String & out_str_, Poco::Logger * logger_) : out_str(out_str_) , logger(logger_) {}
LogToStrImpl & operator -> () { return *this; }
bool is(Poco::Message::Priority priority) { propagate_to_actual_log &= logger->is(priority); return true; }
LogToStrImpl * getChannel() {return this; }
const String & name() const { return logger->name(); }
void log(const Poco::Message & message)
{
out_str = message.getText();
if (!propagate_to_actual_log)
return;
if (auto * channel = logger->getChannel())
channel->log(message);
}
};
#define LogToStr(x, y) std::make_unique<LogToStrImpl>(x, y)
namespace namespace
{ {
template <typename... Ts> constexpr size_t numArgs(Ts &&...) { return sizeof...(Ts); }
template <typename T, typename... Ts> constexpr auto firstArg(T && x, Ts &&...) { return std::forward<T>(x); }
/// For implicit conversion of fmt::basic_runtime<> to char* for std::string ctor
template <typename T, typename... Ts> constexpr auto firstArg(fmt::basic_runtime<T> && data, Ts &&...) { return data.str.data(); }
[[maybe_unused]] const ::Poco::Logger * getLogger(const ::Poco::Logger * logger) { return logger; }; [[maybe_unused]] const ::Poco::Logger * getLogger(const ::Poco::Logger * logger) { return logger; };
[[maybe_unused]] const ::Poco::Logger * getLogger(const std::atomic<::Poco::Logger *> & logger) { return logger.load(); }; [[maybe_unused]] const ::Poco::Logger * getLogger(const std::atomic<::Poco::Logger *> & logger) { return logger.load(); };
[[maybe_unused]] std::unique_ptr<LogToStrImpl> getLogger(std::unique_ptr<LogToStrImpl> && logger) { return logger; };
} }
#define LOG_IMPL_FIRST_ARG(X, ...) X
/// Logs a message to a specified logger with that level. /// Logs a message to a specified logger with that level.
/// If more than one argument is provided, /// If more than one argument is provided,
/// the first argument is interpreted as template with {}-substitutions /// the first argument is interpreted as template with {}-substitutions
@ -30,7 +54,7 @@ namespace
auto _logger = ::getLogger(logger); \ auto _logger = ::getLogger(logger); \
const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \
(DB::CurrentThread::getGroup()->client_logs_level >= (priority)); \ (DB::CurrentThread::getGroup()->client_logs_level >= (priority)); \
if (_logger->is((PRIORITY)) || _is_clients_log) \ if (_is_clients_log || _logger->is((PRIORITY))) \
{ \ { \
std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \ std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \
if (auto _channel = _logger->getChannel()) \ if (auto _channel = _logger->getChannel()) \
@ -40,7 +64,7 @@ namespace
file_function += "; "; \ file_function += "; "; \
file_function += __PRETTY_FUNCTION__; \ file_function += __PRETTY_FUNCTION__; \
Poco::Message poco_message(_logger->name(), formatted_message, \ Poco::Message poco_message(_logger->name(), formatted_message, \
(PRIORITY), file_function.c_str(), __LINE__); \ (PRIORITY), file_function.c_str(), __LINE__, tryGetStaticFormatString(LOG_IMPL_FIRST_ARG(__VA_ARGS__))); \
_channel->log(poco_message); \ _channel->log(poco_message); \
} \ } \
} \ } \

View File

@ -0,0 +1,94 @@
#include <string>
#include <vector>
#include <IO/WriteBufferFromVector.h>
#include <Common/FST.h>
#include <gtest/gtest.h>
TEST(FST, SimpleTest)
{
std::vector<std::pair<std::string, DB::FST::Output>> indexed_data
{
{"mop", 100},
{"moth", 91},
{"pop", 72},
{"star", 83},
{"stop", 54},
{"top", 55},
};
std::vector<std::pair<std::string, DB::FST::Output>> not_indexed_data
{
{"mo", 100},
{"moth1", 91},
{"po", 72},
{"star2", 83},
{"sto", 54},
{"top33", 55},
};
std::vector<UInt8> buffer;
DB::WriteBufferFromVector<std::vector<UInt8>> wbuf(buffer);
DB::FST::FSTBuilder builder(wbuf);
for (auto& [term, output] : indexed_data)
{
builder.add(term, output);
}
builder.build();
wbuf.finalize();
DB::FST::FiniteStateTransducer fst(buffer);
for (auto& [term, output] : indexed_data)
{
auto [result, found] = fst.getOutput(term);
ASSERT_EQ(found, true);
ASSERT_EQ(result, output);
}
for (auto& [term, output] : not_indexed_data)
{
auto [result, found] = fst.getOutput(term);
ASSERT_EQ(found, false);
}
}
TEST(FST, TestForLongTerms)
{
/// Test long terms within limitation
std::string term1(DB::FST::MAX_TERM_LENGTH - 1, 'A');
std::string term2(DB::FST::MAX_TERM_LENGTH, 'B');
DB::FST::Output output1 = 100;
DB::FST::Output output2 = 200;
std::vector<UInt8> buffer;
DB::WriteBufferFromVector<std::vector<UInt8>> wbuf(buffer);
DB::FST::FSTBuilder builder(wbuf);
builder.add(term1, output1);
builder.add(term2, output2);
builder.build();
wbuf.finalize();
DB::FST::FiniteStateTransducer fst(buffer);
auto [result1, found1] = fst.getOutput(term1);
ASSERT_EQ(found1, true);
ASSERT_EQ(result1, output1);
auto [result2, found2] = fst.getOutput(term2);
ASSERT_EQ(found2, true);
ASSERT_EQ(result2, output2);
/// Test exception case when term length exceeds limitation
std::string term3(DB::FST::MAX_TERM_LENGTH + 1, 'C');
DB::FST::Output output3 = 300;
std::vector<UInt8> buffer3;
DB::WriteBufferFromVector<std::vector<UInt8>> wbuf3(buffer3);
DB::FST::FSTBuilder builder3(wbuf3);
EXPECT_THROW(builder3.add(term3, output3), DB::Exception);
}

View File

@ -357,16 +357,12 @@ nuraft::ptr<nuraft::srv_state> KeeperStateManager::read_state()
if (read_checksum != hash.get64()) if (read_checksum != hash.get64())
{ {
const auto error_string = fmt::format( constexpr auto error_format = "Invalid checksum while reading state from {}. Got {}, expected {}";
"Invalid checksum while reading state from {}. Got {}, expected {}",
path.generic_string(),
hash.get64(),
read_checksum);
#ifdef NDEBUG #ifdef NDEBUG
LOG_ERROR(logger, fmt::runtime(error_string)); LOG_ERROR(logger, error_format, path.generic_string(), hash.get64(), read_checksum);
return nullptr; return nullptr;
#else #else
throw Exception(ErrorCodes::CORRUPTED_DATA, error_string); throw Exception(ErrorCodes::CORRUPTED_DATA, error_format, path.generic_string(), hash.get64(), read_checksum);
#endif #endif
} }

View File

@ -95,6 +95,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(UInt64, s3_max_get_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_get_rps`", 0) \ M(UInt64, s3_max_get_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_get_rps`", 0) \
M(UInt64, s3_max_put_rps, 0, "Limit on S3 PUT request per second rate before throttling. Zero means unlimited.", 0) \ M(UInt64, s3_max_put_rps, 0, "Limit on S3 PUT request per second rate before throttling. Zero means unlimited.", 0) \
M(UInt64, s3_max_put_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_put_rps`", 0) \ M(UInt64, s3_max_put_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_put_rps`", 0) \
M(UInt64, s3_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \
M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \
M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \
M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \
@ -182,6 +183,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(UInt64, merge_tree_max_rows_to_use_cache, (128 * 8192), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \ M(UInt64, merge_tree_max_rows_to_use_cache, (128 * 8192), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \
M(UInt64, merge_tree_max_bytes_to_use_cache, (192 * 10 * 1024 * 1024), "The maximum number of bytes per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \ M(UInt64, merge_tree_max_bytes_to_use_cache, (192 * 10 * 1024 * 1024), "The maximum number of bytes per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \
M(Bool, do_not_merge_across_partitions_select_final, false, "Merge parts only in one partition in select final", 0) \ M(Bool, do_not_merge_across_partitions_select_final, false, "Merge parts only in one partition in select final", 0) \
M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \
\ \
M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \
\ \

View File

@ -278,7 +278,7 @@ private:
if (next_pos != std::string_view::npos) if (next_pos != std::string_view::npos)
size = next_pos - pos; size = next_pos - pos;
LOG_FATAL(log, "{}", message.substr(pos, size)); LOG_FATAL(log, fmt::runtime(message.substr(pos, size)));
pos = next_pos; pos = next_pos;
} }
} }

View File

@ -1,9 +1,10 @@
#include <DataTypes/Serializations/SerializationUUID.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <DataTypes/Serializations/SerializationUUID.h>
#include <Formats/ProtobufReader.h> #include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h> #include <Formats/ProtobufWriter.h>
#include <IO/WriteHelpers.h> #include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/assert_cast.h> #include <Common/assert_cast.h>
@ -44,11 +45,44 @@ void SerializationUUID::serializeTextQuoted(const IColumn & column, size_t row_n
void SerializationUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const void SerializationUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{ {
UUID x; UUID uuid;
assertChar('\'', istr); bool fast = false;
readText(x, istr); if (istr.available() >= 38)
assertChar('\'', istr); {
assert_cast<ColumnUUID &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. assertChar('\'', istr);
char * next_pos = find_first_symbols<'\\', '\''>(istr.position(), istr.buffer().end());
size_t len = next_pos - istr.position();
if ((len == 32) && (istr.position()[32] == '\''))
{
parseUUIDWithoutSeparator(
reinterpret_cast<const UInt8 *>(istr.position()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
istr.ignore(33);
fast = true;
}
else if ((len == 36) && (istr.position()[36] == '\''))
{
parseUUID(
reinterpret_cast<const UInt8 *>(istr.position()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
istr.ignore(37);
fast = true;
}
else
{
// It's ok to go back in the position because we haven't read from the buffer except the first char
// and we know there were at least 38 bytes available (so no new read has been triggered)
istr.position()--;
}
}
if (!fast)
{
String quoted_chars;
readQuotedStringInto<false>(quoted_chars, istr);
ReadBufferFromString parsed_quoted_buffer(quoted_chars);
readText(uuid, parsed_quoted_buffer);
}
assert_cast<ColumnUUID &>(column).getData().push_back(std::move(uuid)); /// It's important to do this at the end - for exception safety.
} }
void SerializationUUID::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const void SerializationUUID::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const

View File

@ -82,7 +82,7 @@ void DatabaseAtomic::drop(ContextPtr)
} }
catch (...) catch (...)
{ {
LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); LOG_WARNING(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ true));
} }
fs::remove_all(getMetadataPath()); fs::remove_all(getMetadataPath());
} }
@ -477,7 +477,7 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String &
} }
catch (...) catch (...)
{ {
LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); LOG_WARNING(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ true));
} }
} }
@ -490,7 +490,7 @@ void DatabaseAtomic::tryRemoveSymlink(const String & table_name)
} }
catch (...) catch (...)
{ {
LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); LOG_WARNING(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ true));
} }
} }
@ -535,7 +535,7 @@ void DatabaseAtomic::renameDatabase(ContextPtr query_context, const String & new
} }
catch (...) catch (...)
{ {
LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); LOG_WARNING(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ true));
} }
auto new_name_escaped = escapeForFileName(new_name); auto new_name_escaped = escapeForFileName(new_name);

View File

@ -100,7 +100,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, Co
if (!load_result.config) if (!load_result.config)
{ {
if (throw_on_error) if (throw_on_error)
throw Exception{"Dictionary " + backQuote(table_name) + " doesn't exist", ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY}; throw Exception(ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY, "Dictionary {} doesn't exist", backQuote(table_name));
return {}; return {};
} }

View File

@ -102,7 +102,7 @@ time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name
auto it = tables_cache.find(table_name); auto it = tables_cache.find(table_name);
if (it != tables_cache.end()) if (it != tables_cache.end())
return it->second.metadata_modification_time; return it->second.metadata_modification_time;
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.", backQuote(database_name), backQuote(table_name));
} }
void DatabaseLazy::alterTable( void DatabaseLazy::alterTable(
@ -185,7 +185,7 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
auto it = tables_cache.find(table_name); auto it = tables_cache.find(table_name);
if (it == tables_cache.end()) if (it == tables_cache.end())
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.", backQuote(database_name), backQuote(table_name));
res = it->second.table; res = it->second.table;
if (it->second.expiration_iterator != cache_expiration_queue.end()) if (it->second.expiration_iterator != cache_expiration_queue.end())
cache_expiration_queue.erase(it->second.expiration_iterator); cache_expiration_queue.erase(it->second.expiration_iterator);

View File

@ -312,7 +312,7 @@ void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_na
} }
catch (...) catch (...)
{ {
LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(__PRETTY_FUNCTION__))); LOG_WARNING(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ true));
attachTable(local_context, table_name, table, table_data_path_relative); attachTable(local_context, table_name, table, table_data_path_relative);
if (renamed) if (renamed)
fs::rename(table_metadata_path_drop, table_metadata_path); fs::rename(table_metadata_path_drop, table_metadata_path);
@ -463,8 +463,7 @@ ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, Contex
catch (const Exception & e) catch (const Exception & e)
{ {
if (!has_table && e.code() == ErrorCodes::FILE_DOESNT_EXIST && throw_on_error) if (!has_table && e.code() == ErrorCodes::FILE_DOESNT_EXIST && throw_on_error)
throw Exception{"Table " + backQuote(table_name) + " doesn't exist", throw Exception(ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY, "Table {} doesn't exist", backQuote(table_name));
ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY};
else if (!is_system_storage && throw_on_error) else if (!is_system_storage && throw_on_error)
throw; throw;
} }

View File

@ -136,8 +136,7 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, Context
if (local_tables_cache.find(table_name) == local_tables_cache.end()) if (local_tables_cache.find(table_name) == local_tables_cache.end())
{ {
if (throw_on_error) if (throw_on_error)
throw Exception("MySQL table " + database_name_in_mysql + "." + table_name + " doesn't exist..", throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.", database_name_in_mysql, table_name);
ErrorCodes::UNKNOWN_TABLE);
return nullptr; return nullptr;
} }
@ -181,7 +180,7 @@ time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_nam
fetchTablesIntoLocalCache(getContext()); fetchTablesIntoLocalCache(getContext());
if (local_tables_cache.find(table_name) == local_tables_cache.end()) if (local_tables_cache.find(table_name) == local_tables_cache.end())
throw Exception("MySQL table " + database_name_in_mysql + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.", database_name_in_mysql, table_name);
return time_t(local_tables_cache[table_name].first); return time_t(local_tables_cache[table_name].first);
} }

View File

@ -111,11 +111,6 @@ std::string ClickHouseDictionarySource::getUpdateFieldAndDate()
} }
} }
QueryPipeline ClickHouseDictionarySource::loadAllWithSizeHint(std::atomic<size_t> * result_size_hint)
{
return createStreamForQuery(load_all_query, result_size_hint);
}
QueryPipeline ClickHouseDictionarySource::loadAll() QueryPipeline ClickHouseDictionarySource::loadAll()
{ {
return createStreamForQuery(load_all_query); return createStreamForQuery(load_all_query);
@ -163,7 +158,7 @@ std::string ClickHouseDictionarySource::toString() const
return "ClickHouse: " + configuration.db + '.' + configuration.table + (where.empty() ? "" : ", where: " + where); return "ClickHouse: " + configuration.db + '.' + configuration.table + (where.empty() ? "" : ", where: " + where);
} }
QueryPipeline ClickHouseDictionarySource::createStreamForQuery(const String & query, std::atomic<size_t> * result_size_hint) QueryPipeline ClickHouseDictionarySource::createStreamForQuery(const String & query)
{ {
QueryPipeline pipeline; QueryPipeline pipeline;
@ -186,14 +181,6 @@ QueryPipeline ClickHouseDictionarySource::createStreamForQuery(const String & qu
std::make_shared<RemoteQueryExecutor>(pool, query, empty_sample_block, context_copy), false, false)); std::make_shared<RemoteQueryExecutor>(pool, query, empty_sample_block, context_copy), false, false));
} }
if (result_size_hint)
{
pipeline.setProgressCallback([result_size_hint](const Progress & progress)
{
*result_size_hint += progress.total_rows_to_read;
});
}
return pipeline; return pipeline;
} }

View File

@ -46,8 +46,6 @@ public:
ClickHouseDictionarySource(const ClickHouseDictionarySource & other); ClickHouseDictionarySource(const ClickHouseDictionarySource & other);
ClickHouseDictionarySource & operator=(const ClickHouseDictionarySource &) = delete; ClickHouseDictionarySource & operator=(const ClickHouseDictionarySource &) = delete;
QueryPipeline loadAllWithSizeHint(std::atomic<size_t> * result_size_hint) override;
QueryPipeline loadAll() override; QueryPipeline loadAll() override;
QueryPipeline loadUpdatedAll() override; QueryPipeline loadUpdatedAll() override;
@ -72,7 +70,7 @@ public:
private: private:
std::string getUpdateFieldAndDate(); std::string getUpdateFieldAndDate();
QueryPipeline createStreamForQuery(const String & query, std::atomic<size_t> * result_size_hint = nullptr); QueryPipeline createStreamForQuery(const String & query);
std::string doInvalidateQuery(const std::string & request) const; std::string doInvalidateQuery(const std::string & request) const;

File diff suppressed because it is too large Load Diff

View File

@ -26,14 +26,20 @@ namespace DB
struct HashedDictionaryStorageConfiguration struct HashedDictionaryStorageConfiguration
{ {
const bool preallocate; const UInt64 shards;
const UInt64 shard_load_queue_backlog;
const bool require_nonempty; const bool require_nonempty;
const DictionaryLifetime lifetime; const DictionaryLifetime lifetime;
}; };
template <DictionaryKeyType dictionary_key_type, bool sparse> template <DictionaryKeyType dictionary_key_type, bool sparse, bool sharded>
class ParallelDictionaryLoader;
template <DictionaryKeyType dictionary_key_type, bool sparse, bool sharded>
class HashedDictionary final : public IDictionary class HashedDictionary final : public IDictionary
{ {
friend class ParallelDictionaryLoader<dictionary_key_type, sparse, sharded>;
public: public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
@ -43,6 +49,7 @@ public:
DictionarySourcePtr source_ptr_, DictionarySourcePtr source_ptr_,
const HashedDictionaryStorageConfiguration & configuration_, const HashedDictionaryStorageConfiguration & configuration_,
BlockPtr update_field_loaded_block_ = nullptr); BlockPtr update_field_loaded_block_ = nullptr);
~HashedDictionary() override;
std::string getTypeName() const override std::string getTypeName() const override
{ {
@ -76,7 +83,12 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override std::shared_ptr<const IExternalLoadable> clone() const override
{ {
return std::make_shared<HashedDictionary<dictionary_key_type, sparse>>(getDictionaryID(), dict_struct, source_ptr->clone(), configuration, update_field_loaded_block); return std::make_shared<HashedDictionary<dictionary_key_type, sparse, sharded>>(
getDictionaryID(),
dict_struct,
source_ptr->clone(),
configuration,
update_field_loaded_block);
} }
DictionarySourcePtr getSource() const override { return source_ptr; } DictionarySourcePtr getSource() const override { return source_ptr; }
@ -156,6 +168,9 @@ private:
template <typename Value> template <typename Value>
using CollectionType = std::conditional_t<sparse, CollectionTypeSparse<Value>, CollectionTypeNonSparse<Value>>; using CollectionType = std::conditional_t<sparse, CollectionTypeSparse<Value>, CollectionTypeNonSparse<Value>>;
template <typename Value>
using CollectionsHolder = std::vector<CollectionType<Value>>;
using NoAttributesCollectionType = std::conditional_t<sparse, NoAttributesCollectionTypeSparse, NoAttributesCollectionTypeNonSparse>; using NoAttributesCollectionType = std::conditional_t<sparse, NoAttributesCollectionTypeSparse, NoAttributesCollectionTypeNonSparse>;
using NullableSet = HashSet<KeyType, DefaultHash<KeyType>>; using NullableSet = HashSet<KeyType, DefaultHash<KeyType>>;
@ -166,36 +181,36 @@ private:
std::optional<NullableSet> is_nullable_set; std::optional<NullableSet> is_nullable_set;
std::variant< std::variant<
CollectionType<UInt8>, CollectionsHolder<UInt8>,
CollectionType<UInt16>, CollectionsHolder<UInt16>,
CollectionType<UInt32>, CollectionsHolder<UInt32>,
CollectionType<UInt64>, CollectionsHolder<UInt64>,
CollectionType<UInt128>, CollectionsHolder<UInt128>,
CollectionType<UInt256>, CollectionsHolder<UInt256>,
CollectionType<Int8>, CollectionsHolder<Int8>,
CollectionType<Int16>, CollectionsHolder<Int16>,
CollectionType<Int32>, CollectionsHolder<Int32>,
CollectionType<Int64>, CollectionsHolder<Int64>,
CollectionType<Int128>, CollectionsHolder<Int128>,
CollectionType<Int256>, CollectionsHolder<Int256>,
CollectionType<Decimal32>, CollectionsHolder<Decimal32>,
CollectionType<Decimal64>, CollectionsHolder<Decimal64>,
CollectionType<Decimal128>, CollectionsHolder<Decimal128>,
CollectionType<Decimal256>, CollectionsHolder<Decimal256>,
CollectionType<DateTime64>, CollectionsHolder<DateTime64>,
CollectionType<Float32>, CollectionsHolder<Float32>,
CollectionType<Float64>, CollectionsHolder<Float64>,
CollectionType<UUID>, CollectionsHolder<UUID>,
CollectionType<IPv4>, CollectionsHolder<IPv4>,
CollectionType<IPv6>, CollectionsHolder<IPv6>,
CollectionType<StringRef>, CollectionsHolder<StringRef>,
CollectionType<Array>> CollectionsHolder<Array>>
container; containers;
}; };
void createAttributes(); void createAttributes();
void blockToAttributes(const Block & block); void blockToAttributes(const Block & block, DictionaryKeysArenaHolder<dictionary_key_type> & arena_holder, UInt64 shard);
void updateData(); void updateData();
@ -205,6 +220,22 @@ private:
void calculateBytesAllocated(); void calculateBytesAllocated();
UInt64 getShard(UInt64 key) const
{
if constexpr (!sharded)
return 0;
/// NOTE: function here should not match with the DefaultHash<> since
/// it used for the HashMap/sparse_hash_map.
return intHashCRC32(key) % configuration.shards;
}
UInt64 getShard(StringRef key) const
{
if constexpr (!sharded)
return 0;
return StringRefHash()(key) % configuration.shards;
}
template <typename AttributeType, bool is_nullable, typename ValueSetter, typename DefaultValueExtractor> template <typename AttributeType, bool is_nullable, typename ValueSetter, typename DefaultValueExtractor>
void getItemsImpl( void getItemsImpl(
const Attribute & attribute, const Attribute & attribute,
@ -220,6 +251,8 @@ private:
void resize(size_t added_rows); void resize(size_t added_rows);
Poco::Logger * log;
const DictionaryStructure dict_struct; const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr; const DictionarySourcePtr source_ptr;
const HashedDictionaryStorageConfiguration configuration; const HashedDictionaryStorageConfiguration configuration;
@ -228,21 +261,25 @@ private:
size_t bytes_allocated = 0; size_t bytes_allocated = 0;
size_t hierarchical_index_bytes_allocated = 0; size_t hierarchical_index_bytes_allocated = 0;
size_t element_count = 0; std::atomic<size_t> element_count = 0;
size_t bucket_count = 0; size_t bucket_count = 0;
mutable std::atomic<size_t> query_count{0}; mutable std::atomic<size_t> query_count{0};
mutable std::atomic<size_t> found_count{0}; mutable std::atomic<size_t> found_count{0};
BlockPtr update_field_loaded_block; BlockPtr update_field_loaded_block;
Arena string_arena; std::vector<std::unique_ptr<Arena>> string_arenas;
NoAttributesCollectionType no_attributes_container; std::vector<NoAttributesCollectionType> no_attributes_containers;
DictionaryHierarchicalParentToChildIndexPtr hierarchical_index; DictionaryHierarchicalParentToChildIndexPtr hierarchical_index;
}; };
extern template class HashedDictionary<DictionaryKeyType::Simple, false>; extern template class HashedDictionary<DictionaryKeyType::Simple, /* sparse= */ false, /* sharded= */ false>;
extern template class HashedDictionary<DictionaryKeyType::Simple, true>; extern template class HashedDictionary<DictionaryKeyType::Simple, /* sparse= */ false, /* sharded= */ true>;
extern template class HashedDictionary<DictionaryKeyType::Simple, /* sparse= */ true, /* sharded= */ false>;
extern template class HashedDictionary<DictionaryKeyType::Simple, /* sparse= */ true, /* sharded= */ true>;
extern template class HashedDictionary<DictionaryKeyType::Complex, false>; extern template class HashedDictionary<DictionaryKeyType::Complex, /* sparse= */ false, /* sharded= */ false>;
extern template class HashedDictionary<DictionaryKeyType::Complex, true>; extern template class HashedDictionary<DictionaryKeyType::Complex, /* sparse= */ false, /* sharded= */ true>;
extern template class HashedDictionary<DictionaryKeyType::Complex, /* sparse= */ true, /* sharded= */ false>;
extern template class HashedDictionary<DictionaryKeyType::Complex, /* sparse= */ true, /* sharded= */ true>;
} }

View File

@ -4,7 +4,6 @@
#include <QueryPipeline/QueryPipeline.h> #include <QueryPipeline/QueryPipeline.h>
#include <vector> #include <vector>
#include <atomic>
namespace DB namespace DB
@ -26,44 +25,6 @@ public:
/// Returns a pipe with updated data available from this source. /// Returns a pipe with updated data available from this source.
virtual QueryPipeline loadUpdatedAll() = 0; virtual QueryPipeline loadUpdatedAll() = 0;
/**
* result_size_hint - approx number of rows in the stream.
* Returns a pipe with all the data available from this source.
*
* NOTE: result_size_hint may be changed during you are reading (usually it
* will be non zero for the first block and zero for others, since it uses
* Progress::total_rows_approx,) from the pipe, and may be called
* in parallel, so you should use something like this:
*
* ...
* std::atomic<uint64_t> new_size = 0;
*
* QueryPipeline pipeline;
* pipeline.init(source->loadAll(&new_size));
* PullingPipelineExecutor executor;
*
* Block block;
* while (executor.pull(block))
* {
* if (new_size)
* {
* size_t current_new_size = new_size.exchange(0);
* if (current_new_size)
* resize(current_new_size);
* }
* else
* {
* resize(block.rows());
* }
* }
*
* ...
*/
virtual QueryPipeline loadAllWithSizeHint(std::atomic<size_t> * /* result_size_hint */)
{
return loadAll();
}
/** Indicates whether this source supports "random access" loading of data /** Indicates whether this source supports "random access" loading of data
* loadId and loadIds can only be used if this function returns true. * loadId and loadIds can only be used if this function returns true.
*/ */

View File

@ -243,13 +243,23 @@ void RegExpTreeDictionary::loadData()
initRegexNodes(block); initRegexNodes(block);
} }
initGraph(); initGraph();
if (regexps.empty())
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "There are no available regular expression. Please check your config");
#if USE_VECTORSCAN #if USE_VECTORSCAN
std::vector<std::string_view> regexps_views(regexps.begin(), regexps.end()); try
hyperscan_regex = MultiRegexps::getOrSet<true, false>(regexps_views, std::nullopt); {
/// TODO: fallback when exceptions occur. std::vector<std::string_view> regexps_views(regexps.begin(), regexps.end());
hyperscan_regex->get(); hyperscan_regex = MultiRegexps::getOrSet<true, false>(regexps_views, std::nullopt);
hyperscan_regex->get();
}
catch (Exception & e)
{
/// Some compile errors will be thrown as LOGICAL ERROR and cause crash, e.g. empty expression or expressions are too large.
/// We catch the error here and rethrow again.
/// TODO: fallback to other engine, like re2, when exceptions occur.
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Error occurs when compiling regular expressions, reason: {}", e.message());
}
#endif #endif
} }
else else
{ {

View File

@ -289,6 +289,12 @@ std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(
std::optional<size_t> read_hint, std::optional<size_t> read_hint,
std::optional<size_t> file_size) const std::optional<size_t> file_size) const
{ {
if (read_hint && *read_hint > 0)
read_hint = *read_hint + FileEncryption::Header::kSize;
if (file_size && *file_size > 0)
file_size = *file_size + FileEncryption::Header::kSize;
auto wrapped_path = wrappedPath(path); auto wrapped_path = wrappedPath(path);
auto buffer = delegate->readFile(wrapped_path, settings, read_hint, file_size); auto buffer = delegate->readFile(wrapped_path, settings, read_hint, file_size);
if (buffer->eof()) if (buffer->eof())

View File

@ -113,8 +113,8 @@ DiskSelectorPtr DiskSelector::updateFromConfig(
writeBackQuotedString(name, warning); writeBackQuotedString(name, warning);
} }
writeString(" disappeared from configuration, this change will be applied after restart of ClickHouse", warning); LOG_WARNING(&Poco::Logger::get("DiskSelector"), "{} disappeared from configuration, "
LOG_WARNING(&Poco::Logger::get("DiskSelector"), fmt::runtime(warning.str())); "this change will be applied after restart of ClickHouse", warning.str());
} }
return result; return result;

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