mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge branch 'master' into cancellable-mutex-integration
This commit is contained in:
commit
0fbfa17863
1
.github/workflows/backport_branches.yml
vendored
1
.github/workflows/backport_branches.yml
vendored
@ -683,3 +683,4 @@ jobs:
|
||||
run: |
|
||||
cd "$GITHUB_WORKSPACE/tests/ci"
|
||||
python3 finish_check.py
|
||||
python3 merge_pr.py
|
||||
|
2
.github/workflows/debug.yml
vendored
2
.github/workflows/debug.yml
vendored
@ -8,4 +8,4 @@ jobs:
|
||||
DebugInfo:
|
||||
runs-on: ubuntu-latest
|
||||
steps:
|
||||
- uses: hmarr/debug-action@1201a20fc9d278ddddd5f0f46922d06513892491
|
||||
- uses: hmarr/debug-action@a701ed95a46e6f2fb0df25e1a558c16356fae35a
|
||||
|
1
.github/workflows/docs_check.yml
vendored
1
.github/workflows/docs_check.yml
vendored
@ -169,3 +169,4 @@ jobs:
|
||||
run: |
|
||||
cd "$GITHUB_WORKSPACE/tests/ci"
|
||||
python3 finish_check.py
|
||||
python3 merge_pr.py --check-approved
|
||||
|
1
.github/workflows/pull_request.yml
vendored
1
.github/workflows/pull_request.yml
vendored
@ -4388,3 +4388,4 @@ jobs:
|
||||
run: |
|
||||
cd "$GITHUB_WORKSPACE/tests/ci"
|
||||
python3 finish_check.py
|
||||
python3 merge_pr.py --check-approved
|
||||
|
2
.gitignore
vendored
2
.gitignore
vendored
@ -154,6 +154,8 @@ website/package-lock.json
|
||||
/programs/server/data
|
||||
/programs/server/metadata
|
||||
/programs/server/store
|
||||
/programs/server/uuid
|
||||
/programs/server/coordination
|
||||
|
||||
# temporary test files
|
||||
tests/queries/0_stateless/test_*
|
||||
|
3
.gitmodules
vendored
3
.gitmodules
vendored
@ -327,3 +327,6 @@
|
||||
[submodule "contrib/aws-s2n-tls"]
|
||||
path = contrib/aws-s2n-tls
|
||||
url = https://github.com/ClickHouse/s2n-tls
|
||||
[submodule "contrib/crc32-vpmsum"]
|
||||
path = contrib/crc32-vpmsum
|
||||
url = https://github.com/antonblanchard/crc32-vpmsum.git
|
||||
|
1
contrib/CMakeLists.txt
vendored
1
contrib/CMakeLists.txt
vendored
@ -55,6 +55,7 @@ else ()
|
||||
endif ()
|
||||
add_contrib (miniselect-cmake miniselect)
|
||||
add_contrib (pdqsort-cmake pdqsort)
|
||||
add_contrib (crc32-vpmsum-cmake crc32-vpmsum)
|
||||
add_contrib (sparsehash-c11-cmake sparsehash-c11)
|
||||
add_contrib (abseil-cpp-cmake abseil-cpp)
|
||||
add_contrib (magic-enum-cmake magic_enum)
|
||||
|
1
contrib/crc32-vpmsum
vendored
Submodule
1
contrib/crc32-vpmsum
vendored
Submodule
@ -0,0 +1 @@
|
||||
Subproject commit 452155439389311fc7d143621eaf56a258e02476
|
14
contrib/crc32-vpmsum-cmake/CMakeLists.txt
Normal file
14
contrib/crc32-vpmsum-cmake/CMakeLists.txt
Normal file
@ -0,0 +1,14 @@
|
||||
# module crc32-vpmsum gets build along with the files vec_crc32.h and crc32_constants.h in crc32-vpmsum-cmake
|
||||
# Please see README.md for information about how to generate crc32_constants.h
|
||||
if (NOT ARCH_PPC64LE)
|
||||
message (STATUS "crc32-vpmsum library is only supported on ppc64le")
|
||||
return()
|
||||
endif()
|
||||
|
||||
SET(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/crc32-vpmsum")
|
||||
|
||||
add_library(_crc32-vpmsum
|
||||
"${LIBRARY_DIR}/vec_crc32.c"
|
||||
)
|
||||
target_include_directories(_crc32-vpmsum SYSTEM BEFORE PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}")
|
||||
add_library(ch_contrib::crc32-vpmsum ALIAS _crc32-vpmsum)
|
9
contrib/crc32-vpmsum-cmake/README.md
Normal file
9
contrib/crc32-vpmsum-cmake/README.md
Normal file
@ -0,0 +1,9 @@
|
||||
# To Generate crc32_constants.h
|
||||
|
||||
- Run make file in `../crc32-vpmsum` directory using following options and CRC polynomial. These options should use the same polynomial and order used by intel intrinisic functions
|
||||
```bash
|
||||
make crc32_constants.h CRC="0x11EDC6F41" OPTIONS="-x -r -c"
|
||||
```
|
||||
- move the generated `crc32_constants.h` into this directory
|
||||
- To understand more about this go here: https://masterchef2209.wordpress.com/2020/06/17/guide-to-intel-sse4-2-crc-intrinisics-implementation-for-simde/
|
||||
- Here is the link to information about intel intrinsic functions: https://www.intel.com/content/www/us/en/docs/intrinsics-guide/index.html#text=_mm_crc32_u64&ig_expand=1492,1493,1559
|
1206
contrib/crc32-vpmsum-cmake/crc32_constants.h
Normal file
1206
contrib/crc32-vpmsum-cmake/crc32_constants.h
Normal file
File diff suppressed because it is too large
Load Diff
26
contrib/crc32-vpmsum-cmake/vec_crc32.h
Normal file
26
contrib/crc32-vpmsum-cmake/vec_crc32.h
Normal file
@ -0,0 +1,26 @@
|
||||
#ifndef VEC_CRC32
|
||||
#define VEC_CRC32
|
||||
|
||||
#if ! ((defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__)
|
||||
# error PowerPC architecture is expected
|
||||
#endif
|
||||
|
||||
#ifdef __cplusplus
|
||||
extern "C" {
|
||||
#endif
|
||||
|
||||
unsigned int crc32_vpmsum(unsigned int crc, const unsigned char *p, unsigned long len);
|
||||
|
||||
static inline uint32_t crc32_ppc(uint64_t crc, unsigned char const *buffer, size_t len)
|
||||
{
|
||||
assert(buffer);
|
||||
crc = crc32_vpmsum(crc, buffer, (unsigned long)len);
|
||||
|
||||
return crc;
|
||||
}
|
||||
|
||||
#ifdef __cplusplus
|
||||
}
|
||||
#endif
|
||||
|
||||
#endif
|
2
contrib/poco
vendored
2
contrib/poco
vendored
@ -1 +1 @@
|
||||
Subproject commit 799234226187c0ae0b8c90f23465b25ed7956e56
|
||||
Subproject commit 4b1c8dd9913d2a16db62df0e509fa598da5c8219
|
@ -58,7 +58,7 @@ echo 'SELECT version()' | curl 'http://localhost:18123/' --data-binary @-
|
||||
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
|
||||
docker run -d --network=host --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server
|
||||
|
@ -5,6 +5,7 @@ set -x
|
||||
|
||||
# core.COMM.PID-TID
|
||||
sysctl kernel.core_pattern='core.%e.%p-%P'
|
||||
dmesg --clear ||:
|
||||
|
||||
set -e
|
||||
set -u
|
||||
@ -17,13 +18,25 @@ repo_dir=ch
|
||||
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"}
|
||||
|
||||
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
|
||||
{
|
||||
# For local runs, start directly from the "fuzz" stage.
|
||||
rm -rf "$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"
|
||||
if [ "$PR_TO_TEST" != "0" ]; then
|
||||
@ -368,6 +381,7 @@ if [ -f core.zst ]; then
|
||||
fi
|
||||
|
||||
rg --text -F '<Fatal>' server.log > fatal.log ||:
|
||||
dmesg -T > dmesg.log ||:
|
||||
|
||||
zstd --threads=0 server.log
|
||||
|
||||
@ -396,6 +410,7 @@ p.links a { padding: 5px; margin: 3px; background: #FFF; line-height: 2; white-s
|
||||
<a href="fuzzer.log">fuzzer.log</a>
|
||||
<a href="server.log.zst">server.log.zst</a>
|
||||
<a href="main.log">main.log</a>
|
||||
<a href="dmesg.log">dmesg.log</a>
|
||||
${CORE_LINK}
|
||||
</p>
|
||||
<table>
|
||||
|
@ -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.
|
||||
|
||||
#### Partial Queries
|
||||
#### Backward-incompatible Queries
|
||||
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.
|
||||
|
@ -399,7 +399,7 @@ clickhouse-local --query "
|
||||
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');
|
||||
|
||||
-- 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
|
||||
-- have to show some stats so that the PR author can tweak them.
|
||||
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')
|
||||
;
|
||||
|
||||
-- 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).
|
||||
create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv')
|
||||
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
|
||||
-- calculate and check the average query run time in the report.
|
||||
-- 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
|
||||
-- particular server.
|
||||
--
|
||||
|
@ -30,7 +30,7 @@ faster_queries = 0
|
||||
slower_queries = 0
|
||||
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
|
||||
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")
|
||||
if not rows:
|
||||
return
|
||||
|
||||
global unstable_partial_queries, slow_average_tests, tables
|
||||
text = tableStart("Partial Queries")
|
||||
global unstable_backward_incompatible_queries, slow_average_tests, tables
|
||||
text = tableStart("Backward-incompatible queries")
|
||||
columns = ["Median time, s", "Relative time variance", "Test", "#", "Query"]
|
||||
text += tableHeader(columns)
|
||||
attrs = ["" for c in columns]
|
||||
@ -392,7 +392,7 @@ if args.report == "main":
|
||||
anchor = f"{currentTableAnchor()}.{row[2]}.{row[3]}"
|
||||
if float(row[1]) > 0.10:
|
||||
attrs[1] = f'style="background: {color_bad}"'
|
||||
unstable_partial_queries += 1
|
||||
unstable_backward_incompatible_queries += 1
|
||||
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>"
|
||||
@ -414,7 +414,7 @@ if args.report == "main":
|
||||
text += tableEnd()
|
||||
tables.append(text)
|
||||
|
||||
add_partial()
|
||||
add_backward_incompatible()
|
||||
|
||||
def add_changes():
|
||||
rows = tsvRows("report/changed-perf.tsv")
|
||||
@ -630,8 +630,8 @@ if args.report == "main":
|
||||
status = "failure"
|
||||
message_array.append(str(slower_queries) + " slower")
|
||||
|
||||
if unstable_partial_queries:
|
||||
very_unstable_queries += unstable_partial_queries
|
||||
if unstable_backward_incompatible_queries:
|
||||
very_unstable_queries += unstable_backward_incompatible_queries
|
||||
status = "failure"
|
||||
|
||||
# Don't show mildly unstable queries, only the very unstable ones we
|
||||
|
@ -130,6 +130,8 @@ function run_tests()
|
||||
ADDITIONAL_OPTIONS+=('--report-coverage')
|
||||
fi
|
||||
|
||||
ADDITIONAL_OPTIONS+=('--report-logs-stats')
|
||||
|
||||
set +e
|
||||
clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \
|
||||
--test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
|
||||
|
@ -289,6 +289,7 @@ if __name__ == "__main__":
|
||||
"--database=system",
|
||||
"--hung-check",
|
||||
"--stress",
|
||||
"--report-logs-stats",
|
||||
"00001_select_1",
|
||||
]
|
||||
)
|
||||
|
@ -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).
|
||||
|
||||
## 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}
|
||||
|
||||
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.
|
||||
|
@ -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.
|
||||
|
||||
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”.
|
||||
|
BIN
docs/en/development/images/concurrency.png
Normal file
BIN
docs/en/development/images/concurrency.png
Normal file
Binary file not shown.
After Width: | Height: | Size: 34 KiB |
BIN
docs/en/development/images/find-build-artifact.png
Normal file
BIN
docs/en/development/images/find-build-artifact.png
Normal file
Binary file not shown.
After Width: | Height: | Size: 122 KiB |
@ -136,3 +136,7 @@ DESCRIBE TABLE test_database.test_table;
|
||||
│ data │ Nullable(String) │
|
||||
└────────┴───────────────────┘
|
||||
```
|
||||
|
||||
## Related content
|
||||
|
||||
- Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres)
|
||||
|
@ -175,3 +175,6 @@ CREATE TABLE pg_table_schema_with_dots (a UInt32)
|
||||
|
||||
- [The `postgresql` table function](../../../sql-reference/table-functions/postgresql.md)
|
||||
- [Using PostgreSQL as a dictionary source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql)
|
||||
|
||||
## Related content
|
||||
- Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres)
|
||||
|
@ -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).
|
@ -40,8 +40,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1],
|
||||
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2],
|
||||
...
|
||||
INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1,
|
||||
INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2,
|
||||
INDEX index_name1 expr1 TYPE type1(...) [GRANULARITY value1],
|
||||
INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2],
|
||||
...
|
||||
PROJECTION projection_name_1 (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.
|
||||
|
||||
``` 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.
|
||||
|
||||
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**
|
||||
|
||||
``` 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}
|
||||
|
||||
#### `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.
|
||||
|
||||
#### `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.
|
||||
|
||||
#### `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,
|
||||
- `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.
|
||||
- `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.
|
||||
|
||||
#### `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.
|
||||
|
||||
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
|
||||
## Example of index creation for Map data type
|
||||
|
||||
```
|
||||
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 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}
|
||||
|
||||
:::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`.
|
||||
|
||||
Configuration markup:
|
||||
@ -894,6 +904,7 @@ Configuration markup:
|
||||
<disks>
|
||||
<s3>
|
||||
<type>s3</type>
|
||||
<support_batch_delete>true</support_batch_delete>
|
||||
<endpoint>https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/root-path/</endpoint>
|
||||
<access_key_id>your_access_key_id</access_key_id>
|
||||
<secret_access_key>your_secret_access_key</secret_access_key>
|
||||
@ -927,6 +938,7 @@ Required parameters:
|
||||
Optional parameters:
|
||||
|
||||
- `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_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.
|
||||
|
@ -271,6 +271,9 @@ You’ll 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.
|
||||
|
||||
### 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}
|
||||
|
||||
To start the server as a daemon, run:
|
||||
|
@ -1203,12 +1203,14 @@ SELECT * FROM json_each_row_nested
|
||||
- [input_format_json_read_bools_as_numbers](/docs/en/operations/settings/settings.md/#input_format_json_read_bools_as_numbers) - allow to parse bools as numbers in JSON input formats. Default value - `true`.
|
||||
- [input_format_json_read_numbers_as_strings](/docs/en/operations/settings/settings.md/#input_format_json_read_numbers_as_strings) - allow to parse numbers as strings in JSON input formats. Default value - `false`.
|
||||
- [input_format_json_read_objects_as_strings](/docs/en/operations/settings/settings.md/#input_format_json_read_objects_as_strings) - allow to parse JSON objects as strings in JSON input formats. Default value - `false`.
|
||||
- [input_format_json_named_tuples_as_objects](/docs/en/operations/settings/settings.md/#input_format_json_named_tuples_as_objects) - parse named tuple columns as JSON objects. Default value - `true`.
|
||||
- [input_format_json_defaults_for_missing_elements_in_named_tuple](/docs/en/operations/settings/settings.md/#input_format_json_defaults_for_missing_elements_in_named_tuple) - insert default values for missing elements in JSON object while parsing named tuple. Default value - `true`.
|
||||
- [output_format_json_quote_64bit_integers](/docs/en/operations/settings/settings.md/#output_format_json_quote_64bit_integers) - controls quoting of 64-bit integers in JSON output format. Default value - `true`.
|
||||
- [output_format_json_quote_64bit_floats](/docs/en/operations/settings/settings.md/#output_format_json_quote_64bit_floats) - controls quoting of 64-bit floats in JSON output format. Default value - `false`.
|
||||
- [output_format_json_quote_denormals](/docs/en/operations/settings/settings.md/#output_format_json_quote_denormals) - enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format. Default value - `false`.
|
||||
- [output_format_json_quote_decimals](/docs/en/operations/settings/settings.md/#output_format_json_quote_decimals) - controls quoting of decimals in JSON output format. Default value - `false`.
|
||||
- [output_format_json_escape_forward_slashes](/docs/en/operations/settings/settings.md/#output_format_json_escape_forward_slashes) - controls escaping forward slashes for string outputs in JSON output format. Default value - `true`.
|
||||
- [output_format_json_named_tuples_as_objects](/docs/en/operations/settings/settings.md/#output_format_json_named_tuples_as_objects) - serialize named tuple columns as JSON objects. Default value - `false`.
|
||||
- [output_format_json_named_tuples_as_objects](/docs/en/operations/settings/settings.md/#output_format_json_named_tuples_as_objects) - serialize named tuple columns as JSON objects. Default value - `true`.
|
||||
- [output_format_json_array_of_rows](/docs/en/operations/settings/settings.md/#output_format_json_array_of_rows) - output a JSON array of all rows in JSONEachRow(Compact) format. Default value - `false`.
|
||||
- [output_format_json_validate_utf8](/docs/en/operations/settings/settings.md/#output_format_json_validate_utf8) - enables validation of UTF-8 sequences in JSON output formats (note that it doesn't impact formats JSON/JSONCompact/JSONColumnsWithMetadata, they always validate utf8). Default value - `false`.
|
||||
|
||||
|
@ -757,6 +757,10 @@ Possible values:
|
||||
|
||||
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}
|
||||
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).
|
||||
@ -768,6 +772,12 @@ Possible values:
|
||||
|
||||
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}
|
||||
|
||||
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.
|
||||
- `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.
|
||||
- `storage_policy` – Name of storage policy to use for the table (optional)
|
||||
|
||||
**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.
|
||||
- `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.
|
||||
- `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.
|
||||
|
||||
@ -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.
|
||||
- `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.
|
||||
- `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.
|
||||
|
||||
@ -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.
|
||||
- `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.
|
||||
- `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.
|
||||
|
||||
@ -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.
|
||||
- `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.
|
||||
- `storage_policy` – Name of storage policy to use for the table (optional)
|
||||
|
||||
**Example**
|
||||
```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.
|
||||
- `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.
|
||||
- `storage_policy` – Name of storage policy to use for the table (optional)
|
||||
|
||||
The default server configuration file `config.xml` contains the following settings section:
|
||||
|
||||
|
@ -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.
|
||||
|
||||
## 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
|
||||
|
||||
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 │
|
||||
│ s │ Sparse │
|
||||
└────────┴────────────────────┘
|
||||
```
|
||||
```
|
||||
|
@ -266,7 +266,7 @@ Default value: 0.
|
||||
|
||||
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.
|
||||
|
||||
|
@ -402,40 +402,62 @@ Default value: `ALL`.
|
||||
|
||||
## 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.
|
||||
|
||||
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 don’t 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}
|
||||
|
||||
Changes behaviour of join operations with `ANY` strictness.
|
||||
Changes the behaviour of join operations with `ANY` strictness.
|
||||
|
||||
:::warning
|
||||
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.
|
||||
|
||||
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:
|
||||
|
||||
@ -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.
|
||||
:::
|
||||
|
||||
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 `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 `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}
|
||||
|
||||
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:
|
||||
|
||||
@ -706,7 +728,7 @@ log_queries=1
|
||||
|
||||
## 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_thread_log`
|
||||
@ -741,7 +763,7 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING'
|
||||
|
||||
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:
|
||||
|
||||
@ -760,7 +782,7 @@ log_query_threads=1
|
||||
|
||||
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:
|
||||
|
||||
@ -787,7 +809,7 @@ It can be used to improve the readability of server logs. Additionally, it helps
|
||||
|
||||
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.
|
||||
|
||||
@ -821,11 +843,11 @@ The setting also does not have a purpose when using INSERT SELECT, since data is
|
||||
|
||||
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}
|
||||
|
||||
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:
|
||||
|
||||
@ -891,7 +913,7 @@ Higher values will lead to higher memory usage.
|
||||
|
||||
## 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
|
||||
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}
|
||||
|
||||
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).
|
||||
|
||||
@ -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 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}
|
||||
|
||||
Enables or disables the deduplication check for materialized views that receive data from Replicated\* tables.
|
||||
@ -4122,7 +4160,20 @@ Enabled by default.
|
||||
|
||||
Serialize named tuple columns as JSON objects.
|
||||
|
||||
Disabled by default.
|
||||
Enabled by default.
|
||||
|
||||
### input_format_json_named_tuples_as_objects {#input_format_json_named_tuples_as_objects}
|
||||
|
||||
Parse named tuple columns as JSON objects.
|
||||
|
||||
Enabled by default.
|
||||
|
||||
### input_format_json_defaults_for_missing_elements_in_named_tuple {#input_format_json_defaults_for_missing_elements_in_named_tuple}
|
||||
|
||||
Insert default values for missing elements in JSON object while parsing named tuple.
|
||||
This setting works only when setting `input_format_json_named_tuples_as_objects` is enabled.
|
||||
|
||||
Enabled by default.
|
||||
|
||||
### output_format_json_array_of_rows {#output_format_json_array_of_rows}
|
||||
|
||||
|
@ -120,5 +120,6 @@ Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec.
|
||||
|
||||
## Related Content
|
||||
|
||||
- [Extracting, converting, and querying data in local files using clickhouse-local](https://clickhouse.com/blog/extracting-converting-querying-local-files-with-sql-clickhouse-local)
|
||||
- [Getting Data Into ClickHouse - Part 1](https://clickhouse.com/blog/getting-data-into-clickhouse-part-1)
|
||||
- [Exploring massive, real-world data sets: 100+ Years of Weather Records in ClickHouse](https://clickhouse.com/blog/real-world-data-noaa-climate-data)
|
||||
|
@ -57,6 +57,7 @@ ClickHouse-specific aggregate functions:
|
||||
- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md)
|
||||
- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md)
|
||||
- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md)
|
||||
- [uniqTheta](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md)
|
||||
- [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md)
|
||||
- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md)
|
||||
- [quantileExact](../../../sql-reference/aggregate-functions/reference/quantileexact.md)
|
||||
@ -77,4 +78,6 @@ ClickHouse-specific aggregate functions:
|
||||
- [contingency](./contingency.md)
|
||||
- [cramersV](./cramersv.md)
|
||||
- [cramersVBiasCorrected](./cramersvbiascorrected.md)
|
||||
- [theilsU](./theilsu.md)
|
||||
- [theilsU](./theilsu.md)
|
||||
- [maxIntersections](./maxintersections.md)
|
||||
- [maxIntersectionsPosition](./maxintersectionsposition.md)
|
||||
|
@ -0,0 +1,64 @@
|
||||
---
|
||||
slug: /en/sql-reference/aggregate-functions/reference/maxintersections
|
||||
sidebar_position: 360
|
||||
title: maxIntersections
|
||||
---
|
||||
|
||||
# maxIntersections
|
||||
|
||||
Aggregate function that calculates the maximum number of times that a group of intervals intersects each other (if all the intervals intersect at least once).
|
||||
|
||||
The syntax is:
|
||||
|
||||
```sql
|
||||
maxIntersections(start_column, end_column)
|
||||
```
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `start_column` – the numeric column that represents the start of each interval. If `start_column` is `NULL` or 0 then the interval will be skipped.
|
||||
|
||||
- `end_column` - the numeric column that represents the end of each interval. If `end_column` is `NULL` or 0 then the interval will be skipped.
|
||||
|
||||
**Returned value**
|
||||
|
||||
Returns the maximum number of intersected intervals.
|
||||
|
||||
**Example**
|
||||
|
||||
```sql
|
||||
CREATE TABLE my_events (
|
||||
start UInt32,
|
||||
end UInt32
|
||||
)
|
||||
Engine = MergeTree
|
||||
ORDER BY tuple();
|
||||
|
||||
INSERT INTO my_events VALUES
|
||||
(1, 3),
|
||||
(1, 6),
|
||||
(2, 5),
|
||||
(3, 7);
|
||||
```
|
||||
|
||||
The intervals look like the following:
|
||||
|
||||
```response
|
||||
1 - 3
|
||||
1 - - - - 6
|
||||
2 - - 5
|
||||
3 - - - 7
|
||||
```
|
||||
|
||||
Three of these intervals have a common value (the value is `4`, but the value that is common is not important, we are measuring the count of the intersections). The intervals `(1,3)` and `(3,7)` share an endpoint but are not considered intersecting by the `maxIntersections` function.
|
||||
|
||||
```sql
|
||||
SELECT maxIntersections(start, end) FROM my_events;
|
||||
```
|
||||
|
||||
Response:
|
||||
```response
|
||||
3
|
||||
```
|
||||
|
||||
If you have multiple occurrences of the maximum interval, you can use the [`maxIntersectionsPosition` function](./maxintersectionsposition.md) to locate the number and location of those occurrences.
|
@ -0,0 +1,64 @@
|
||||
---
|
||||
slug: /en/sql-reference/aggregate-functions/reference/maxintersectionsposition
|
||||
sidebar_position: 361
|
||||
title: maxIntersectionsPosition
|
||||
---
|
||||
|
||||
# maxIntersectionsPosition
|
||||
|
||||
Aggregate function that calculates the positions of the occurrences of the [`maxIntersections` function](./maxintersections.md).
|
||||
|
||||
The syntax is:
|
||||
|
||||
```sql
|
||||
maxIntersectionsPosition(start_column, end_column)
|
||||
```
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `start_column` – the numeric column that represents the start of each interval. If `start_column` is `NULL` or 0 then the interval will be skipped.
|
||||
|
||||
- `end_column` - the numeric column that represents the end of each interval. If `end_column` is `NULL` or 0 then the interval will be skipped.
|
||||
|
||||
**Returned value**
|
||||
|
||||
Returns the start positions of the maximum number of intersected intervals.
|
||||
|
||||
**Example**
|
||||
|
||||
```sql
|
||||
CREATE TABLE my_events (
|
||||
start UInt32,
|
||||
end UInt32
|
||||
)
|
||||
Engine = MergeTree
|
||||
ORDER BY tuple();
|
||||
|
||||
INSERT INTO my_events VALUES
|
||||
(1, 3),
|
||||
(1, 6),
|
||||
(2, 5),
|
||||
(3, 7);
|
||||
```
|
||||
|
||||
The intervals look like the following:
|
||||
|
||||
```response
|
||||
1 - 3
|
||||
1 - - - - 6
|
||||
2 - - 5
|
||||
3 - - - 7
|
||||
```
|
||||
|
||||
Notice that three of these intervals have the value 4 in common, and that starts with the 2nd interval:
|
||||
|
||||
```sql
|
||||
SELECT maxIntersectionsPosition(start, end) FROM my_events;
|
||||
```
|
||||
|
||||
Response:
|
||||
```response
|
||||
2
|
||||
```
|
||||
|
||||
In other words, the `(1,6)` row is the start of the 3 intervals that intersect, and 3 is the maximum number of intervals that intersect.
|
@ -0,0 +1,68 @@
|
||||
---
|
||||
slug: /en/sql-reference/aggregate-functions/reference/quantileInterpolatedWeighted
|
||||
sidebar_position: 203
|
||||
---
|
||||
|
||||
# quantileInterpolatedWeighted
|
||||
|
||||
Computes [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence using linear interpolation, taking into account the weight of each element.
|
||||
|
||||
To get the interpolated value, all the passed values are combined into an array, which are then sorted by their corresponding weights. Quantile interpolation is then performed using the [weighted percentile method](https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method) by building a cumulative distribution based on weights and then a linear interpolation is performed using the weights and the values to compute the quantiles.
|
||||
|
||||
When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function.
|
||||
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
quantileInterpolatedWeighted(level)(expr, weight)
|
||||
```
|
||||
|
||||
Alias: `medianInterpolatedWeighted`.
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
|
||||
- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md).
|
||||
- `weight` — Column with weights of sequence members. Weight is a number of value occurrences.
|
||||
|
||||
**Returned value**
|
||||
|
||||
- Quantile of the specified level.
|
||||
|
||||
Type:
|
||||
|
||||
- [Float64](../../../sql-reference/data-types/float.md) for numeric data type input.
|
||||
- [Date](../../../sql-reference/data-types/date.md) if input values have the `Date` type.
|
||||
- [DateTime](../../../sql-reference/data-types/datetime.md) if input values have the `DateTime` type.
|
||||
|
||||
**Example**
|
||||
|
||||
Input table:
|
||||
|
||||
``` text
|
||||
┌─n─┬─val─┐
|
||||
│ 0 │ 3 │
|
||||
│ 1 │ 2 │
|
||||
│ 2 │ 1 │
|
||||
│ 5 │ 4 │
|
||||
└───┴─────┘
|
||||
```
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT quantileInterpolatedWeighted(n, val) FROM t
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─quantileInterpolatedWeighted(n, val)─┐
|
||||
│ 1 │
|
||||
└──────────────────────────────────────┘
|
||||
```
|
||||
|
||||
**See Also**
|
||||
|
||||
- [median](../../../sql-reference/aggregate-functions/reference/median.md#median)
|
||||
- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles)
|
@ -9,7 +9,7 @@ sidebar_position: 201
|
||||
|
||||
Syntax: `quantiles(level1, level2, …)(x)`
|
||||
|
||||
All the quantile functions also have corresponding quantiles functions: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantilesTDigest`, `quantilesBFloat16`. These functions calculate all the quantiles of the listed levels in one pass, and return an array of the resulting values.
|
||||
All the quantile functions also have corresponding quantiles functions: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantileInterpolatedWeighted`, `quantilesTDigest`, `quantilesBFloat16`. These functions calculate all the quantiles of the listed levels in one pass, and return an array of the resulting values.
|
||||
|
||||
## quantilesExactExclusive
|
||||
|
||||
|
@ -4,9 +4,9 @@ sidebar_position: 43
|
||||
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
|
||||
|
@ -4,7 +4,7 @@ sidebar_position: 45
|
||||
sidebar_label: FixedString(N)
|
||||
---
|
||||
|
||||
# Fixedstring
|
||||
# FixedString
|
||||
|
||||
A fixed-length string of `N` bytes (neither characters nor code points).
|
||||
|
||||
|
@ -6,6 +6,10 @@ sidebar_label: JSON
|
||||
|
||||
# JSON
|
||||
|
||||
:::warning
|
||||
This feature is experimental and is not production ready. If you need to work with JSON documents, consider using [this guide](/docs/en/guides/developer/working-with-json/json-load-data.md) instead.
|
||||
:::
|
||||
|
||||
Stores JavaScript Object Notation (JSON) documents in a single column.
|
||||
|
||||
`JSON` is an alias for `Object('json')`.
|
||||
|
@ -4,7 +4,7 @@ sidebar_position: 51
|
||||
sidebar_label: LowCardinality
|
||||
---
|
||||
|
||||
# LowCardinality Data Type
|
||||
# LowCardinality
|
||||
|
||||
Changes the internal representation of other data types to be dictionary-encoded.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
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.
|
||||
|
||||
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
|
||||
<layout>
|
||||
<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>
|
||||
</layout>
|
||||
```
|
||||
@ -153,7 +173,7 @@ Configuration example:
|
||||
or
|
||||
|
||||
``` sql
|
||||
LAYOUT(HASHED(PREALLOCATE 0))
|
||||
LAYOUT(HASHED(SHARDS 10 [SHARD_LOAD_QUEUE_BACKLOG 10000]))
|
||||
```
|
||||
|
||||
### 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.
|
||||
|
||||
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:
|
||||
|
||||
``` xml
|
||||
@ -175,9 +193,11 @@ Configuration example:
|
||||
or
|
||||
|
||||
``` 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
|
||||
|
||||
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
|
||||
<layout>
|
||||
<complex_key_hashed />
|
||||
<complex_key_hashed>
|
||||
<shards>1</shards>
|
||||
<!-- <shard_load_queue_backlog>10000</shard_load_queue_backlog> -->
|
||||
</complex_key_hashed>
|
||||
</layout>
|
||||
```
|
||||
|
||||
or
|
||||
|
||||
``` sql
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
LAYOUT(COMPLEX_KEY_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000]))
|
||||
```
|
||||
|
||||
### complex_key_sparse_hashed
|
||||
@ -204,14 +227,16 @@ Configuration example:
|
||||
|
||||
``` xml
|
||||
<layout>
|
||||
<complex_key_sparse_hashed />
|
||||
<complex_key_sparse_hashed>
|
||||
<shards>1</shards>
|
||||
</complex_key_sparse_hashed>
|
||||
</layout>
|
||||
```
|
||||
|
||||
or
|
||||
|
||||
``` sql
|
||||
LAYOUT(COMPLEX_KEY_SPARSE_HASHED())
|
||||
LAYOUT(COMPLEX_KEY_SPARSE_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000]))
|
||||
```
|
||||
|
||||
### hashed_array
|
||||
|
@ -121,7 +121,7 @@ Accepts an empty array and returns a one-element array that is equal to the defa
|
||||
|
||||
## range(end), range(\[start, \] end \[, step\])
|
||||
|
||||
Returns an array of `UInt` numbers from `start` to `end - 1` by `step`.
|
||||
Returns an array of numbers from `start` to `end - 1` by `step`. The supported types are [UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64](../data-types/int-uint.md).
|
||||
|
||||
**Syntax**
|
||||
``` sql
|
||||
@ -130,31 +130,30 @@ range([start, ] end [, step])
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `start` — The first element of the array. Optional, required if `step` is used. Default value: 0. [UInt](../data-types/int-uint.md)
|
||||
- `end` — The number before which the array is constructed. Required. [UInt](../data-types/int-uint.md)
|
||||
- `step` — Determines the incremental step between each element in the array. Optional. Default value: 1. [UInt](../data-types/int-uint.md)
|
||||
- `start` — The first element of the array. Optional, required if `step` is used. Default value: 0.
|
||||
- `end` — The number before which the array is constructed. Required.
|
||||
- `step` — Determines the incremental step between each element in the array. Optional. Default value: 1.
|
||||
|
||||
**Returned value**
|
||||
|
||||
- Array of `UInt` numbers from `start` to `end - 1` by `step`.
|
||||
- Array of numbers from `start` to `end - 1` by `step`.
|
||||
|
||||
**Implementation details**
|
||||
|
||||
- All arguments must be positive values: `start`, `end`, `step` are `UInt` data types, as well as elements of the returned array.
|
||||
- All arguments `start`, `end`, `step` must be below data types: `UInt8`, `UInt16`, `UInt32`, `UInt64`,`Int8`, `Int16`, `Int32`, `Int64`, as well as elements of the returned array, which's type is a super type of all arguments's.
|
||||
- An exception is thrown if query results in arrays with a total length of more than number of elements specified by the [function_range_max_elements_in_block](../../operations/settings/settings.md#settings-function_range_max_elements_in_block) setting.
|
||||
|
||||
|
||||
**Examples**
|
||||
|
||||
Query:
|
||||
``` sql
|
||||
SELECT range(5), range(1, 5), range(1, 5, 2);
|
||||
SELECT range(5), range(1, 5), range(1, 5, 2), range(-1, 5, 2);
|
||||
```
|
||||
Result:
|
||||
```txt
|
||||
┌─range(5)────┬─range(1, 5)─┬─range(1, 5, 2)─┐
|
||||
│ [0,1,2,3,4] │ [1,2,3,4] │ [1,3] │
|
||||
└─────────────┴─────────────┴────────────────┘
|
||||
┌─range(5)────┬─range(1, 5)─┬─range(1, 5, 2)─┬─range(-1, 5, 2)─┐
|
||||
│ [0,1,2,3,4] │ [1,2,3,4] │ [1,3] │ [-1,1,3] │
|
||||
└─────────────┴─────────────┴────────────────┴─────────────────┘
|
||||
```
|
||||
|
||||
## array(x1, …), operator \[x1, …\]
|
||||
|
@ -39,3 +39,16 @@ SELECT * FROM generateRandom('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(
|
||||
│ [68] │ -67417.0770 │ ('2080-03-12 14:17:31.269','110425e5-413f-10a6-05ba-fa6b3e929f15') │
|
||||
└──────────┴──────────────┴────────────────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
```sql
|
||||
CREATE TABLE random (a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) engine=Memory;
|
||||
INSERT INTO random SELECT * FROM generateRandom() LIMIT 2;
|
||||
SELECT * FROM random;
|
||||
```
|
||||
|
||||
```text
|
||||
┌─a────────────────────────────┬────────────d─┬─c──────────────────────────────────────────────────────────────────┐
|
||||
│ [] │ 68091.8197 │ ('2037-10-02 12:44:23.368','039ecab7-81c2-45ee-208c-844e5c6c5652') │
|
||||
│ [8,-83,0,-22,65,9,-30,28,64] │ -186233.4909 │ ('2062-01-11 00:06:04.124','69563ea1-5ad1-f870-16d8-67061da0df25') │
|
||||
└──────────────────────────────┴──────────────┴────────────────────────────────────────────────────────────────────┘
|
||||
```
|
@ -131,27 +131,20 @@ LAYOUT(FLAT(INITIAL_ARRAY_SIZE 50000 MAX_ARRAY_SIZE 5000000))
|
||||
|
||||
Ключ словаря имеет тип [UInt64](../../../sql-reference/data-types/int-uint.md).
|
||||
|
||||
Если `preallocate` имеет значение `true` (по умолчанию `false`), хеш-таблица будет предварительно определена (это ускорит загрузку словаря). Используйте этот метод только в случае, если:
|
||||
|
||||
- Источник поддерживает произвольное количество элементов (пока поддерживается только источником `ClickHouse`).
|
||||
- В данных нет дубликатов (иначе это может увеличить объем используемой памяти хеш-таблицы).
|
||||
|
||||
Поддерживаются все виды источников. При обновлении данные (из файла, из таблицы) читаются целиком.
|
||||
|
||||
Пример конфигурации:
|
||||
|
||||
``` xml
|
||||
<layout>
|
||||
<hashed>
|
||||
<preallocate>0</preallocate>
|
||||
</hashed>
|
||||
<hashed/>
|
||||
</layout>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
``` sql
|
||||
LAYOUT(HASHED(PREALLOCATE 0))
|
||||
LAYOUT(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).
|
||||
|
||||
Для этого типа размещения также можно задать `preallocate` в значении `true`. В данном случае это более важно, чем для типа `hashed`.
|
||||
|
||||
Пример конфигурации:
|
||||
|
||||
``` xml
|
||||
@ -173,7 +164,7 @@ LAYOUT(HASHED(PREALLOCATE 0))
|
||||
или
|
||||
|
||||
``` sql
|
||||
LAYOUT(SPARSE_HASHED([PREALLOCATE 0]))
|
||||
LAYOUT(SPARSE_HASHED())
|
||||
```
|
||||
|
||||
### complex_key_hashed {#complex-key-hashed}
|
||||
|
@ -1,97 +1 @@
|
||||
## Generating ClickHouse documentation {#how-clickhouse-documentation-is-generated}
|
||||
|
||||
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 there’s 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.
|
||||
See https://github.com/ClickHouse/clickhouse-docs/blob/main/contrib-writing-guide.md
|
||||
|
@ -117,7 +117,7 @@ SELECT notEmpty([1,2]);
|
||||
|
||||
## range(end), range(\[start, \] end \[, step\]) {#range}
|
||||
|
||||
返回一个以`step`作为增量步长的从`start`到`end - 1`的`UInt`类型数字数组。
|
||||
返回一个以`step`作为增量步长的从`start`到`end - 1`的整形数字数组, 支持类型包括[`UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`](../data-types/int-uint.md)。
|
||||
|
||||
**语法**
|
||||
``` sql
|
||||
@ -126,31 +126,30 @@ range([start, ] end [, step])
|
||||
|
||||
**参数**
|
||||
|
||||
- `start` — 数组的第一个元素。可选项,如果设置了`step`时同样需要`start`,默认值为:0,类型为[UInt](../data-types/int-uint.md)。
|
||||
- `end` — 计数到`end`结束,但不包括`end`,必填项,类型为[UInt](../data-types/int-uint.md)。
|
||||
- `step` — 确定数组中每个元素之间的增量步长。可选项,默认值为:1,类型为[UInt](../data-types/int-uint.md)。
|
||||
- `start` — 数组的第一个元素。可选项,如果设置了`step`时同样需要`start`,默认值为:0。
|
||||
- `end` — 计数到`end`结束,但不包括`end`,必填项。
|
||||
- `step` — 确定数组中每个元素之间的增量步长。可选项,默认值为:1。
|
||||
|
||||
**返回值**
|
||||
|
||||
- 以`step`作为增量步长的从`start`到`end - 1`的`UInt`类型数字数组。
|
||||
- 以`step`作为增量步长的从`start`到`end - 1`的数字数组。
|
||||
|
||||
**注意事项**
|
||||
|
||||
- 所有参数必须是正值:`start`、`end`、`step`,类型均为`UInt`,结果数组的元素与此相同。
|
||||
- 所有参数`start`、`end`、`step`必须属于以下几种类型之一:[`UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`](../data-types/int-uint.md)。结果数组的元素数据类型为所有入参类型的最小超类,也必须属于以上几种类型之一。
|
||||
- 如果查询结果的数组总长度超过[function_range_max_elements_in_block](../../operations/settings/settings.md#settings-function_range_max_elements_in_block)指定的元素数,将会抛出异常。
|
||||
|
||||
|
||||
**示例**
|
||||
|
||||
查询语句:
|
||||
``` sql
|
||||
SELECT range(5), range(1, 5), range(1, 5, 2);
|
||||
SELECT range(5), range(1, 5), range(1, 5, 2), range(-1, 5, 2);
|
||||
```
|
||||
结果:
|
||||
```txt
|
||||
┌─range(5)────┬─range(1, 5)─┬─range(1, 5, 2)─┐
|
||||
│ [0,1,2,3,4] │ [1,2,3,4] │ [1,3] │
|
||||
└─────────────┴─────────────┴────────────────┘
|
||||
┌─range(5)────┬─range(1, 5)─┬─range(1, 5, 2)─┬─range(-1, 5, 2)─┐
|
||||
│ [0,1,2,3,4] │ [1,2,3,4] │ [1,3] │ [-1,1,3] │
|
||||
└─────────────┴─────────────┴────────────────┴─────────────────┘
|
||||
```
|
||||
|
||||
## array(x1, …), operator \[x1, …\] {#arrayx1-operator-x1}
|
||||
|
@ -140,6 +140,7 @@ namespace CurrentMetrics
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event MainConfigLoads;
|
||||
extern const Event ServerStartupMilliseconds;
|
||||
}
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
@ -652,6 +653,8 @@ static void sanityChecks(Server & server)
|
||||
int Server::main(const std::vector<std::string> & /*args*/)
|
||||
try
|
||||
{
|
||||
Stopwatch startup_watch;
|
||||
|
||||
Poco::Logger * log = &logger();
|
||||
|
||||
UseSSL use_ssl;
|
||||
@ -1822,6 +1825,9 @@ try
|
||||
LOG_INFO(log, "Ready for connections.");
|
||||
}
|
||||
|
||||
startup_watch.stop();
|
||||
ProfileEvents::increment(ProfileEvents::ServerStartupMilliseconds, startup_watch.elapsedMilliseconds());
|
||||
|
||||
try
|
||||
{
|
||||
global_context->startClusterDiscovery();
|
||||
|
@ -1073,6 +1073,9 @@
|
||||
|
||||
<!-- Interval of flushing data. -->
|
||||
<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>
|
||||
|
||||
<!-- Trace log. Stores stack traces collected by query profilers.
|
||||
|
@ -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,
|
||||
/// 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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -167,6 +167,7 @@ enum class AccessType
|
||||
M(SYSTEM_SYNC_REPLICA, "SYNC REPLICA", TABLE, SYSTEM) \
|
||||
M(SYSTEM_RESTART_REPLICA, "RESTART REPLICA", TABLE, SYSTEM) \
|
||||
M(SYSTEM_RESTORE_REPLICA, "RESTORE REPLICA", TABLE, SYSTEM) \
|
||||
M(SYSTEM_WAIT_LOADING_PARTS, "WAIT LOADING PARTS", TABLE, SYSTEM) \
|
||||
M(SYSTEM_SYNC_DATABASE_REPLICA, "SYNC DATABASE REPLICA", DATABASE, SYSTEM) \
|
||||
M(SYSTEM_SYNC_TRANSACTION_LOG, "SYNC TRANSACTION LOG", GLOBAL, SYSTEM) \
|
||||
M(SYSTEM_FLUSH_DISTRIBUTED, "FLUSH DISTRIBUTED", TABLE, SYSTEM_FLUSH) \
|
||||
|
@ -53,7 +53,7 @@ TEST(AccessRights, Union)
|
||||
"SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, "
|
||||
"SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, "
|
||||
"SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, "
|
||||
"SYSTEM RESTORE REPLICA, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*");
|
||||
"SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*");
|
||||
}
|
||||
|
||||
|
||||
|
@ -118,8 +118,7 @@ public:
|
||||
const auto * y_arg = arguments.at(1).get();
|
||||
|
||||
if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber())
|
||||
throw Exception("Illegal types of arguments of aggregate function " + getName() + ", must have number representation.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal types of arguments of aggregate function {}, must have number representation.", getName());
|
||||
}
|
||||
|
||||
bool allocatesMemoryInArena() const override { return false; }
|
||||
|
@ -207,7 +207,7 @@ private:
|
||||
{
|
||||
// Fuse points if their text representations differ only in last digit
|
||||
auto min_diff = 10 * (points[left].mean + points[right].mean) * std::numeric_limits<Mean>::epsilon();
|
||||
if (points[left].mean + min_diff >= points[right].mean)
|
||||
if (points[left].mean + std::fabs(min_diff) >= points[right].mean)
|
||||
{
|
||||
points[left] = points[left] + points[right];
|
||||
}
|
||||
|
@ -226,7 +226,7 @@ public:
|
||||
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
|
||||
{
|
||||
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);
|
||||
|
||||
|
@ -232,6 +232,9 @@ struct NameQuantilesExactInclusive { static constexpr auto name = "quantilesExac
|
||||
struct NameQuantileExactWeighted { static constexpr auto name = "quantileExactWeighted"; };
|
||||
struct NameQuantilesExactWeighted { static constexpr auto name = "quantilesExactWeighted"; };
|
||||
|
||||
struct NameQuantileInterpolatedWeighted { static constexpr auto name = "quantileInterpolatedWeighted"; };
|
||||
struct NameQuantilesInterpolatedWeighted { static constexpr auto name = "quantilesInterpolatedWeighted"; };
|
||||
|
||||
struct NameQuantileTiming { static constexpr auto name = "quantileTiming"; };
|
||||
struct NameQuantileTimingWeighted { static constexpr auto name = "quantileTimingWeighted"; };
|
||||
struct NameQuantilesTiming { static constexpr auto name = "quantilesTiming"; };
|
||||
|
@ -0,0 +1,70 @@
|
||||
#include <AggregateFunctions/AggregateFunctionQuantile.h>
|
||||
#include <AggregateFunctions/QuantileInterpolatedWeighted.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct Settings;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <typename Value, bool _> using FuncQuantileInterpolatedWeighted = AggregateFunctionQuantile<Value, QuantileInterpolatedWeighted<Value>, NameQuantileInterpolatedWeighted, true, void, false>;
|
||||
template <typename Value, bool _> using FuncQuantilesInterpolatedWeighted = AggregateFunctionQuantile<Value, QuantileInterpolatedWeighted<Value>, NameQuantilesInterpolatedWeighted, true, void, true>;
|
||||
|
||||
template <template <typename, bool> class Function>
|
||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
||||
{
|
||||
/// Second argument type check doesn't depend on the type of the first one.
|
||||
Function<void, true>::assertSecondArg(argument_types);
|
||||
|
||||
const DataTypePtr & argument_type = argument_types[0];
|
||||
WhichDataType which(argument_type);
|
||||
|
||||
#define DISPATCH(TYPE) \
|
||||
if (which.idx == TypeIndex::TYPE) return std::make_shared<Function<TYPE, true>>(argument_types, params);
|
||||
FOR_BASIC_NUMERIC_TYPES(DISPATCH)
|
||||
#undef DISPATCH
|
||||
if (which.idx == TypeIndex::Date) return std::make_shared<Function<DataTypeDate::FieldType, false>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::DateTime) return std::make_shared<Function<DataTypeDateTime::FieldType, false>>(argument_types, params);
|
||||
|
||||
if (which.idx == TypeIndex::Decimal32) return std::make_shared<Function<Decimal32, false>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::Decimal64) return std::make_shared<Function<Decimal64, false>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::Decimal128) return std::make_shared<Function<Decimal128, false>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::Decimal256) return std::make_shared<Function<Decimal256, false>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::DateTime64) return std::make_shared<Function<DateTime64, false>>(argument_types, params);
|
||||
|
||||
if (which.idx == TypeIndex::Int128) return std::make_shared<Function<Int128, true>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::UInt128) return std::make_shared<Function<UInt128, true>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::Int256) return std::make_shared<Function<Int256, true>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::UInt256) return std::make_shared<Function<UInt256, true>>(argument_types, params);
|
||||
|
||||
throw Exception("Illegal type " + argument_type->getName() + " of argument for aggregate function " + name,
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
}
|
||||
|
||||
void registerAggregateFunctionsQuantileInterpolatedWeighted(AggregateFunctionFactory & factory)
|
||||
{
|
||||
/// For aggregate functions returning array we cannot return NULL on empty set.
|
||||
AggregateFunctionProperties properties = { .returns_default_when_only_null = true };
|
||||
|
||||
factory.registerFunction(NameQuantileInterpolatedWeighted::name, createAggregateFunctionQuantile<FuncQuantileInterpolatedWeighted>);
|
||||
factory.registerFunction(NameQuantilesInterpolatedWeighted::name, { createAggregateFunctionQuantile<FuncQuantilesInterpolatedWeighted>, properties });
|
||||
|
||||
/// 'median' is an alias for 'quantile'
|
||||
factory.registerAlias("medianInterpolatedWeighted", NameQuantileInterpolatedWeighted::name);
|
||||
}
|
||||
|
||||
}
|
308
src/AggregateFunctions/QuantileInterpolatedWeighted.h
Normal file
308
src/AggregateFunctions/QuantileInterpolatedWeighted.h
Normal file
@ -0,0 +1,308 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct Settings;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
/** Approximates Quantile by:
|
||||
* - sorting input values and weights
|
||||
* - building a cumulative distribution based on weights
|
||||
* - performing linear interpolation between the weights and values
|
||||
*
|
||||
*/
|
||||
template <typename Value>
|
||||
struct QuantileInterpolatedWeighted
|
||||
{
|
||||
struct Int128Hash
|
||||
{
|
||||
size_t operator()(Int128 x) const
|
||||
{
|
||||
return CityHash_v1_0_2::Hash128to64({x >> 64, x & 0xffffffffffffffffll});
|
||||
}
|
||||
};
|
||||
|
||||
using Weight = UInt64;
|
||||
using UnderlyingType = NativeType<Value>;
|
||||
using Hasher = std::conditional_t<std::is_same_v<Value, Decimal128>, Int128Hash, HashCRC32<UnderlyingType>>;
|
||||
|
||||
/// When creating, the hash table must be small.
|
||||
using Map = HashMapWithStackMemory<UnderlyingType, Weight, Hasher, 4>;
|
||||
|
||||
Map map;
|
||||
|
||||
void add(const Value & x)
|
||||
{
|
||||
/// We must skip NaNs as they are not compatible with comparison sorting.
|
||||
if (!isNaN(x))
|
||||
++map[x];
|
||||
}
|
||||
|
||||
void add(const Value & x, Weight weight)
|
||||
{
|
||||
if (!isNaN(x))
|
||||
map[x] += weight;
|
||||
}
|
||||
|
||||
void merge(const QuantileInterpolatedWeighted & rhs)
|
||||
{
|
||||
for (const auto & pair : rhs.map)
|
||||
map[pair.getKey()] += pair.getMapped();
|
||||
}
|
||||
|
||||
void serialize(WriteBuffer & buf) const
|
||||
{
|
||||
map.write(buf);
|
||||
}
|
||||
|
||||
void deserialize(ReadBuffer & buf)
|
||||
{
|
||||
typename Map::Reader reader(buf);
|
||||
while (reader.next())
|
||||
{
|
||||
const auto & pair = reader.get();
|
||||
map[pair.first] = pair.second;
|
||||
}
|
||||
}
|
||||
|
||||
Value get(Float64 level) const
|
||||
{
|
||||
return getImpl<Value>(level);
|
||||
}
|
||||
|
||||
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result) const
|
||||
{
|
||||
getManyImpl<Value>(levels, indices, size, result);
|
||||
}
|
||||
|
||||
/// The same, but in the case of an empty state, NaN is returned.
|
||||
Float64 getFloat(Float64) const
|
||||
{
|
||||
throw Exception("Method getFloat is not implemented for QuantileInterpolatedWeighted", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void getManyFloat(const Float64 *, const size_t *, size_t, Float64 *) const
|
||||
{
|
||||
throw Exception("Method getManyFloat is not implemented for QuantileInterpolatedWeighted", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
private:
|
||||
using Pair = typename std::pair<UnderlyingType, Float64>;
|
||||
|
||||
/// Get the value of the `level` quantile. The level must be between 0 and 1.
|
||||
template <typename T>
|
||||
T getImpl(Float64 level) const
|
||||
{
|
||||
size_t size = map.size();
|
||||
|
||||
if (0 == size)
|
||||
return std::numeric_limits<Value>::quiet_NaN();
|
||||
|
||||
/// Maintain a vector of pair of values and weights for easier sorting and for building
|
||||
/// a cumulative distribution using the provided weights.
|
||||
std::vector<Pair> value_weight_pairs;
|
||||
value_weight_pairs.reserve(size);
|
||||
|
||||
/// Note: weight provided must be a 64-bit integer
|
||||
/// Float64 is used as accumulator here to get approximate results.
|
||||
/// But weight used in the internal array is stored as Float64 as we
|
||||
/// do some quantile estimation operation which involves division and
|
||||
/// require Float64 level of precision.
|
||||
|
||||
Float64 sum_weight = 0;
|
||||
for (const auto & pair : map)
|
||||
{
|
||||
sum_weight += pair.getMapped();
|
||||
auto value = pair.getKey();
|
||||
auto weight = pair.getMapped();
|
||||
value_weight_pairs.push_back({value, weight});
|
||||
}
|
||||
|
||||
::sort(value_weight_pairs.begin(), value_weight_pairs.end(), [](const Pair & a, const Pair & b) { return a.first < b.first; });
|
||||
|
||||
Float64 accumulated = 0;
|
||||
|
||||
/// vector for populating and storing the cumulative sum using the provided weights.
|
||||
/// example: [0,1,2,3,4,5] -> [0,1,3,6,10,15]
|
||||
std::vector<Float64> weights_cum_sum;
|
||||
weights_cum_sum.reserve(size);
|
||||
|
||||
for (size_t idx = 0; idx < size; ++idx)
|
||||
{
|
||||
accumulated += value_weight_pairs[idx].second;
|
||||
weights_cum_sum.push_back(accumulated);
|
||||
}
|
||||
|
||||
/// The following estimation of quantile is general and the idea is:
|
||||
/// https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method
|
||||
|
||||
/// calculates a simple cumulative distribution based on weights
|
||||
if (sum_weight != 0)
|
||||
{
|
||||
for (size_t idx = 0; idx < size; ++idx)
|
||||
value_weight_pairs[idx].second = (weights_cum_sum[idx] - 0.5 * value_weight_pairs[idx].second) / sum_weight;
|
||||
}
|
||||
|
||||
/// perform linear interpolation
|
||||
size_t idx = 0;
|
||||
if (size >= 2)
|
||||
{
|
||||
if (level >= value_weight_pairs[size - 2].second)
|
||||
{
|
||||
idx = size - 2;
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t start = 0, end = size - 1;
|
||||
while (start <= end)
|
||||
{
|
||||
size_t mid = start + (end - start) / 2;
|
||||
if (mid > size)
|
||||
break;
|
||||
if (level > value_weight_pairs[mid + 1].second)
|
||||
start = mid + 1;
|
||||
else
|
||||
{
|
||||
idx = mid;
|
||||
end = mid - 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
size_t l = idx;
|
||||
size_t u = idx + 1 < size ? idx + 1 : idx;
|
||||
|
||||
Float64 xl = value_weight_pairs[l].second, xr = value_weight_pairs[u].second;
|
||||
UnderlyingType yl = value_weight_pairs[l].first, yr = value_weight_pairs[u].first;
|
||||
|
||||
if (level < xl)
|
||||
yr = yl;
|
||||
if (level > xr)
|
||||
yl = yr;
|
||||
|
||||
return static_cast<T>(interpolate(level, xl, xr, yl, yr));
|
||||
}
|
||||
|
||||
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
|
||||
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
|
||||
template <typename T>
|
||||
void getManyImpl(const Float64 * levels, const size_t * indices, size_t num_levels, Value * result) const
|
||||
{
|
||||
size_t size = map.size();
|
||||
|
||||
if (0 == size)
|
||||
{
|
||||
for (size_t i = 0; i < num_levels; ++i)
|
||||
result[i] = Value();
|
||||
return;
|
||||
}
|
||||
|
||||
std::vector<Pair> value_weight_pairs;
|
||||
value_weight_pairs.reserve(size);
|
||||
|
||||
Float64 sum_weight = 0;
|
||||
for (const auto & pair : map)
|
||||
{
|
||||
sum_weight += pair.getMapped();
|
||||
auto value = pair.getKey();
|
||||
auto weight = pair.getMapped();
|
||||
value_weight_pairs.push_back({value, weight});
|
||||
}
|
||||
|
||||
::sort(value_weight_pairs.begin(), value_weight_pairs.end(), [](const Pair & a, const Pair & b) { return a.first < b.first; });
|
||||
|
||||
Float64 accumulated = 0;
|
||||
|
||||
/// vector for populating and storing the cumulative sum using the provided weights.
|
||||
/// example: [0,1,2,3,4,5] -> [0,1,3,6,10,15]
|
||||
std::vector<Float64> weights_cum_sum;
|
||||
weights_cum_sum.reserve(size);
|
||||
|
||||
for (size_t idx = 0; idx < size; ++idx)
|
||||
{
|
||||
accumulated += value_weight_pairs[idx].second;
|
||||
weights_cum_sum.emplace_back(accumulated);
|
||||
}
|
||||
|
||||
|
||||
/// The following estimation of quantile is general and the idea is:
|
||||
/// https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method
|
||||
|
||||
/// calculates a simple cumulative distribution based on weights
|
||||
if (sum_weight != 0)
|
||||
{
|
||||
for (size_t idx = 0; idx < size; ++idx)
|
||||
value_weight_pairs[idx].second = (weights_cum_sum[idx] - 0.5 * value_weight_pairs[idx].second) / sum_weight;
|
||||
}
|
||||
|
||||
for (size_t level_index = 0; level_index < num_levels; ++level_index)
|
||||
{
|
||||
/// perform linear interpolation for every level
|
||||
auto level = levels[indices[level_index]];
|
||||
|
||||
size_t idx = 0;
|
||||
if (size >= 2)
|
||||
{
|
||||
if (level >= value_weight_pairs[size - 2].second)
|
||||
{
|
||||
idx = size - 2;
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t start = 0, end = size - 1;
|
||||
while (start <= end)
|
||||
{
|
||||
size_t mid = start + (end - start) / 2;
|
||||
if (mid > size)
|
||||
break;
|
||||
if (level > value_weight_pairs[mid + 1].second)
|
||||
start = mid + 1;
|
||||
else
|
||||
{
|
||||
idx = mid;
|
||||
end = mid - 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
size_t l = idx;
|
||||
size_t u = idx + 1 < size ? idx + 1 : idx;
|
||||
|
||||
Float64 xl = value_weight_pairs[l].second, xr = value_weight_pairs[u].second;
|
||||
UnderlyingType yl = value_weight_pairs[l].first, yr = value_weight_pairs[u].first;
|
||||
|
||||
if (level < xl)
|
||||
yr = yl;
|
||||
if (level > xr)
|
||||
yl = yr;
|
||||
|
||||
result[indices[level_index]] = static_cast<T>(interpolate(level, xl, xr, yl, yr));
|
||||
}
|
||||
}
|
||||
|
||||
/// This ignores overflows or NaN's that might arise during add, sub and mul operations and doesn't aim to provide exact
|
||||
/// results since `the quantileInterpolatedWeighted` function itself relies mainly on approximation.
|
||||
UnderlyingType NO_SANITIZE_UNDEFINED interpolate(Float64 level, Float64 xl, Float64 xr, UnderlyingType yl, UnderlyingType yr) const
|
||||
{
|
||||
UnderlyingType dy = yr - yl;
|
||||
Float64 dx = xr - xl;
|
||||
dx = dx == 0 ? 1 : dx; /// to handle NaN behavior that might arise during integer division below.
|
||||
|
||||
/// yl + (dy / dx) * (level - xl)
|
||||
return static_cast<UnderlyingType>(yl + (dy / dx) * (level - xl));
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -21,6 +21,7 @@ void registerAggregateFunctionsQuantile(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileExactWeighted(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileInterpolatedWeighted(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileExactLow(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileExactHigh(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileExactInclusive(AggregateFunctionFactory &);
|
||||
@ -106,6 +107,7 @@ void registerAggregateFunctions()
|
||||
registerAggregateFunctionsQuantileDeterministic(factory);
|
||||
registerAggregateFunctionsQuantileExact(factory);
|
||||
registerAggregateFunctionsQuantileExactWeighted(factory);
|
||||
registerAggregateFunctionsQuantileInterpolatedWeighted(factory);
|
||||
registerAggregateFunctionsQuantileExactLow(factory);
|
||||
registerAggregateFunctionsQuantileExactHigh(factory);
|
||||
registerAggregateFunctionsQuantileExactInclusive(factory);
|
||||
|
@ -48,7 +48,7 @@ void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Analyzer/ConstantNode.h>
|
||||
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
@ -31,6 +32,15 @@ FunctionNode::FunctionNode(String function_name_)
|
||||
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
|
||||
{
|
||||
const auto & arguments = getArguments().getNodes();
|
||||
|
@ -85,6 +85,7 @@ public:
|
||||
/// Get arguments node
|
||||
QueryTreeNodePtr & getArgumentsNode() { return children[arguments_child_index]; }
|
||||
|
||||
const DataTypes & getArgumentTypes() const;
|
||||
ColumnsWithTypeAndName getArgumentColumns() const;
|
||||
|
||||
/// Returns true if function node has window, false otherwise
|
||||
@ -144,6 +145,11 @@ public:
|
||||
*/
|
||||
void resolveAsFunction(FunctionBasePtr function_value);
|
||||
|
||||
void resolveAsFunction(const FunctionOverloadResolverPtr & resolver)
|
||||
{
|
||||
resolveAsFunction(resolver->build(getArgumentColumns()));
|
||||
}
|
||||
|
||||
/** Resolve function node as aggregate function.
|
||||
* It is important that function name is updated with resolved function name.
|
||||
* Main motivation for this is query tree optimizations.
|
||||
|
@ -214,6 +214,11 @@ IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const
|
||||
}
|
||||
|
||||
QueryTreeNodePtr IQueryTreeNode::clone() const
|
||||
{
|
||||
return cloneAndReplace({});
|
||||
}
|
||||
|
||||
QueryTreeNodePtr IQueryTreeNode::cloneAndReplace(const ReplacementMap & replacement_map) const
|
||||
{
|
||||
/** Clone tree with this node as root.
|
||||
*
|
||||
@ -236,11 +241,11 @@ QueryTreeNodePtr IQueryTreeNode::clone() const
|
||||
const auto [node_to_clone, place_for_cloned_node] = nodes_to_clone.back();
|
||||
nodes_to_clone.pop_back();
|
||||
|
||||
auto node_clone = node_to_clone->cloneImpl();
|
||||
auto it = replacement_map.find(node_to_clone);
|
||||
auto node_clone = it != replacement_map.end() ? it->second : node_to_clone->cloneImpl();
|
||||
*place_for_cloned_node = node_clone;
|
||||
|
||||
node_clone->setAlias(node_to_clone->alias);
|
||||
node_clone->setOriginalAST(node_to_clone->original_ast);
|
||||
node_clone->children = node_to_clone->children;
|
||||
node_clone->weak_pointers = node_to_clone->weak_pointers;
|
||||
|
||||
|
@ -110,6 +110,13 @@ public:
|
||||
/// Get a deep copy of the query tree
|
||||
QueryTreeNodePtr clone() const;
|
||||
|
||||
/** Get a deep copy of the query tree.
|
||||
* If node to clone is key in replacement map, then instead of clone it
|
||||
* use value node from replacement map.
|
||||
*/
|
||||
using ReplacementMap = std::unordered_map<const IQueryTreeNode *, QueryTreeNodePtr>;
|
||||
QueryTreeNodePtr cloneAndReplace(const ReplacementMap & replacement_map) const;
|
||||
|
||||
/// Returns true if node has alias, false otherwise
|
||||
bool hasAlias() const
|
||||
{
|
||||
|
@ -1,8 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <optional>
|
||||
#include <utility>
|
||||
#include <Common/SettingsChanges.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/QueryNode.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -16,6 +16,8 @@ using ListNodePtr = std::shared_ptr<ListNode>;
|
||||
class ListNode final : public IQueryTreeNode
|
||||
{
|
||||
public:
|
||||
using iterator = QueryTreeNodes::iterator;
|
||||
|
||||
/// Initialize list node with empty nodes
|
||||
ListNode();
|
||||
|
||||
@ -41,6 +43,9 @@ public:
|
||||
|
||||
void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override;
|
||||
|
||||
iterator begin() { return children.begin(); }
|
||||
iterator end() { return children.end(); }
|
||||
|
||||
protected:
|
||||
bool isEqualImpl(const IQueryTreeNode & rhs) const override;
|
||||
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||
#include <Parsers/ASTColumnsMatcher.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTColumnsTransformers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -206,19 +207,43 @@ QueryTreeNodePtr MatcherNode::cloneImpl() const
|
||||
ASTPtr MatcherNode::toASTImpl() const
|
||||
{
|
||||
ASTPtr result;
|
||||
ASTPtr transformers;
|
||||
|
||||
if (!children.empty())
|
||||
{
|
||||
transformers = std::make_shared<ASTColumnsTransformerList>();
|
||||
|
||||
for (const auto & child : children)
|
||||
transformers->children.push_back(child->toAST());
|
||||
}
|
||||
|
||||
if (matcher_type == MatcherNodeType::ASTERISK)
|
||||
{
|
||||
if (qualified_identifier.empty())
|
||||
{
|
||||
result = std::make_shared<ASTAsterisk>();
|
||||
auto asterisk = std::make_shared<ASTAsterisk>();
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
asterisk->transformers = std::move(transformers);
|
||||
asterisk->children.push_back(asterisk->transformers);
|
||||
}
|
||||
|
||||
result = asterisk;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto qualified_asterisk = std::make_shared<ASTQualifiedAsterisk>();
|
||||
|
||||
auto identifier_parts = qualified_identifier.getParts();
|
||||
qualified_asterisk->children.push_back(std::make_shared<ASTIdentifier>(std::move(identifier_parts)));
|
||||
qualified_asterisk->qualifier = std::make_shared<ASTIdentifier>(std::move(identifier_parts));
|
||||
qualified_asterisk->children.push_back(qualified_asterisk->qualifier);
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
qualified_asterisk->transformers = std::move(transformers);
|
||||
qualified_asterisk->children.push_back(qualified_asterisk->transformers);
|
||||
}
|
||||
|
||||
result = qualified_asterisk;
|
||||
}
|
||||
@ -229,6 +254,13 @@ ASTPtr MatcherNode::toASTImpl() const
|
||||
{
|
||||
auto regexp_matcher = std::make_shared<ASTColumnsRegexpMatcher>();
|
||||
regexp_matcher->setPattern(columns_matcher->pattern());
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
regexp_matcher->transformers = std::move(transformers);
|
||||
regexp_matcher->children.push_back(regexp_matcher->transformers);
|
||||
}
|
||||
|
||||
result = regexp_matcher;
|
||||
}
|
||||
else
|
||||
@ -237,7 +269,14 @@ ASTPtr MatcherNode::toASTImpl() const
|
||||
regexp_matcher->setPattern(columns_matcher->pattern());
|
||||
|
||||
auto identifier_parts = qualified_identifier.getParts();
|
||||
regexp_matcher->children.push_back(std::make_shared<ASTIdentifier>(std::move(identifier_parts)));
|
||||
regexp_matcher->qualifier = std::make_shared<ASTIdentifier>(std::move(identifier_parts));
|
||||
regexp_matcher->children.push_back(regexp_matcher->qualifier);
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
regexp_matcher->transformers = std::move(transformers);
|
||||
regexp_matcher->children.push_back(regexp_matcher->transformers);
|
||||
}
|
||||
|
||||
result = regexp_matcher;
|
||||
}
|
||||
@ -257,23 +296,36 @@ ASTPtr MatcherNode::toASTImpl() const
|
||||
{
|
||||
auto columns_list_matcher = std::make_shared<ASTColumnsListMatcher>();
|
||||
columns_list_matcher->column_list = std::move(column_list);
|
||||
columns_list_matcher->children.push_back(columns_list_matcher->column_list);
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
columns_list_matcher->transformers = std::move(transformers);
|
||||
columns_list_matcher->children.push_back(columns_list_matcher->transformers);
|
||||
}
|
||||
|
||||
result = columns_list_matcher;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto columns_list_matcher = std::make_shared<ASTQualifiedColumnsListMatcher>();
|
||||
columns_list_matcher->column_list = std::move(column_list);
|
||||
|
||||
auto identifier_parts = qualified_identifier.getParts();
|
||||
columns_list_matcher->children.push_back(std::make_shared<ASTIdentifier>(std::move(identifier_parts)));
|
||||
columns_list_matcher->qualifier = std::make_shared<ASTIdentifier>(std::move(identifier_parts));
|
||||
columns_list_matcher->column_list = std::move(column_list);
|
||||
columns_list_matcher->children.push_back(columns_list_matcher->qualifier);
|
||||
columns_list_matcher->children.push_back(columns_list_matcher->column_list);
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
columns_list_matcher->transformers = std::move(transformers);
|
||||
columns_list_matcher->children.push_back(columns_list_matcher->transformers);
|
||||
}
|
||||
|
||||
result = columns_list_matcher;
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto & child : children)
|
||||
result->children.push_back(child->toAST());
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
@ -47,19 +48,23 @@ Field zeroField(const Field & value)
|
||||
class AggregateFunctionsArithmericOperationsVisitor : public InDepthQueryTreeVisitor<AggregateFunctionsArithmericOperationsVisitor>
|
||||
{
|
||||
public:
|
||||
explicit AggregateFunctionsArithmericOperationsVisitor(ContextPtr context_)
|
||||
: context(std::move(context_))
|
||||
{}
|
||||
|
||||
/// Traverse tree bottom to top
|
||||
static bool shouldTraverseTopToBottom()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
static void visitImpl(QueryTreeNodePtr & node)
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
auto * aggregate_function_node = node->as<FunctionNode>();
|
||||
if (!aggregate_function_node || !aggregate_function_node->isAggregateFunction())
|
||||
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"}},
|
||||
{"min", {"multiply", "divide", "plus", "minus"}},
|
||||
{"max", {"multiply", "divide", "plus", "minus"}},
|
||||
@ -69,85 +74,112 @@ public:
|
||||
if (aggregate_function_arguments_nodes.size() != 1)
|
||||
return;
|
||||
|
||||
auto * inner_function_node = aggregate_function_arguments_nodes[0]->as<FunctionNode>();
|
||||
if (!inner_function_node)
|
||||
const auto & arithmetic_function_node = aggregate_function_arguments_nodes[0];
|
||||
auto * arithmetic_function_node_typed = arithmetic_function_node->as<FunctionNode>();
|
||||
if (!arithmetic_function_node_typed)
|
||||
return;
|
||||
|
||||
auto & inner_function_arguments_nodes = inner_function_node->getArguments().getNodes();
|
||||
if (inner_function_arguments_nodes.size() != 2)
|
||||
const auto & arithmetic_function_arguments_nodes = arithmetic_function_node_typed->getArguments().getNodes();
|
||||
if (arithmetic_function_arguments_nodes.size() != 2)
|
||||
return;
|
||||
|
||||
/// 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);
|
||||
if (supported_function_it == supported_functions.end())
|
||||
auto supported_aggregate_function_it = supported_aggregate_functions.find(lower_aggregate_function_name);
|
||||
if (supported_aggregate_function_it == supported_aggregate_functions.end())
|
||||
return;
|
||||
|
||||
const auto & inner_function_name = inner_function_node->getFunctionName();
|
||||
|
||||
if (!supported_function_it->second.contains(inner_function_name))
|
||||
const auto & arithmetic_function_name = arithmetic_function_node_typed->getFunctionName();
|
||||
if (!supported_aggregate_function_it->second.contains(arithmetic_function_name))
|
||||
return;
|
||||
|
||||
const auto * left_argument_constant_node = inner_function_arguments_nodes[0]->as<ConstantNode>();
|
||||
const auto * right_argument_constant_node = inner_function_arguments_nodes[1]->as<ConstantNode>();
|
||||
const auto * left_argument_constant_node = arithmetic_function_arguments_nodes[0]->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.
|
||||
*
|
||||
* Example: SELECT min(-1 * id);
|
||||
* Result: SELECT -1 * max(id);
|
||||
*/
|
||||
std::string function_name_if_constant_is_negative;
|
||||
if (inner_function_name == "multiply" || inner_function_name == "divide")
|
||||
std::string aggregate_function_name_if_constant_is_negative;
|
||||
if (arithmetic_function_name == "multiply" || arithmetic_function_name == "divide")
|
||||
{
|
||||
if (lower_function_name == "min")
|
||||
function_name_if_constant_is_negative = "max";
|
||||
else if (lower_function_name == "max")
|
||||
function_name_if_constant_is_negative = "min";
|
||||
if (lower_aggregate_function_name == "min")
|
||||
aggregate_function_name_if_constant_is_negative = "max";
|
||||
else if (lower_aggregate_function_name == "max")
|
||||
aggregate_function_name_if_constant_is_negative = "min";
|
||||
}
|
||||
|
||||
size_t arithmetic_function_argument_index = 0;
|
||||
|
||||
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
|
||||
if (inner_function_name == "divide")
|
||||
if (arithmetic_function_name == "divide")
|
||||
return;
|
||||
|
||||
/// 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();
|
||||
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))
|
||||
{
|
||||
lower_function_name = function_name_if_constant_is_negative;
|
||||
lower_aggregate_function_name = aggregate_function_name_if_constant_is_negative;
|
||||
}
|
||||
resolveAggregateFunctionNode(*aggregate_function_node, inner_function_arguments_nodes[1], lower_function_name);
|
||||
|
||||
auto inner_function = aggregate_function_arguments_nodes[0];
|
||||
auto inner_function_right_argument = std::move(inner_function_arguments_nodes[1]);
|
||||
aggregate_function_arguments_nodes = {inner_function_right_argument};
|
||||
inner_function_arguments_nodes[1] = node;
|
||||
node = std::move(inner_function);
|
||||
arithmetic_function_argument_index = 1;
|
||||
}
|
||||
else if (right_argument_constant_node)
|
||||
{
|
||||
/// 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();
|
||||
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))
|
||||
{
|
||||
lower_function_name = function_name_if_constant_is_negative;
|
||||
lower_aggregate_function_name = aggregate_function_name_if_constant_is_negative;
|
||||
}
|
||||
resolveAggregateFunctionNode(*aggregate_function_node, inner_function_arguments_nodes[0], function_name_if_constant_is_negative);
|
||||
|
||||
auto inner_function = aggregate_function_arguments_nodes[0];
|
||||
auto inner_function_left_argument = std::move(inner_function_arguments_nodes[0]);
|
||||
aggregate_function_arguments_nodes = {inner_function_left_argument};
|
||||
inner_function_arguments_nodes[0] = node;
|
||||
node = std::move(inner_function);
|
||||
arithmetic_function_argument_index = 0;
|
||||
}
|
||||
|
||||
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:
|
||||
static inline void resolveAggregateFunctionNode(FunctionNode & function_node, QueryTreeNodePtr & argument, const String & aggregate_function_name)
|
||||
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)
|
||||
{
|
||||
auto function_aggregate_function = function_node.getAggregateFunction();
|
||||
|
||||
@ -159,13 +191,15 @@ private:
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
|
134
src/Analyzer/Passes/ConvertOrLikeChainPass.cpp
Normal file
134
src/Analyzer/Passes/ConvertOrLikeChainPass.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
20
src/Analyzer/Passes/ConvertOrLikeChainPass.h
Normal file
20
src/Analyzer/Passes/ConvertOrLikeChainPass.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
253
src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp
Normal file
253
src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
||||
|
31
src/Analyzer/Passes/GroupingFunctionsResolvePass.h
Normal file
31
src/Analyzer/Passes/GroupingFunctionsResolvePass.h
Normal 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;
|
||||
|
||||
};
|
||||
|
||||
}
|
113
src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp
Normal file
113
src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
22
src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.h
Normal file
22
src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
124
src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp
Normal file
124
src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp
Normal file
@ -0,0 +1,124 @@
|
||||
#include <Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.h>
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/HashUtils.h>
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/QueryNode.h>
|
||||
#include <Analyzer/SortNode.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class OptimizeRedundantFunctionsInOrderByVisitor : public InDepthQueryTreeVisitor<OptimizeRedundantFunctionsInOrderByVisitor>
|
||||
{
|
||||
public:
|
||||
static bool needChildVisit(QueryTreeNodePtr & node, QueryTreeNodePtr & /*parent*/)
|
||||
{
|
||||
if (node->as<FunctionNode>())
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
auto * query = node->as<QueryNode>();
|
||||
if (!query)
|
||||
return;
|
||||
|
||||
if (!query->hasOrderBy())
|
||||
return;
|
||||
|
||||
auto & order_by = query->getOrderBy();
|
||||
for (auto & elem : order_by.getNodes())
|
||||
{
|
||||
auto * order_by_elem = elem->as<SortNode>();
|
||||
if (order_by_elem->withFill())
|
||||
return;
|
||||
}
|
||||
|
||||
QueryTreeNodes new_order_by_nodes;
|
||||
new_order_by_nodes.reserve(order_by.getNodes().size());
|
||||
|
||||
for (auto & elem : order_by.getNodes())
|
||||
{
|
||||
auto & order_by_expr = elem->as<SortNode>()->getExpression();
|
||||
switch (order_by_expr->getNodeType())
|
||||
{
|
||||
case QueryTreeNodeType::FUNCTION:
|
||||
{
|
||||
if (isRedundantExpression(order_by_expr))
|
||||
continue;
|
||||
break;
|
||||
}
|
||||
case QueryTreeNodeType::COLUMN:
|
||||
{
|
||||
existing_keys.insert(order_by_expr);
|
||||
break;
|
||||
}
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
new_order_by_nodes.push_back(elem);
|
||||
}
|
||||
existing_keys.clear();
|
||||
|
||||
if (new_order_by_nodes.size() < order_by.getNodes().size())
|
||||
order_by.getNodes() = std::move(new_order_by_nodes);
|
||||
}
|
||||
|
||||
private:
|
||||
QueryTreeNodePtrWithHashSet existing_keys;
|
||||
|
||||
bool isRedundantExpression(QueryTreeNodePtr function)
|
||||
{
|
||||
QueryTreeNodes nodes_to_process{ function };
|
||||
while (!nodes_to_process.empty())
|
||||
{
|
||||
auto node = nodes_to_process.back();
|
||||
nodes_to_process.pop_back();
|
||||
|
||||
// TODO: handle constants here
|
||||
switch (node->getNodeType())
|
||||
{
|
||||
case QueryTreeNodeType::FUNCTION:
|
||||
{
|
||||
auto * function_node = node->as<FunctionNode>();
|
||||
const auto & function_arguments = function_node->getArguments().getNodes();
|
||||
if (function_arguments.empty())
|
||||
return false;
|
||||
const auto & function_base = function_node->getFunction();
|
||||
if (!function_base || !function_base->isDeterministicInScopeOfQuery())
|
||||
return false;
|
||||
|
||||
// Process arguments in order
|
||||
for (auto it = function_arguments.rbegin(); it != function_arguments.rend(); ++it)
|
||||
nodes_to_process.push_back(*it);
|
||||
break;
|
||||
}
|
||||
case QueryTreeNodeType::COLUMN:
|
||||
{
|
||||
if (!existing_keys.contains(node))
|
||||
return false;
|
||||
break;
|
||||
}
|
||||
default:
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
void OptimizeRedundantFunctionsInOrderByPass::run(QueryTreeNodePtr query_tree_node, ContextPtr /*context*/)
|
||||
{
|
||||
OptimizeRedundantFunctionsInOrderByVisitor().visit(query_tree_node);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
#include <Analyzer/IQueryTreePass.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** If ORDER BY has argument x followed by f(x) transforms it to ORDER BY x.
|
||||
* Optimize ORDER BY x, y, f(x), g(x, y), f(h(x)), t(f(x), g(x)) into ORDER BY x, y
|
||||
* in case if f(), g(), h(), t() are deterministic (in scope of query).
|
||||
* Don't optimize ORDER BY f(x), g(x), x even if f(x) is bijection for x or g(x).
|
||||
*/
|
||||
class OptimizeRedundantFunctionsInOrderByPass final : public IQueryTreePass
|
||||
{
|
||||
public:
|
||||
String getName() override { return "OptimizeRedundantFunctionsInOrderBy"; }
|
||||
|
||||
String getDescription() override { return "If ORDER BY has argument x followed by f(x) transforms it to ORDER BY x."; }
|
||||
|
||||
void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
|
||||
};
|
||||
|
||||
}
|
@ -1,6 +1,7 @@
|
||||
#include <Analyzer/Passes/QueryAnalysisPass.h>
|
||||
|
||||
#include <Common/NamePrompter.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
@ -66,6 +67,14 @@
|
||||
#include <Analyzer/UnionNode.h>
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.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>
|
||||
|
||||
@ -1049,6 +1058,8 @@ private:
|
||||
|
||||
static bool isTableExpressionNodeType(QueryTreeNodeType node_type);
|
||||
|
||||
static DataTypePtr getExpressionNodeResultTypeOrNull(const QueryTreeNodePtr & query_tree_node);
|
||||
|
||||
static ProjectionName calculateFunctionProjectionName(const QueryTreeNodePtr & function_node,
|
||||
const ProjectionNames & parameters_projection_names,
|
||||
const ProjectionNames & arguments_projection_names);
|
||||
@ -1097,7 +1108,7 @@ private:
|
||||
|
||||
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);
|
||||
|
||||
@ -1207,6 +1218,9 @@ private:
|
||||
/// Global resolve expression node to projection names map
|
||||
std::unordered_map<QueryTreeNodePtr, ProjectionNames> resolved_expressions;
|
||||
|
||||
/// Results of scalar sub queries
|
||||
std::unordered_map<QueryTreeNodeConstRawPtrWithHash, std::shared_ptr<ConstantValue>> scalars;
|
||||
|
||||
};
|
||||
|
||||
/// Utility functions implementation
|
||||
@ -1229,6 +1243,34 @@ bool QueryAnalyzer::isTableExpressionNodeType(QueryTreeNodeType node_type)
|
||||
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,
|
||||
const ProjectionNames & arguments_projection_names)
|
||||
{
|
||||
@ -1534,12 +1576,12 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection(
|
||||
auto expression_identifier = Identifier(name);
|
||||
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,
|
||||
expression->getResultType(),
|
||||
result_type,
|
||||
expression_identifier,
|
||||
valid_identifiers_result);
|
||||
}
|
||||
@ -1571,21 +1613,23 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection(
|
||||
|
||||
for (const auto & [argument_name, expression] : scope.expression_argument_name_to_node)
|
||||
{
|
||||
assert(expression);
|
||||
auto expression_node_type = expression->getNodeType();
|
||||
|
||||
if (allow_expression_identifiers && isExpressionNodeType(expression_node_type))
|
||||
{
|
||||
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,
|
||||
expression->getResultType(),
|
||||
result_type,
|
||||
expression_identifier,
|
||||
valid_identifiers_result);
|
||||
}
|
||||
|
||||
valid_identifiers_result.insert(expression_identifier);
|
||||
}
|
||||
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->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);
|
||||
|
||||
Settings subquery_settings = context->getSettings();
|
||||
@ -1695,14 +1749,15 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size
|
||||
subquery_context->setSettings(subquery_settings);
|
||||
|
||||
auto options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth, true /*is_subquery*/);
|
||||
auto interpreter = std::make_unique<InterpreterSelectQueryAnalyzer>(node, options, subquery_context);
|
||||
auto interpreter = std::make_unique<InterpreterSelectQueryAnalyzer>(node, subquery_context, options);
|
||||
|
||||
auto io = interpreter->execute();
|
||||
|
||||
Block block;
|
||||
PullingAsyncPipelineExecutor executor(io.pipeline);
|
||||
io.pipeline.setProgressCallback(context->getProgressCallback());
|
||||
|
||||
Block block;
|
||||
|
||||
while (block.rows() == 0 && executor.pull(block))
|
||||
{
|
||||
}
|
||||
@ -1743,7 +1798,6 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size
|
||||
block = materializeBlock(block);
|
||||
size_t columns = block.columns();
|
||||
|
||||
// Block scalar;
|
||||
Field scalar_value;
|
||||
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));
|
||||
scalars[node.get()] = constant_value;
|
||||
node = std::make_shared<ConstantNode>(std::move(constant_value), node);
|
||||
}
|
||||
|
||||
@ -2027,7 +2082,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con
|
||||
auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
|
||||
auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context);
|
||||
|
||||
return std::make_shared<TableNode>(std::move(storage), storage_lock, storage_snapshot);
|
||||
return std::make_shared<TableNode>(std::move(storage), std::move(storage_lock), std::move(storage_snapshot));
|
||||
}
|
||||
|
||||
/// Resolve identifier from compound expression
|
||||
@ -4297,7 +4352,8 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
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_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;
|
||||
}
|
||||
}
|
||||
|
@ -77,11 +77,11 @@ public:
|
||||
if (!nested_function || nested_function->getFunctionName() != "if")
|
||||
return;
|
||||
|
||||
auto & nested_if_function_arguments_nodes = nested_function->getArguments().getNodes();
|
||||
const auto & nested_if_function_arguments_nodes = nested_function->getArguments().getNodes();
|
||||
if (nested_if_function_arguments_nodes.size() != 3)
|
||||
return;
|
||||
|
||||
auto & cond_argument = nested_if_function_arguments_nodes[0];
|
||||
const auto & cond_argument = nested_if_function_arguments_nodes[0];
|
||||
const auto * if_true_condition_constant_node = nested_if_function_arguments_nodes[1]->as<ConstantNode>();
|
||||
const auto * if_false_condition_constant_node = nested_if_function_arguments_nodes[2]->as<ConstantNode>();
|
||||
|
||||
@ -101,7 +101,7 @@ public:
|
||||
/// Rewrite `sum(if(cond, 1, 0))` into `countIf(cond)`.
|
||||
if (if_true_condition_value == 1 && if_false_condition_value == 0)
|
||||
{
|
||||
function_node_arguments_nodes[0] = std::move(nested_if_function_arguments_nodes[0]);
|
||||
function_node_arguments_nodes[0] = nested_if_function_arguments_nodes[0];
|
||||
function_node_arguments_nodes.resize(1);
|
||||
|
||||
resolveAsCountIfAggregateFunction(*function_node, function_node_arguments_nodes[0]->getResultType());
|
||||
@ -120,7 +120,7 @@ public:
|
||||
auto not_function = std::make_shared<FunctionNode>("not");
|
||||
|
||||
auto & not_function_arguments = not_function->getArguments().getNodes();
|
||||
not_function_arguments.push_back(std::move(nested_if_function_arguments_nodes[0]));
|
||||
not_function_arguments.push_back(nested_if_function_arguments_nodes[0]);
|
||||
|
||||
not_function->resolveAsFunction(FunctionFactory::instance().get("not", context)->build(not_function->getArgumentColumns()));
|
||||
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <fmt/core.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -179,6 +180,16 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s
|
||||
buffer << '\n' << std::string(indent + 2, ' ') << "OFFSET\n";
|
||||
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
|
||||
|
@ -111,7 +111,7 @@ private:
|
||||
|
||||
QueryTreeNodePtr buildJoinTree(const ASTPtr & tables_in_select_query, const ContextPtr & context) const;
|
||||
|
||||
ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index, const ContextPtr & context) const;
|
||||
ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const;
|
||||
|
||||
ASTPtr query;
|
||||
QueryTreeNodePtr query_tree_node;
|
||||
@ -439,13 +439,13 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
|
||||
}
|
||||
else if (const auto * asterisk = expression->as<ASTAsterisk>())
|
||||
{
|
||||
auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context);
|
||||
auto column_transformers = buildColumnTransformers(asterisk->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(std::move(column_transformers));
|
||||
}
|
||||
else if (const auto * qualified_asterisk = expression->as<ASTQualifiedAsterisk>())
|
||||
{
|
||||
auto & qualified_identifier = qualified_asterisk->children.at(0)->as<ASTTableIdentifier &>();
|
||||
auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context);
|
||||
auto & qualified_identifier = qualified_asterisk->qualifier->as<ASTIdentifier &>();
|
||||
auto column_transformers = buildColumnTransformers(qualified_asterisk->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), std::move(column_transformers));
|
||||
}
|
||||
else if (const auto * ast_literal = expression->as<ASTLiteral>())
|
||||
@ -543,7 +543,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
|
||||
}
|
||||
else if (const auto * columns_regexp_matcher = expression->as<ASTColumnsRegexpMatcher>())
|
||||
{
|
||||
auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context);
|
||||
auto column_transformers = buildColumnTransformers(columns_regexp_matcher->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(columns_regexp_matcher->getMatcher(), std::move(column_transformers));
|
||||
}
|
||||
else if (const auto * columns_list_matcher = expression->as<ASTColumnsListMatcher>())
|
||||
@ -557,18 +557,18 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
|
||||
column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts});
|
||||
}
|
||||
|
||||
auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context);
|
||||
auto column_transformers = buildColumnTransformers(columns_list_matcher->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(std::move(column_list_identifiers), std::move(column_transformers));
|
||||
}
|
||||
else if (const auto * qualified_columns_regexp_matcher = expression->as<ASTQualifiedColumnsRegexpMatcher>())
|
||||
{
|
||||
auto & qualified_identifier = qualified_columns_regexp_matcher->children.at(0)->as<ASTTableIdentifier &>();
|
||||
auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context);
|
||||
auto & qualified_identifier = qualified_columns_regexp_matcher->qualifier->as<ASTIdentifier &>();
|
||||
auto column_transformers = buildColumnTransformers(qualified_columns_regexp_matcher->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), qualified_columns_regexp_matcher->getMatcher(), std::move(column_transformers));
|
||||
}
|
||||
else if (const auto * qualified_columns_list_matcher = expression->as<ASTQualifiedColumnsListMatcher>())
|
||||
{
|
||||
auto & qualified_identifier = qualified_columns_list_matcher->children.at(0)->as<ASTTableIdentifier &>();
|
||||
auto & qualified_identifier = qualified_columns_list_matcher->qualifier->as<ASTIdentifier &>();
|
||||
|
||||
Identifiers column_list_identifiers;
|
||||
column_list_identifiers.reserve(qualified_columns_list_matcher->column_list->children.size());
|
||||
@ -579,7 +579,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
|
||||
column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts});
|
||||
}
|
||||
|
||||
auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context);
|
||||
auto column_transformers = buildColumnTransformers(qualified_columns_list_matcher->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), std::move(column_list_identifiers), std::move(column_transformers));
|
||||
}
|
||||
else
|
||||
@ -833,15 +833,15 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select
|
||||
}
|
||||
|
||||
|
||||
ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index, const ContextPtr & context) const
|
||||
ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const
|
||||
{
|
||||
ColumnTransformersNodes column_transformers;
|
||||
size_t children_size = matcher_expression->children.size();
|
||||
|
||||
for (; start_child_index < children_size; ++start_child_index)
|
||||
if (!matcher_expression)
|
||||
return column_transformers;
|
||||
|
||||
for (const auto & child : matcher_expression->children)
|
||||
{
|
||||
const auto & child = matcher_expression->children[start_child_index];
|
||||
|
||||
if (auto * apply_transformer = child->as<ASTColumnsApplyTransformer>())
|
||||
{
|
||||
if (apply_transformer->lambda)
|
||||
|
@ -1,5 +1,19 @@
|
||||
#include <memory>
|
||||
#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/CountDistinctPass.h>
|
||||
#include <Analyzer/Passes/FunctionToSubcolumnsPass.h>
|
||||
@ -14,16 +28,11 @@
|
||||
#include <Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h>
|
||||
#include <Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h>
|
||||
#include <Analyzer/Passes/FuseFunctionsPass.h>
|
||||
#include <Analyzer/Passes/OptimizeGroupByFunctionKeysPass.h>
|
||||
#include <Analyzer/Passes/IfTransformStringsToEnumPass.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>
|
||||
#include <Analyzer/Passes/ConvertOrLikeChainPass.h>
|
||||
#include <Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.h>
|
||||
#include <Analyzer/Passes/GroupingFunctionsResolvePass.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -44,24 +53,6 @@ namespace
|
||||
*/
|
||||
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:
|
||||
explicit ValidationChecker(String pass_name_)
|
||||
: pass_name(std::move(pass_name_))
|
||||
@ -74,6 +65,57 @@ public:
|
||||
else if (auto * function = node->as<FunctionNode>())
|
||||
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
|
||||
|
||||
@ -87,11 +129,9 @@ public:
|
||||
* TODO: Support setting optimize_using_constraints.
|
||||
* TODO: Support setting optimize_substitute_columns.
|
||||
* 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_aggregators_of_group_by_keys.
|
||||
* TODO: Support setting optimize_duplicate_order_by_and_distinct.
|
||||
* TODO: Support setting optimize_redundant_functions_in_order_by.
|
||||
* TODO: Support setting optimize_monotonous_functions_in_order_by.
|
||||
* TODO: Support settings.optimize_or_like_chain.
|
||||
* TODO: Add optimizations based on function semantics. Example: SELECT * FROM test_table WHERE id != id. (id is not nullable column).
|
||||
@ -195,6 +235,9 @@ void addQueryTreePasses(QueryTreePassManager & manager)
|
||||
if (settings.optimize_injective_functions_inside_uniq)
|
||||
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)
|
||||
manager.addPass(std::make_unique<MultiIfToIfPass>());
|
||||
|
||||
@ -203,6 +246,9 @@ void addQueryTreePasses(QueryTreePassManager & manager)
|
||||
if (settings.optimize_if_chain_to_multiif)
|
||||
manager.addPass(std::make_unique<IfChainToMultiIfPass>());
|
||||
|
||||
if (settings.optimize_redundant_functions_in_order_by)
|
||||
manager.addPass(std::make_unique<OptimizeRedundantFunctionsInOrderByPass>());
|
||||
|
||||
manager.addPass(std::make_unique<OrderByTupleEliminationPass>());
|
||||
manager.addPass(std::make_unique<OrderByLimitByDuplicateEliminationPass>());
|
||||
|
||||
@ -211,6 +257,10 @@ void addQueryTreePasses(QueryTreePassManager & manager)
|
||||
|
||||
if (settings.optimize_if_transform_strings_to_enum)
|
||||
manager.addPass(std::make_unique<IfTransformStringsToEnumPass>());
|
||||
|
||||
manager.addPass(std::make_unique<ConvertOrLikeChainPass>());
|
||||
|
||||
manager.addPass(std::make_unique<GroupingFunctionsResolvePass>());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -130,7 +130,7 @@ BackupEntries BackupEntriesCollector::run()
|
||||
|
||||
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;
|
||||
|
||||
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)
|
||||
inconsistency_error->rethrow();
|
||||
else
|
||||
LOG_WARNING(log, "{}", inconsistency_error->displayText());
|
||||
LOG_WARNING(log, getExceptionMessageAndPattern(*inconsistency_error, /* with_stacktrace */ false));
|
||||
}
|
||||
|
||||
auto sleep_time = getSleepTimeAfterInconsistencyError(pass);
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
#include <filesystem>
|
||||
#include <queue>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -19,7 +19,7 @@ BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile(
|
||||
std::unique_ptr<SeekableReadBuffer> BackupEntryFromAppendOnlyFile::getReadBuffer() const
|
||||
{
|
||||
auto buf = BackupEntryFromImmutableFile::getReadBuffer();
|
||||
return std::make_unique<LimitSeekableReadBuffer>(std::move(buf), limit);
|
||||
return std::make_unique<LimitSeekableReadBuffer>(std::move(buf), 0, limit);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,16 +12,19 @@ namespace ErrorCodes
|
||||
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);
|
||||
copyData(*source, *write_buffer);
|
||||
auto read_buffer = create_read_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();
|
||||
}
|
||||
|
||||
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");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -24,6 +24,8 @@ public:
|
||||
class IBackupWriter /// BackupWriterFile, BackupWriterDisk
|
||||
{
|
||||
public:
|
||||
using CreateReadBufferFunction = std::function<std::unique_ptr<SeekableReadBuffer>()>;
|
||||
|
||||
virtual ~IBackupWriter() = default;
|
||||
virtual bool fileExists(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 removeFiles(const Strings & file_names) = 0;
|
||||
virtual DataSourceDescription getDataSourceDescription() const = 0;
|
||||
virtual void copyFileThroughBuffer(std::unique_ptr<SeekableReadBuffer> && source, const String & file_name);
|
||||
|
||||
virtual bool supportNativeCopy(DataSourceDescription /* data_source_description */) const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
virtual void copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to);
|
||||
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 void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -105,13 +105,21 @@ bool BackupWriterDisk::supportNativeCopy(DataSourceDescription data_source_descr
|
||||
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");
|
||||
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());
|
||||
from_disk->copyFile(file_name_from, *disk, file_path);
|
||||
src_disk->copyFile(src_file_name, *disk, file_path);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -39,8 +39,8 @@ public:
|
||||
DataSourceDescription getDataSourceDescription() 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:
|
||||
DiskPtr disk;
|
||||
std::filesystem::path path;
|
||||
|
@ -125,17 +125,24 @@ bool BackupWriterFile::supportNativeCopy(DataSourceDescription data_source_descr
|
||||
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;
|
||||
if (from_disk)
|
||||
abs_source_path = fullPath(from_disk, file_name_from);
|
||||
if (src_disk)
|
||||
abs_source_path = fullPath(src_disk, src_file_name);
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -35,8 +35,7 @@ public:
|
||||
void removeFiles(const Strings & file_names) override;
|
||||
DataSourceDescription getDataSourceDescription() 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:
|
||||
std::filesystem::path path;
|
||||
|
@ -4,17 +4,19 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
#include <IO/IOThreadPool.h>
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <IO/HTTPHeaderEntries.h>
|
||||
#include <IO/S3/copyDataToS3.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
#include <aws/core/auth/AWSCredentials.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 <filesystem>
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
@ -24,7 +26,6 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int S3_ERROR;
|
||||
extern const int INVALID_CONFIG_PARAMETER;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
@ -150,179 +151,33 @@ bool BackupWriterS3::supportNativeCopy(DataSourceDescription data_source_descrip
|
||||
return getDataSourceDescription() == data_source_description;
|
||||
}
|
||||
|
||||
|
||||
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
|
||||
void BackupWriterS3::copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name)
|
||||
{
|
||||
size_t size = head.GetContentLength();
|
||||
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)
|
||||
if (!src_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)
|
||||
{
|
||||
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
|
||||
{
|
||||
auto object_storage = from_disk->getObjectStorage();
|
||||
std::string source_bucket = object_storage->getObjectsNamespace();
|
||||
auto file_path = fs::path(s3_uri.key) / file_name_to;
|
||||
|
||||
auto head = S3::headObject(*client, source_bucket, objects[0].absolute_path).GetResult();
|
||||
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);
|
||||
}
|
||||
auto object_storage = src_disk->getObjectStorage();
|
||||
std::string src_bucket = object_storage->getObjectsNamespace();
|
||||
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, {},
|
||||
threadPoolCallbackRunner<void>(IOThreadPool::get(), "BackupWriterS3"));
|
||||
}
|
||||
}
|
||||
|
||||
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;
|
||||
|
||||
|
@ -4,22 +4,11 @@
|
||||
|
||||
#if USE_AWS_S3
|
||||
#include <Backups/BackupIO.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <Storages/StorageS3Settings.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
|
||||
{
|
||||
@ -54,12 +43,15 @@ public:
|
||||
UInt64 getFileSize(const String & file_name) override;
|
||||
bool fileContentsEqual(const String & file_name, const String & expected_file_contents) 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 removeFiles(const Strings & file_names) override;
|
||||
|
||||
DataSourceDescription getDataSourceDescription() 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:
|
||||
void copyObjectImpl(
|
||||
@ -67,7 +59,7 @@ private:
|
||||
const String & src_key,
|
||||
const String & dst_bucket,
|
||||
const String & dst_key,
|
||||
const Aws::S3::Model::HeadObjectResult & head,
|
||||
size_t size,
|
||||
const std::optional<ObjectAttributes> & metadata = std::nullopt) const;
|
||||
|
||||
void copyObjectMultipartImpl(
|
||||
@ -75,7 +67,7 @@ private:
|
||||
const String & src_key,
|
||||
const String & dst_bucket,
|
||||
const String & dst_key,
|
||||
const Aws::S3::Model::HeadObjectResult & head,
|
||||
size_t size,
|
||||
const std::optional<ObjectAttributes> & metadata = std::nullopt) const;
|
||||
|
||||
void removeFilesBatch(const Strings & file_names);
|
||||
|
@ -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
|
||||
/// 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.
|
||||
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.
|
||||
|
||||
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
|
||||
{
|
||||
LOG_TRACE(log, "Will copy file {} through memory buffers", 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);
|
||||
LOG_TRACE(log, "Will copy file {}", adjusted_path);
|
||||
|
||||
if (!num_files_written)
|
||||
checkLockFile(true);
|
||||
@ -919,13 +914,18 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
|
||||
coordination->updateFileInfo(info);
|
||||
}
|
||||
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);
|
||||
out->finalize();
|
||||
}
|
||||
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.
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
#include <Backups/SettingsFieldOptionalUUID.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -16,48 +16,6 @@ namespace ErrorCodes
|
||||
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.
|
||||
#define LIST_OF_BACKUP_SETTINGS(M) \
|
||||
M(String, id) \
|
||||
|
@ -30,6 +30,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CONCURRENT_ACCESS_NOT_SUPPORTED;
|
||||
}
|
||||
|
||||
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)
|
||||
, restores_thread_pool(num_restore_threads, /* max_free_threads = */ 0, num_restore_threads)
|
||||
, 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.
|
||||
}
|
||||
@ -157,6 +160,16 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
|
||||
else
|
||||
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;
|
||||
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_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.
|
||||
OperationID restore_id;
|
||||
if (restore_settings.internal)
|
||||
@ -377,7 +393,17 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
else if (!restore_settings.id.empty())
|
||||
restore_id = restore_settings.id;
|
||||
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;
|
||||
if (restore_settings.internal)
|
||||
@ -471,6 +497,7 @@ void BackupsWorker::doRestore(
|
||||
backup_open_params.context = context;
|
||||
backup_open_params.backup_info = 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;
|
||||
BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params);
|
||||
|
||||
@ -687,6 +714,30 @@ std::vector<BackupsWorker::Info> BackupsWorker::getAllInfos() const
|
||||
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()
|
||||
{
|
||||
bool has_active_backups_and_restores = (num_active_backups || num_active_restores);
|
||||
|
@ -23,7 +23,7 @@ class IRestoreCoordination;
|
||||
class BackupsWorker
|
||||
{
|
||||
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.
|
||||
void shutdown();
|
||||
@ -103,6 +103,8 @@ private:
|
||||
void setStatus(const OperationID & id, BackupStatus status, bool throw_if_error = true);
|
||||
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);
|
||||
std::vector<Info> getAllActiveBackupInfos() const;
|
||||
std::vector<Info> getAllActiveRestoreInfos() const;
|
||||
|
||||
ThreadPool backups_thread_pool;
|
||||
ThreadPool restores_thread_pool;
|
||||
@ -113,6 +115,8 @@ private:
|
||||
std::atomic<size_t> num_active_restores = 0;
|
||||
mutable std::mutex infos_mutex;
|
||||
Poco::Logger * log;
|
||||
const bool allow_concurrent_backups;
|
||||
const bool allow_concurrent_restores;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Backups/SettingsFieldOptionalUUID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -162,7 +163,9 @@ namespace
|
||||
M(RestoreUDFCreationMode, create_function) \
|
||||
M(Bool, internal) \
|
||||
M(String, host_id) \
|
||||
M(String, coordination_zk_path)
|
||||
M(String, coordination_zk_path) \
|
||||
M(OptionalUUID, backup_uuid)
|
||||
|
||||
|
||||
RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query)
|
||||
{
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user