Merge branch 'master' into null-as-default-schema-inference

This commit is contained in:
Kruglov Pavel 2023-06-08 12:54:18 +02:00 committed by GitHub
commit 4727c85e1f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
436 changed files with 8097 additions and 5144 deletions

View File

@ -46,7 +46,12 @@ jobs:
- name: Python unit tests
run: |
cd "$GITHUB_WORKSPACE/tests/ci"
echo "Testing the main ci directory"
python3 -m unittest discover -s . -p '*_test.py'
for dir in *_lambda/; do
echo "Testing $dir"
python3 -m unittest discover -s "$dir" -p '*_test.py'
done
DockerHubPushAarch64:
needs: CheckLabels
runs-on: [self-hosted, style-checker-aarch64]

View File

@ -28,14 +28,28 @@ uint64_t getMemoryAmountOrZero()
#if defined(OS_LINUX)
// Try to lookup at the Cgroup limit
std::ifstream cgroup_limit("/sys/fs/cgroup/memory/memory.limit_in_bytes");
if (cgroup_limit.is_open())
// CGroups v2
std::ifstream cgroupv2_limit("/sys/fs/cgroup/memory.max");
if (cgroupv2_limit.is_open())
{
uint64_t memory_limit = 0; // in case of read error
cgroup_limit >> memory_limit;
uint64_t memory_limit = 0;
cgroupv2_limit >> memory_limit;
if (memory_limit > 0 && memory_limit < memory_amount)
memory_amount = memory_limit;
}
else
{
// CGroups v1
std::ifstream cgroup_limit("/sys/fs/cgroup/memory/memory.limit_in_bytes");
if (cgroup_limit.is_open())
{
uint64_t memory_limit = 0; // in case of read error
cgroup_limit >> memory_limit;
if (memory_limit > 0 && memory_limit < memory_amount)
memory_amount = memory_limit;
}
}
#endif
return memory_amount;

View File

@ -274,7 +274,9 @@ void SocketImpl::shutdown()
int SocketImpl::sendBytes(const void* buffer, int length, int flags)
{
if (_isBrokenTimeout)
bool blocking = _blocking && (flags & MSG_DONTWAIT) == 0;
if (_isBrokenTimeout && blocking)
{
if (_sndTimeout.totalMicroseconds() != 0)
{
@ -289,11 +291,13 @@ int SocketImpl::sendBytes(const void* buffer, int length, int flags)
if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException();
rc = ::send(_sockfd, reinterpret_cast<const char*>(buffer), length, flags);
}
while (_blocking && rc < 0 && lastError() == POCO_EINTR);
while (blocking && rc < 0 && lastError() == POCO_EINTR);
if (rc < 0)
{
int err = lastError();
if (err == POCO_EAGAIN || err == POCO_ETIMEDOUT)
if ((err == POCO_EAGAIN || err == POCO_EWOULDBLOCK) && !blocking)
;
else if (err == POCO_EAGAIN || err == POCO_ETIMEDOUT)
throw TimeoutException();
else
error(err);

View File

@ -183,6 +183,16 @@ namespace Net
/// Returns true iff a reused session was negotiated during
/// the handshake.
virtual void setBlocking(bool flag);
/// Sets the socket in blocking mode if flag is true,
/// disables blocking mode if flag is false.
virtual bool getBlocking() const;
/// Returns the blocking mode of the socket.
/// This method will only work if the blocking modes of
/// the socket are changed via the setBlocking method!
protected:
void acceptSSL();
/// Assume per-object mutex is locked.

View File

@ -201,6 +201,16 @@ namespace Net
/// Returns true iff a reused session was negotiated during
/// the handshake.
virtual void setBlocking(bool flag);
/// Sets the socket in blocking mode if flag is true,
/// disables blocking mode if flag is false.
virtual bool getBlocking() const;
/// Returns the blocking mode of the socket.
/// This method will only work if the blocking modes of
/// the socket are changed via the setBlocking method!
protected:
void acceptSSL();
/// Performs a SSL server-side handshake.

View File

@ -629,5 +629,15 @@ bool SecureSocketImpl::sessionWasReused()
return false;
}
void SecureSocketImpl::setBlocking(bool flag)
{
_pSocket->setBlocking(flag);
}
bool SecureSocketImpl::getBlocking() const
{
return _pSocket->getBlocking();
}
} } // namespace Poco::Net

View File

@ -237,5 +237,15 @@ int SecureStreamSocketImpl::completeHandshake()
return _impl.completeHandshake();
}
bool SecureStreamSocketImpl::getBlocking() const
{
return _impl.getBlocking();
}
void SecureStreamSocketImpl::setBlocking(bool flag)
{
_impl.setBlocking(flag);
}
} } // namespace Poco::Net

View File

@ -1,2 +1,2 @@
wget https://github.com/phracker/MacOSX-SDKs/releases/download/10.15/MacOSX10.15.sdk.tar.xz
tar xJf MacOSX10.15.sdk.tar.xz --strip-components=1
wget https://github.com/phracker/MacOSX-SDKs/releases/download/11.3/MacOSX11.0.sdk.tar.xz
tar xJf MacOSX11.0.sdk.tar.xz --strip-components=1

View File

@ -12,6 +12,7 @@ add_library (_lz4 ${SRCS})
add_library (ch_contrib::lz4 ALIAS _lz4)
target_compile_definitions (_lz4 PUBLIC LZ4_DISABLE_DEPRECATE_WARNINGS=1)
target_compile_definitions (_lz4 PUBLIC LZ4_FAST_DEC_LOOP=1)
if (SANITIZE STREQUAL "undefined")
target_compile_options (_lz4 PRIVATE -fno-sanitize=undefined)
endif ()

View File

@ -46,10 +46,12 @@ ENV CXX=clang++-${LLVM_VERSION}
# Rust toolchain and libraries
ENV RUSTUP_HOME=/rust/rustup
ENV CARGO_HOME=/rust/cargo
ENV PATH="/rust/cargo/env:${PATH}"
ENV PATH="/rust/cargo/bin:${PATH}"
RUN curl https://sh.rustup.rs -sSf | bash -s -- -y && \
chmod 777 -R /rust && \
rustup toolchain install nightly && \
rustup default nightly && \
rustup component add rust-src && \
rustup target add aarch64-unknown-linux-gnu && \
rustup target add x86_64-apple-darwin && \
rustup target add x86_64-unknown-freebsd && \

View File

@ -11,9 +11,11 @@ ccache_status () {
[ -O /build ] || git config --global --add safe.directory /build
mkdir -p /build/cmake/toolchain/darwin-x86_64
tar xJf /MacOSX11.0.sdk.tar.xz -C /build/cmake/toolchain/darwin-x86_64 --strip-components=1
ln -sf darwin-x86_64 /build/cmake/toolchain/darwin-aarch64
if [ "$EXTRACT_TOOLCHAIN_DARWIN" = "1" ]; then
mkdir -p /build/cmake/toolchain/darwin-x86_64
tar xJf /MacOSX11.0.sdk.tar.xz -C /build/cmake/toolchain/darwin-x86_64 --strip-components=1
ln -sf darwin-x86_64 /build/cmake/toolchain/darwin-aarch64
fi
# Uncomment to debug ccache. Don't put ccache log in /output right away, or it
# will be confusingly packed into the "performance" package.

View File

@ -167,6 +167,7 @@ def parse_env_variables(
cmake_flags.append(
"-DCMAKE_TOOLCHAIN_FILE=/build/cmake/darwin/toolchain-x86_64.cmake"
)
result.append("EXTRACT_TOOLCHAIN_DARWIN=1")
elif is_cross_darwin_arm:
cc = compiler[: -len(DARWIN_ARM_SUFFIX)]
cmake_flags.append("-DCMAKE_AR:FILEPATH=/cctools/bin/aarch64-apple-darwin-ar")
@ -181,6 +182,7 @@ def parse_env_variables(
cmake_flags.append(
"-DCMAKE_TOOLCHAIN_FILE=/build/cmake/darwin/toolchain-aarch64.cmake"
)
result.append("EXTRACT_TOOLCHAIN_DARWIN=1")
elif is_cross_arm:
cc = compiler[: -len(ARM_SUFFIX)]
cmake_flags.append(

View File

@ -626,7 +626,9 @@ if args.report == "main":
message_array.append(str(faster_queries) + " faster")
if slower_queries:
if slower_queries > 3:
# This threshold should be synchronized with the value in https://github.com/ClickHouse/ClickHouse/blob/master/tests/ci/performance_comparison_check.py#L225
# False positives rate should be < 1%: https://shorturl.at/CDEK8
if slower_queries > 5:
status = "failure"
message_array.append(str(slower_queries) + " slower")

View File

@ -120,3 +120,93 @@ Values can be updated using the `ALTER TABLE` query. The primary key cannot be u
```sql
ALTER TABLE test UPDATE v1 = v1 * 10 + 2 WHERE key LIKE 'some%' AND v3 > 3.1;
```
### Joins
A special `direct` join with EmbeddedRocksDB tables is supported.
This direct join avoids forming a hash table in memory and accesses
the data directly from the EmbeddedRocksDB.
With large joins you may see much lower memory usage with direct joins
because the hash table is not created.
To enable direct joins:
```sql
SET join_algorithm = 'direct, hash'
```
:::tip
When the `join_algorithm` is set to `direct, hash`, direct joins will be used
when possible, and hash otherwise.
:::
#### Example
##### Create and populate an EmbeddedRocksDB table:
```sql
CREATE TABLE rdb
(
`key` UInt32,
`value` Array(UInt32),
`value2` String
)
ENGINE = EmbeddedRocksDB
PRIMARY KEY key
```
```sql
INSERT INTO rdb
SELECT
toUInt32(sipHash64(number) % 10) as key,
[key, key+1] as value,
('val2' || toString(key)) as value2
FROM numbers_mt(10);
```
##### Create and populate a table to join with table `rdb`:
```sql
CREATE TABLE t2
(
`k` UInt16
)
ENGINE = TinyLog
```
```sql
INSERT INTO t2 SELECT number AS k
FROM numbers_mt(10)
```
##### Set the join algorithm to `direct`:
```sql
SET join_algorithm = 'direct'
```
##### An INNER JOIN:
```sql
SELECT *
FROM
(
SELECT k AS key
FROM t2
) AS t2
INNER JOIN rdb ON rdb.key = t2.key
ORDER BY key ASC
```
```response
┌─key─┬─rdb.key─┬─value──┬─value2─┐
│ 0 │ 0 │ [0,1] │ val20 │
│ 2 │ 2 │ [2,3] │ val22 │
│ 3 │ 3 │ [3,4] │ val23 │
│ 6 │ 6 │ [6,7] │ val26 │
│ 7 │ 7 │ [7,8] │ val27 │
│ 8 │ 8 │ [8,9] │ val28 │
│ 9 │ 9 │ [9,10] │ val29 │
└─────┴─────────┴────────┴────────┘
```
### More information on Joins
- [`join_algorithm` setting](/docs/en/operations/settings/settings.md#settings-join_algorithm)
- [JOIN clause](/docs/en/sql-reference/statements/select/join.md)

View File

@ -42,7 +42,6 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
[rabbitmq_queue_consume = false,]
[rabbitmq_address = '',]
[rabbitmq_vhost = '/',]
[rabbitmq_queue_consume = false,]
[rabbitmq_username = '',]
[rabbitmq_password = '',]
[rabbitmq_commit_on_select = false,]

View File

@ -109,7 +109,7 @@ INSERT INTO test.visits (StartDate, CounterID, Sign, UserID)
VALUES (1667446031, 1, 6, 3)
```
The data are inserted in both the table and the materialized view `test.mv_visits`.
The data is inserted in both the table and the materialized view `test.mv_visits`.
To get the aggregated data, we need to execute a query such as `SELECT ... GROUP BY ...` from the materialized view `test.mv_visits`:

View File

@ -1,147 +1,156 @@
# Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex}
The main task that indexes achieve is to quickly find nearest neighbors for multidimensional data. An example of such a problem can be finding similar pictures (texts) for a given picture (text). That problem can be reduced to finding the nearest [embeddings](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning). They can be created from data using [UDF](/docs/en/sql-reference/functions/index.md/#executable-user-defined-functions).
Nearest neighborhood search refers to the problem of finding the point(s) with the smallest distance to a given point in an n-dimensional
space. Since exact search is in practice usually typically too slow, the task is often solved with approximate algorithms. A popular use
case of of neighbor search is finding similar pictures (texts) for a given picture (text). Pictures (texts) can be decomposed into
[embeddings](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning), and instead of
comparing pictures (texts) pixel-by-pixel (character-by-character), only the embeddings are compared.
The next queries find the closest neighbors in N-dimensional space using the L2 (Euclidean) distance:
``` sql
SELECT *
FROM table_name
WHERE L2Distance(Column, Point) < MaxDistance
In terms of SQL, the problem can be expressed as follows:
``` sql
SELECT *
FROM table
WHERE L2Distance(column, Point) < MaxDistance
LIMIT N
```
``` sql
SELECT *
FROM table_name
ORDER BY L2Distance(Column, Point)
``` sql
SELECT *
FROM table
ORDER BY L2Distance(column, Point)
LIMIT N
```
But it will take some time for execution because of the long calculation of the distance between `TargetEmbedding` and all other vectors. This is where ANN indexes can help. They store a compact approximation of the search space (e.g. using clustering, search trees, etc.) and are able to compute approximate neighbors quickly.
## Indexes Structure
The queries are expensive because the L2 (Euclidean) distance between `Point` and all points in `column` and must be computed. To speed this process up, Approximate Nearest Neighbor Search Indexes (ANN indexes) store a compact representation of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer quickly.
Approximate Nearest Neighbor Search Indexes (`ANNIndexes`) are similar to skip indexes. They are constructed by some granules and determine which of them should be skipped. Compared to skip indices, ANN indices use their results not only to skip some group of granules, but also to select particular granules from a set of granules.
# Creating ANN Indexes
`ANNIndexes` are designed to speed up two types of queries:
As long as ANN indexes are experimental, you first need to `SET allow_experimental_annoy_index = 1`.
- ###### Type 1: Where
``` sql
SELECT *
FROM table_name
WHERE DistanceFunction(Column, Point) < MaxDistance
Syntax to create an ANN index over an `Array` column:
```sql
CREATE TABLE table
(
`id` Int64,
`embedding` Array(Float32),
INDEX <ann_index_name> embedding TYPE <ann_index_type>(<ann_index_parameters>) GRANULARITY <N>
)
ENGINE = MergeTree
ORDER BY id;
```
Syntax to create an ANN index over a `Tuple` column:
```sql
CREATE TABLE table
(
`id` Int64,
`embedding` Tuple(Float32[, Float32[, ...]]),
INDEX <ann_index_name> embedding TYPE <ann_index_type>(<ann_index_parameters>) GRANULARITY <N>
)
ENGINE = MergeTree
ORDER BY id;
```
ANN indexes are built during column insertion and merge and `INSERT` and `OPTIMIZE` statements will be slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively there are much more read requests than write requests.
Similar to regular skip indexes, ANN indexes are constructed over granules and each indexed block consists of `GRANULARITY = <N>`-many
granules. For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and `GRANULARITY = 2`,
then each indexed block will consist of 16384 rows. However, unlike skip indexes, ANN indexes are not only able to skip the entire indexed
block, they are able to skip individual granules in indexed blocks. As a result, the `GRANULARITY` parameter has a different meaning in ANN
indexes than in normal skip indexes. Basically, the bigger `GRANULARITY` is chosen, the more data is provided to a single ANN index, and the
higher the chance that with the right hyper parameters, the index will remember the data structure better.
# Using ANN Indexes
ANN indexes support two types of queries:
- WHERE queries:
``` sql
SELECT *
FROM table
WHERE DistanceFunction(column, Point) < MaxDistance
LIMIT N
```
- ###### Type 2: Order by
- ORDER BY queries:
``` sql
SELECT *
FROM table_name [WHERE ...]
ORDER BY DistanceFunction(Column, Point)
SELECT *
FROM table
[WHERE ...]
ORDER BY DistanceFunction(column, Point)
LIMIT N
```
In these queries, `DistanceFunction` is selected from [distance functions](/docs/en/sql-reference/functions/distance-functions.md). `Point` is a known vector (something like `(0.1, 0.1, ... )`). To avoid writing large vectors, use [client parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters). `Value` - a float value that will bound the neighbourhood.
`DistanceFunction` is a [distance function](/docs/en/sql-reference/functions/distance-functions.md), `Point` is a reference vector (e.g. `(0.17, 0.33, ...)`) and `MaxDistance` is a floating point value which restricts the size of the neighbourhood.
:::note
ANN index can't speed up query that satisfies both types (`where + order by`, only one of them). All queries must have the limit, as algorithms are used to find nearest neighbors and need a specific number of them.
:::tip
To avoid writing out large vectors, you can use [query parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g.
```bash
clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(embedding, {vec: Array(Float32)}) < 1.0"
```
:::
:::note
Indexes are applied only to queries with a limit less than the `max_limit_for_ann_queries` setting. This helps to avoid memory overflows in queries with a large limit. `max_limit_for_ann_queries` setting can be changed if you know you can provide enough memory. The default value is `1000000`.
:::
ANN indexes cannot speed up queries that contain both a `WHERE DistanceFunction(column, Point) < MaxDistance` and an `ORDER BY DistanceFunction(column, Point)` clause. Also, the approximate algorithms used to determine the nearest neighbors require a limit, hence queries that use an ANN index must have a `LIMIT` clause.
Both types of queries are handled the same way. The indexes get `n` neighbors (where `n` is taken from the `LIMIT` clause) and work with them. In `ORDER BY` query they remember the numbers of all parts of the granule that have at least one of neighbor. In `WHERE` query they remember only those parts that satisfy the requirements.
An ANN index is only used if the query has a `LIMIT` value smaller than setting `max_limit_for_ann_queries` (default: 1 million rows). This is a safety measure which helps to avoid large memory consumption by external libraries for approximate neighbor search.
# Available ANN Indexes
## Create table with ANNIndex
This feature is disabled by default. To enable it, set `allow_experimental_annoy_index` to 1. Also, this feature is disabled on ARM, due to likely problems with the algorithm.
```sql
CREATE TABLE t
(
`id` Int64,
`data` Tuple(Float32, Float32, Float32),
INDEX ann_index_name data TYPE ann_index_type(ann_index_parameters) GRANULARITY N
)
ENGINE = MergeTree
ORDER BY id;
```
```sql
CREATE TABLE t
(
`id` Int64,
`data` Array(Float32),
INDEX ann_index_name data TYPE ann_index_type(ann_index_parameters) GRANULARITY N
)
ENGINE = MergeTree
ORDER BY id;
```
With greater `GRANULARITY` indexes remember the data structure better. The `GRANULARITY` indicates how many granules will be used to construct the index. The more data is provided for the index, the more of it can be handled by one index and the more chances that with the right hyper parameters the index will remember the data structure better. But some indexes can't be built if they don't have enough data, so this granule will always participate in the query. For more information, see the description of indexes.
As the indexes are built only during insertions into table, `INSERT` and `OPTIMIZE` queries are slower than for ordinary table. At this stage indexes remember all the information about the given data. ANNIndexes should be used if you have immutable or rarely changed data and many read requests.
You can create your table with index which uses certain algorithm. Now only indices based on the following algorithms are supported:
# Index list
- [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy)
# Annoy {#annoy}
Implementation of the algorithm was taken from [this repository](https://github.com/spotify/annoy).
## Annoy {#annoy}
Short description of the algorithm:
The algorithm recursively divides in half all space by random linear surfaces (lines in 2D, planes in 3D etc.). Thus it makes tree of polyhedrons and points that they contains. Repeating the operation several times for greater accuracy it creates a forest.
To find K Nearest Neighbours it goes down through the trees and fills the buffer of closest points using the priority queue of polyhedrons. Next, it sorts buffer and return the nearest K points.
(currently disabled on ARM due to memory safety problems with the algorithm)
This type of ANN index implements [the Annoy algorithm](https://github.com/spotify/annoy) which uses a recursive division of the space in random linear surfaces (lines in 2D, planes in 3D etc.).
Syntax to create a Annoy index over a `Array` column:
__Examples__:
```sql
CREATE TABLE t
CREATE TABLE table
(
id Int64,
data Tuple(Float32, Float32, Float32),
INDEX ann_index_name data TYPE annoy(NumTrees, DistanceName) GRANULARITY N
embedding Array(Float32),
INDEX <ann_index_name> embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N
)
ENGINE = MergeTree
ORDER BY id;
```
Syntax to create a Annoy index over a `Tuple` column:
```sql
CREATE TABLE t
CREATE TABLE table
(
id Int64,
data Array(Float32),
INDEX ann_index_name data TYPE annoy(NumTrees, DistanceName) GRANULARITY N
embedding Tuple(Float32[, Float32[, ...]]),
INDEX <ann_index_name> embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N
)
ENGINE = MergeTree
ORDER BY id;
```
Parameter `DistanceName` is name of a distance function (default `L2Distance`). Annoy currently supports `L2Distance` and `cosineDistance` as distance functions. Parameter `NumTrees` (default: 100) is the number of trees which the algorithm will create. Higher values of `NumTree` mean slower `CREATE` and `SELECT` statements (approximately linearly), but increase the accuracy of search results.
:::note
Table with array field will work faster, but all arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(data) = 256`.
Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(embedding) = 256`.
:::
Parameter `NumTrees` is the number of trees which the algorithm will create. The bigger it is, the slower (approximately linear) it works (in both `CREATE` and `SELECT` requests), but the better accuracy you get (adjusted for randomness). By default it is set to `100`. Parameter `DistanceName` is name of distance function. By default it is set to `L2Distance`. It can be set without changing first parameter, for example
```sql
CREATE TABLE t
(
id Int64,
data Array(Float32),
INDEX ann_index_name data TYPE annoy('cosineDistance') GRANULARITY N
)
ENGINE = MergeTree
ORDER BY id;
```
Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. It can be used to
balance runtime and accuracy at runtime.
Annoy supports `L2Distance` and `cosineDistance`.
Example:
In the `SELECT` in the settings (`ann_index_select_query_params`) you can specify the size of the internal buffer (more details in the description above or in the [original repository](https://github.com/spotify/annoy)). During the query it will inspect up to `search_k` nodes which defaults to `n_trees * n` if not provided. `search_k` gives you a run-time trade-off between better accuracy and speed.
__Example__:
``` sql
SELECT *
FROM table_name [WHERE ...]
ORDER BY L2Distance(Column, Point)
SELECT *
FROM table_name [WHERE ...]
ORDER BY L2Distance(column, Point)
LIMIT N
SETTING ann_index_select_query_params=`k_search=100`
SETTINGS annoy_index_search_k_nodes=100
```

View File

@ -15,6 +15,18 @@ tokenized cells of the string column. For example, the string cell "I will be a
" 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.
<div class='vimeo-container'>
<iframe src="//www.youtube.com/embed/O_MnyUkrIq8"
width="640"
height="360"
frameborder="0"
allow="autoplay;
fullscreen;
picture-in-picture"
allowfullscreen>
</iframe>
</div>
:::note
Inverted indexes are experimental and should not be used in production environments yet. They may change in the future in backward-incompatible
ways, for example with respect to their DDL/DQL syntax or performance/compression characteristics.

View File

@ -75,7 +75,7 @@ SELECT
payment_type,
pickup_ntaname,
dropoff_ntaname
FROM s3(
FROM gcs(
'https://storage.googleapis.com/clickhouse-public-datasets/nyc-taxi/trips_{0..2}.gz',
'TabSeparatedWithNames'
);

View File

@ -467,6 +467,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe
- [output_format_csv_crlf_end_of_line](/docs/en/operations/settings/settings-formats.md/#output_format_csv_crlf_end_of_line) - if it is set to true, end of line in CSV output format will be `\r\n` instead of `\n`. Default value - `false`.
- [input_format_csv_skip_first_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_first_lines) - skip the specified number of lines at the beginning of data. Default value - `0`.
- [input_format_csv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_csv_detect_header) - automatically detect header with names and types in CSV format. Default value - `true`.
- [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`.
## CSVWithNames {#csvwithnames}

View File

@ -882,6 +882,38 @@ My NULL
My NULL
```
### input_format_csv_trim_whitespaces {#input_format_csv_trim_whitespaces}
Trims spaces and tabs in non-quoted CSV strings.
Default value: `true`.
**Examples**
Query
```bash
echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true
```
Result
```text
"string"
```
Query
```bash
echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=false
```
Result
```text
" string "
```
## Values format settings {#values-format-settings}
### input_format_values_interpret_expressions {#input_format_values_interpret_expressions}

View File

@ -227,6 +227,89 @@ SELECT * FROM data_01515 WHERE d1 = 0 SETTINGS force_data_skipping_indices='`d1_
SELECT * FROM data_01515 WHERE d1 = 0 AND assumeNotNull(d1_null) = 0 SETTINGS force_data_skipping_indices='`d1_idx`, d1_null_idx'; -- Ok.
```
## ignore_data_skipping_indices {#settings-ignore_data_skipping_indices}
Ignores the skipping indexes specified if used by the query.
Consider the following example:
```sql
CREATE TABLE data
(
key Int,
x Int,
y Int,
INDEX x_idx x TYPE minmax GRANULARITY 1,
INDEX y_idx y TYPE minmax GRANULARITY 1,
INDEX xy_idx (x,y) TYPE minmax GRANULARITY 1
)
Engine=MergeTree()
ORDER BY key;
INSERT INTO data VALUES (1, 2, 3);
SELECT * FROM data;
SELECT * FROM data SETTINGS ignore_data_skipping_indices=''; -- query will produce CANNOT_PARSE_TEXT error.
SELECT * FROM data SETTINGS ignore_data_skipping_indices='x_idx'; -- Ok.
SELECT * FROM data SETTINGS ignore_data_skipping_indices='na_idx'; -- Ok.
SELECT * FROM data WHERE x = 1 AND y = 1 SETTINGS ignore_data_skipping_indices='xy_idx',force_data_skipping_indices='xy_idx' ; -- query will produce INDEX_NOT_USED error, since xy_idx is explictly ignored.
SELECT * FROM data WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx';
```
The query without ignoring any indexes:
```sql
EXPLAIN indexes = 1 SELECT * FROM data WHERE x = 1 AND y = 2;
Expression ((Projection + Before ORDER BY))
Filter (WHERE)
ReadFromMergeTree (default.data)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 1/1
Skip
Name: x_idx
Description: minmax GRANULARITY 1
Parts: 0/1
Granules: 0/1
Skip
Name: y_idx
Description: minmax GRANULARITY 1
Parts: 0/0
Granules: 0/0
Skip
Name: xy_idx
Description: minmax GRANULARITY 1
Parts: 0/0
Granules: 0/0
```
Ignoring the `xy_idx` index:
```sql
EXPLAIN indexes = 1 SELECT * FROM data WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx';
Expression ((Projection + Before ORDER BY))
Filter (WHERE)
ReadFromMergeTree (default.data)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 1/1
Skip
Name: x_idx
Description: minmax GRANULARITY 1
Parts: 0/1
Granules: 0/1
Skip
Name: y_idx
Description: minmax GRANULARITY 1
Parts: 0/0
Granules: 0/0
```
Works with tables in the MergeTree family.
## convert_query_to_cnf {#convert_query_to_cnf}
@ -4229,6 +4312,12 @@ Default value: `2000`
If it's enabled, in hedged requests we can start new connection until receiving first data packet even if we have already made some progress
(but progress haven't updated for `receive_data_timeout` timeout), otherwise we disable changing replica after the first time we made progress.
## parallel_view_processing
Enables pushing to attached views concurrently instead of sequentially.
Default value: `false`.
## partial_result_on_first_cancel {#partial_result_on_first_cancel}
When set to `true` and the user wants to interrupt a query (for example using `Ctrl+C` on the client), then the query continues execution only on data that was already read from the table. Afterwards, it will return a partial result of the query for the part of the table that was read. To fully stop the execution of a query without a partial result, the user should send 2 cancel requests.

View File

@ -10,14 +10,14 @@ Columns:
- `user` (String) The user who made the query. Keep in mind that for distributed processing, queries are sent to remote servers under the `default` user. The field contains the username for a specific query, not for a query that this query initiated.
- `address` (String) The IP address the request was made from. The same for distributed processing. To track where a distributed query was originally made from, look at `system.processes` on the query requestor server.
- `elapsed` (Float64) The time in seconds since request execution started.
- `rows_read` (UInt64) The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
- `bytes_read` (UInt64) The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
- `read_rows` (UInt64) The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
- `read_bytes` (UInt64) The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
- `total_rows_approx` (UInt64) The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known.
- `memory_usage` (UInt64) Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting.
- `memory_usage` (Int64) Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting.
- `query` (String) The query text. For `INSERT`, it does not include the data to insert.
- `query_id` (String) Query ID, if defined.
- `is_cancelled` (Int8) Was query cancelled.
- `is_all_data_sent` (Int8) Was all data sent to the client (in other words query had been finished on the server).
- `is_cancelled` (UInt8) Was query cancelled.
- `is_all_data_sent` (UInt8) Was all data sent to the client (in other words query had been finished on the server).
```sql
SELECT * FROM system.processes LIMIT 10 FORMAT Vertical;

View File

@ -0,0 +1,28 @@
---
slug: /en/operations/system-tables/user_processes
---
# user_processes
This system table can be used to get overview of memory usage and ProfileEvents of users.
Columns:
- `user` ([String](../../sql-reference/data-types/string.md)) — User name.
- `memory_usage` ([Int64](../../sql-reference/data-types/int-uint#int-ranges)) Sum of RAM used by all processes of the user. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting.
- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint#int-ranges)) — The peak of memory usage of the user. It can be reset when no queries are run for the user.
- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/map)) Summary of ProfileEvents that measure different metrics for the user. The description of them could be found in the table [system.events](../../operations/system-tables/events.md#system_tables-events)
```sql
SELECT * FROM system.user_processes LIMIT 10 FORMAT Vertical;
```
```response
Row 1:
──────
user: default
memory_usage: 9832
peak_memory_usage: 9832
ProfileEvents: {'Query':5,'SelectQuery':5,'QueriesWithSubqueries':38,'SelectQueriesWithSubqueries':38,'QueryTimeMicroseconds':842048,'SelectQueryTimeMicroseconds':842048,'ReadBufferFromFileDescriptorRead':6,'ReadBufferFromFileDescriptorReadBytes':234,'IOBufferAllocs':3,'IOBufferAllocBytes':98493,'ArenaAllocChunks':283,'ArenaAllocBytes':1482752,'FunctionExecute':670,'TableFunctionExecute':16,'DiskReadElapsedMicroseconds':19,'NetworkSendElapsedMicroseconds':684,'NetworkSendBytes':139498,'SelectedRows':6076,'SelectedBytes':685802,'ContextLock':1140,'RWLockAcquiredReadLocks':193,'RWLockReadersWaitMilliseconds':4,'RealTimeMicroseconds':1585163,'UserTimeMicroseconds':889767,'SystemTimeMicroseconds':13630,'SoftPageFaults':1947,'OSCPUWaitMicroseconds':6,'OSCPUVirtualTimeMicroseconds':903251,'OSReadChars':28631,'OSWriteChars':28888,'QueryProfilerRuns':3,'LogTrace':79,'LogDebug':24}
1 row in set. Elapsed: 0.010 sec.
```

View File

@ -0,0 +1,53 @@
---
slug: /en/operations/utilities/clickhouse-keeper-client
sidebar_label: clickhouse-keeper-client
---
# clickhouse-keeper-client
A client application to interact with clickhouse-keeper by its native protocol.
## Keys {#clickhouse-keeper-client}
- `-q QUERY`, `--query=QUERY` — Query to execute. If this parameter is not passed, `clickhouse-keeper-client` will start in interactive mode.
- `-h HOST`, `--host=HOST` — Server host. Default value: `localhost`.
- `-p N`, `--port=N` — Server port. Default value: 2181
- `--connection-timeout=TIMEOUT` — Set connection timeout in seconds. Default value: 10s.
- `--session-timeout=TIMEOUT` — Set session timeout in seconds. Default value: 10s.
- `--operation-timeout=TIMEOUT` — Set operation timeout in seconds. Default value: 10s.
- `--history-file=FILE_PATH` — Set path of history file. Default value: `~/.keeper-client-history`.
- `--help` — Shows the help message.
## Example {#clickhouse-keeper-client-example}
```bash
./clickhouse-keeper-client -h localhost:2181 --connection-timeout 30 --session-timeout 30 --operation-timeout 30
Connected to ZooKeeper at [::1]:2181 with session_id 137
/ :) ls
keeper foo bar
/ :) cd keeper
/keeper :) ls
api_version
/keeper :) cd api_version
/keeper/api_version :) ls
/keeper/api_version :) cd xyz
Path /keeper/api_version/xyz does not exists
/keeper/api_version :) cd ../../
/ :) ls
keeper foo bar
/ :) get keeper/api_version
2
```
## Commands {#clickhouse-keeper-client-commands}
- `ls [path]` -- Lists the nodes for the given path (default: cwd)
- `cd [path]` -- Change the working path (default `.`)
- `set <path> <value> [version]` -- Updates the node's value. Only update if version matches (default: -1)
- `create <path> <value>` -- Creates new node
- `get <path>` -- Returns the node's value
- `remove <path>` -- Remove the node
- `rmr <path>` -- Recursively deletes path. Confirmation required
- `flwc <command>` -- Executes four-letter-word command
- `help` -- Prints this message

View File

@ -30,7 +30,34 @@ Example 2: `uniqArray(arr)` Counts the number of unique elements in all a
The -Map suffix can be appended to any aggregate function. This will create an aggregate function which gets Map type as an argument, and aggregates values of each key of the map separately using the specified aggregate function. The result is also of a Map type.
Examples: `sumMap(map(1,1))`, `avgMap(map('a', 1))`.
**Example**
```sql
CREATE TABLE map_map(
date Date,
timeslot DateTime,
status Map(String, UInt64)
) ENGINE = Log;
INSERT INTO map_map VALUES
('2000-01-01', '2000-01-01 00:00:00', (['a', 'b', 'c'], [10, 10, 10])),
('2000-01-01', '2000-01-01 00:00:00', (['c', 'd', 'e'], [10, 10, 10])),
('2000-01-01', '2000-01-01 00:01:00', (['d', 'e', 'f'], [10, 10, 10])),
('2000-01-01', '2000-01-01 00:01:00', (['f', 'g', 'g'], [10, 10, 10]));
SELECT
timeslot,
sumMap(status),
avgMap(status),
minMap(status)
FROM map_map
GROUP BY timeslot;
┌────────────timeslot─┬─sumMap(status)───────────────────────┬─avgMap(status)───────────────────────┬─minMap(status)───────────────────────┐
│ 2000-01-01 00:00:00 │ {'a':10,'b':10,'c':20,'d':10,'e':10} │ {'a':10,'b':10,'c':10,'d':10,'e':10} │ {'a':10,'b':10,'c':10,'d':10,'e':10} │
│ 2000-01-01 00:01:00 │ {'d':10,'e':10,'f':20,'g':20} │ {'d':10,'e':10,'f':10,'g':10} │ {'d':10,'e':10,'f':10,'g':10} │
└─────────────────────┴──────────────────────────────────────┴──────────────────────────────────────┴──────────────────────────────────────┘
```
## -SimpleState

View File

@ -4,7 +4,7 @@ sidebar_label: Aggregate Functions
sidebar_position: 33
---
# Aggregate Functions
# Aggregate Functions
Aggregate functions work in the [normal](http://www.sql-tutorial.com/sql-aggregate-functions-sql-tutorial) way as expected by database experts.
@ -72,3 +72,16 @@ FROM t_null_big
│ 2.3333333333333335 │ 1.4 │
└────────────────────┴─────────────────────┘
```
Also you can use [Tuple](/docs/en/sql-reference/data-types/tuple.md) to work around NULL skipping behavior. The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value.
```sql
SELECT
groupArray(y),
groupArray(tuple(y)).1
FROM t_null_big;
┌─groupArray(y)─┬─tupleElement(groupArray(tuple(y)), 1)─┐
│ [2,2,3] │ [2,NULL,2,3,NULL] │
└───────────────┴───────────────────────────────────────┘
```

View File

@ -6,6 +6,7 @@ sidebar_position: 106
# argMax
Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered.
Both parts the `arg` and the `max` behave as [aggregate functions](/docs/en/sql-reference/aggregate-functions/index.md), they both [skip `Null`](/docs/en/sql-reference/aggregate-functions/index.md#null-processing) during processing and return not `Null` values if not `Null` values are available.
**Syntax**
@ -49,3 +50,60 @@ Result:
│ director │
└──────────────────────┘
```
**Extended example**
```sql
CREATE TABLE test
(
a Nullable(String),
b Nullable(Int64)
)
ENGINE = Memory AS
SELECT *
FROM VALUES(('a', 1), ('b', 2), ('c', 2), (NULL, 3), (NULL, NULL), ('d', NULL));
select * from test;
┌─a────┬────b─┐
│ a │ 1 │
│ b │ 2 │
│ c │ 2 │
│ ᴺᵁᴸᴸ │ 3 │
│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │
│ d │ ᴺᵁᴸᴸ │
└──────┴──────┘
SELECT argMax(a, b), max(b) FROM test;
┌─argMax(a, b)─┬─max(b)─┐
│ b │ 3 │ -- argMax = 'b' because it the first not Null value, max(b) is from another row!
└──────────────┴────────┘
SELECT argMax(tuple(a), b) FROM test;
┌─argMax(tuple(a), b)─┐
│ (NULL) │ -- The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value
└─────────────────────┘
SELECT (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB FROM test;
┌─argMaxA─┬─argMaxB─┐
│ ᴺᵁᴸᴸ │ 3 │ -- you can use Tuple and get both (all - tuple(*)) columns for the according max(b)
└─────────┴─────────┘
SELECT argMax(a, b), max(b) FROM test WHERE a IS NULL AND b IS NULL;
┌─argMax(a, b)─┬─max(b)─┐
│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- All aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL`
└──────────────┴────────┘
SELECT argMax(a, (b,a)) FROM test;
┌─argMax(a, tuple(b, a))─┐
│ c │ -- There are two rows with b=2, `Tuple` in the `Max` allows to get not the first `arg`
└────────────────────────┘
SELECT argMax(a, tuple(b)) FROM test;
┌─argMax(a, tuple(b))─┐
│ b │ -- `Tuple` can be used in `Max` to not skip Nulls in `Max`
└─────────────────────┘
```
**See also**
- [Tuple](/docs/en/sql-reference/data-types/tuple.md)

View File

@ -6,6 +6,7 @@ sidebar_position: 105
# argMin
Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered.
Both parts the `arg` and the `min` behave as [aggregate functions](/docs/en/sql-reference/aggregate-functions/index.md), they both [skip `Null`](/docs/en/sql-reference/aggregate-functions/index.md#null-processing) during processing and return not `Null` values if not `Null` values are available.
**Syntax**
@ -49,3 +50,65 @@ Result:
│ worker │
└──────────────────────┘
```
**Extended example**
```sql
CREATE TABLE test
(
a Nullable(String),
b Nullable(Int64)
)
ENGINE = Memory AS
SELECT *
FROM VALUES((NULL, 0), ('a', 1), ('b', 2), ('c', 2), (NULL, NULL), ('d', NULL));
select * from test;
┌─a────┬────b─┐
│ ᴺᵁᴸᴸ │ 0 │
│ a │ 1 │
│ b │ 2 │
│ c │ 2 │
│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │
│ d │ ᴺᵁᴸᴸ │
└──────┴──────┘
SELECT argMin(a, b), min(b) FROM test;
┌─argMin(a, b)─┬─min(b)─┐
│ a │ 0 │ -- argMin = a because it the first not `NULL` value, min(b) is from another row!
└──────────────┴────────┘
SELECT argMin(tuple(a), b) FROM test;
┌─argMin(tuple(a), b)─┐
│ (NULL) │ -- The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value
└─────────────────────┘
SELECT (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test;
┌─argMinA─┬─argMinB─┐
│ ᴺᵁᴸᴸ │ 0 │ -- you can use `Tuple` and get both (all - tuple(*)) columns for the according max(b)
└─────────┴─────────┘
SELECT argMin(a, b), min(b) FROM test WHERE a IS NULL and b IS NULL;
┌─argMin(a, b)─┬─min(b)─┐
│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- All aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL`
└──────────────┴────────┘
SELECT argMin(a, (b, a)), min(tuple(b, a)) FROM test;
┌─argMin(a, tuple(b, a))─┬─min(tuple(b, a))─┐
│ d │ (NULL,NULL) │ -- 'd' is the first not `NULL` value for the min
└────────────────────────┴──────────────────┘
SELECT argMin((a, b), (b, a)), min(tuple(b, a)) FROM test;
┌─argMin(tuple(a, b), tuple(b, a))─┬─min(tuple(b, a))─┐
│ (NULL,NULL) │ (NULL,NULL) │ -- argMin returns (NULL,NULL) here because `Tuple` allows to don't skip `NULL` and min(tuple(b, a)) in this case is minimal value for this dataset
└──────────────────────────────────┴──────────────────┘
SELECT argMin(a, tuple(b)) FROM test;
┌─argMax(a, tuple(b))─┐
│ d │ -- `Tuple` can be used in `min` to not skip rows with `NULL` values as b.
└─────────────────────┘
```
**See also**
- [Tuple](/docs/en/sql-reference/data-types/tuple.md)

View File

@ -6,24 +6,32 @@ sidebar_position: 7
# first_value
Selects the first encountered value, similar to `any`, but could accept NULL.
Mostly it should be used with [Window Functions](../../window-functions/index.md).
Without Window Functions the result will be random if the source stream is not ordered.
## examples
```sql
insert into test_data (a,b) values (1,null), (2,3), (4, 5), (6,null)
CREATE TABLE test_data
(
a Int64,
b Nullable(Int64)
)
ENGINE = Memory;
INSERT INTO test_data (a, b) Values (1,null), (2,3), (4, 5), (6,null);
```
### example1
The NULL value is ignored at default.
```sql
select first_value(b) from test_data
select first_value(b) from test_data;
```
```text
┌─first_value_ignore_nulls(b)─┐
│ 3 │
└─────────────────────────────┘
```
### example2
@ -36,7 +44,6 @@ select first_value(b) ignore nulls from test_data
┌─first_value_ignore_nulls(b)─┐
│ 3 │
└─────────────────────────────┘
```
### example3
@ -46,10 +53,28 @@ select first_value(b) respect nulls from test_data
```
```text
┌─first_value_respect_nulls(b)─┐
│ ᴺᵁᴸᴸ │
└──────────────────────────────┘
```
### example4
Stabilized result using the sub-query with `ORDER BY`.
```sql
SELECT
first_value_respect_nulls(b),
first_value(b)
FROM
(
SELECT *
FROM test_data
ORDER BY a ASC
)
```
```text
┌─first_value_respect_nulls(b)─┬─first_value(b)─┐
│ ᴺᵁᴸᴸ │ 3 │
└──────────────────────────────┴────────────────┘
```

View File

@ -6,12 +6,20 @@ sidebar_position: 8
# last_value
Selects the last encountered value, similar to `anyLast`, but could accept NULL.
Mostly it should be used with [Window Functions](../../window-functions/index.md).
Without Window Functions the result will be random if the source stream is not ordered.
## examples
```sql
insert into test_data (a,b) values (1,null), (2,3), (4, 5), (6,null)
CREATE TABLE test_data
(
a Int64,
b Nullable(Int64)
)
ENGINE = Memory;
INSERT INTO test_data (a, b) Values (1,null), (2,3), (4, 5), (6,null)
```
### example1
@ -50,4 +58,24 @@ select last_value(b) respect nulls from test_data
└─────────────────────────────┘
```
### example4
Stabilized result using the sub-query with `ORDER BY`.
```sql
SELECT
last_value_respect_nulls(b),
last_value(b)
FROM
(
SELECT *
FROM test_data
ORDER BY a ASC
)
```
```text
┌─last_value_respect_nulls(b)─┬─last_value(b)─┐
│ ᴺᵁᴸᴸ │ 5 │
└─────────────────────────────┴───────────────┘
```

View File

@ -5,7 +5,11 @@ sidebar_position: 141
# sumMap
Syntax: `sumMap(key, value)` or `sumMap(Tuple(key, value))`
Syntax: `sumMap(key <Array>, value <Array>)` [Array type](../../data-types/array.md) or `sumMap(Tuple(key <Array>, value <Array>))` [Tuple type](../../data-types/tuple.md).
Arguments:
Alias: `sumMappedArrays`.
Totals the `value` array according to the keys specified in the `key` array.
@ -27,6 +31,7 @@ CREATE TABLE sum_map(
),
statusMapTuple Tuple(Array(Int32), Array(Int32))
) ENGINE = Log;
INSERT INTO sum_map VALUES
('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10], ([1, 2, 3], [10, 10, 10])),
('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10], ([3, 4, 5], [10, 10, 10])),
@ -47,3 +52,7 @@ GROUP BY timeslot
│ 2000-01-01 00:01:00 │ ([4,5,6,7,8],[10,10,20,10,10]) │ ([4,5,6,7,8],[10,10,20,10,10]) │
└─────────────────────┴──────────────────────────────────────────────┴────────────────────────────────┘
```
**See Also**
- [-Map combinator for Map datatype](../combinators.md#-map)

View File

@ -63,7 +63,7 @@ SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Asia/
``` text
┌───────────────timestamp─┬─event_id─┐
│ 2019-01-01 00:00:00.000 │ 2
│ 2019-01-01 00:00:00.000 │ 3
└─────────────────────────┴──────────┘
```
@ -75,8 +75,8 @@ SELECT * FROM dt WHERE timestamp = toDateTime64(1546300800.123, 3);
``` text
┌───────────────timestamp─┬─event_id─┐
│ 2019-01-01 00:00:00.123 │ 1 │
│ 2019-01-01 00:00:00.123 │ 2 │
│ 2019-01-01 03:00:00.123 │ 1 │
│ 2019-01-01 03:00:00.123 │ 2 │
└─────────────────────────┴──────────┘
```
@ -91,7 +91,7 @@ SELECT toDateTime64(now(), 3, 'Asia/Istanbul') AS column, toTypeName(column) AS
``` text
┌──────────────────column─┬─x──────────────────────────────┐
│ 2019-10-16 04:12:04.000 │ DateTime64(3, 'Asia/Istanbul') │
│ 2023-06-05 00:09:52.000 │ DateTime64(3, 'Asia/Istanbul') │
└─────────────────────────┴────────────────────────────────┘
```
@ -100,13 +100,14 @@ SELECT toDateTime64(now(), 3, 'Asia/Istanbul') AS column, toTypeName(column) AS
``` sql
SELECT
toDateTime64(timestamp, 3, 'Europe/London') as lon_time,
toDateTime64(timestamp, 3, 'Asia/Istanbul') as mos_time
toDateTime64(timestamp, 3, 'Asia/Istanbul') as istanbul_time
FROM dt;
```
``` text
┌───────────────lon_time──┬────────────────mos_time─┐
│ 2019-01-01 00:00:00.000 │ 2019-01-01 03:00:00.000 │
┌────────────────lon_time─┬───────────istanbul_time─┐
│ 2019-01-01 00:00:00.123 │ 2019-01-01 03:00:00.123 │
│ 2019-01-01 00:00:00.123 │ 2019-01-01 03:00:00.123 │
│ 2018-12-31 21:00:00.000 │ 2019-01-01 00:00:00.000 │
└─────────────────────────┴─────────────────────────┘
```
@ -115,10 +116,9 @@ FROM dt;
- [Type conversion functions](../../sql-reference/functions/type-conversion-functions.md)
- [Functions for working with dates and times](../../sql-reference/functions/date-time-functions.md)
- [Functions for working with arrays](../../sql-reference/functions/array-functions.md)
- [The `date_time_input_format` setting](../../operations/settings/settings.md#settings-date_time_input_format)
- [The `date_time_output_format` setting](../../operations/settings/settings.md#settings-date_time_output_format)
- [The `date_time_input_format` setting](../../operations/settings/settings-formats.md#date_time_input_format)
- [The `date_time_output_format` setting](../../operations/settings/settings-formats.md#date_time_output_format)
- [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone)
- [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-datetime)
- [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-for-working-with-dates-and-times)
- [`Date` data type](../../sql-reference/data-types/date.md)
- [`DateTime` data type](../../sql-reference/data-types/datetime.md)

View File

@ -108,6 +108,7 @@ Result:
- [map()](../../sql-reference/functions/tuple-map-functions.md#function-map) function
- [CAST()](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) function
- [-Map combinator for Map datatype](../aggregate-functions/combinators.md#-map)
## Related content

View File

@ -2280,7 +2280,7 @@ This config consists of a list of regular expression tree nodes. Each node has t
- The value of an attribute may contain **back references**, referring to capture groups of the matched regular expression. In the example, the value of attribute `version` in the first node consists of a back-reference `\1` to capture group `(\d+[\.\d]*)` in the regular expression. Back-reference numbers range from 1 to 9 and are written as `$1` or `\1` (for number 1). The back reference is replaced by the matched capture group during query execution.
- **child nodes**: a list of children of a regexp tree node, each of which has its own attributes and (potentially) children nodes. String matching proceeds in a depth-first fashion. If a string matches a regexp node, the dictionary checks if it also matches the nodes' child nodes. If that is the case, the attributes of the deepest matching node are assigned. Attributes of a child node overwrite equally named attributes of parent nodes. The name of child nodes in YAML files can be arbitrary, e.g. `versions` in above example.
Regexp tree dictionaries only allow access using the functions `dictGet` and `dictGetOrDefault`.
Regexp tree dictionaries only allow access using the functions `dictGet`, `dictGetOrDefault`, and `dictGetAll`.
Example:
@ -2300,6 +2300,67 @@ In this case, we first match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)
With a powerful YAML configure file, we can use a regexp tree dictionaries as a user agent string parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh)
#### Collecting Attribute Values
Sometimes it is useful to return values from multiple regular expressions that matched, rather than just the value of a leaf node. In these cases, the specialized [`dictGetAll`](../../sql-reference/functions/ext-dict-functions.md#dictgetall) function can be used. If a node has an attribute value of type `T`, `dictGetAll` will return an `Array(T)` containing zero or more values.
By default, the number of matches returned per key is unbounded. A bound can be passed as an optional fourth argument to `dictGetAll`. The array is populated in _topological order_, meaning that child nodes come before parent nodes, and sibling nodes follow the ordering in the source.
Example:
```sql
CREATE DICTIONARY regexp_dict
(
regexp String,
tag String,
topological_index Int64,
captured Nullable(String),
parent String
)
PRIMARY KEY(regexp)
SOURCE(YAMLRegExpTree(PATH '/var/lib/clickhouse/user_files/regexp_tree.yaml'))
LAYOUT(regexp_tree)
LIFETIME(0)
```
```yaml
# /var/lib/clickhouse/user_files/regexp_tree.yaml
- regexp: 'clickhouse\.com'
tag: 'ClickHouse'
topological_index: 1
paths:
- regexp: 'clickhouse\.com/docs(.*)'
tag: 'ClickHouse Documentation'
topological_index: 0
captured: '\1'
parent: 'ClickHouse'
- regexp: '/docs(/|$)'
tag: 'Documentation'
topological_index: 2
- regexp: 'github.com'
tag: 'GitHub'
topological_index: 3
captured: 'NULL'
```
```sql
CREATE TABLE urls (url String) ENGINE=MergeTree ORDER BY url;
INSERT INTO urls VALUES ('clickhouse.com'), ('clickhouse.com/docs/en'), ('github.com/clickhouse/tree/master/docs');
SELECT url, dictGetAll('regexp_dict', ('tag', 'topological_index', 'captured', 'parent'), url, 2) FROM urls;
```
Result:
```text
┌─url────────────────────────────────────┬─dictGetAll('regexp_dict', ('tag', 'topological_index', 'captured', 'parent'), url, 2)─┐
│ clickhouse.com │ (['ClickHouse'],[1],[],[]) │
│ clickhouse.com/docs/en │ (['ClickHouse Documentation','ClickHouse'],[0,1],['/en'],['ClickHouse']) │
│ github.com/clickhouse/tree/master/docs │ (['Documentation','GitHub'],[2,3],[NULL],[]) │
└────────────────────────────────────────┴───────────────────────────────────────────────────────────────────────────────────────┘
```
### Use Regular Expression Tree Dictionary in ClickHouse Cloud
Above used `YAMLRegExpTree` source works in ClickHouse Open Source but not in ClickHouse Cloud. To use regexp tree dictionaries in ClickHouse could, first create a regexp tree dictionary from a YAML file locally in ClickHouse Open Source, then dump this dictionary into a CSV file using the `dictionary` table function and the [INTO OUTFILE](../statements/select/into-outfile.md) clause.

View File

@ -403,6 +403,84 @@ SELECT dictGetDescendants('hierarchy_flat_dictionary', number, 1) FROM system.nu
└────────────────────────────────────────────────────────────┘
```
## dictGetAll
Retrieves the attribute values of all nodes that matched each key in a [regular expression tree dictionary](../../sql-reference/dictionaries/index.md#regexp-tree-dictionary).
Besides returning values of type `Array(T)` instead of `T`, this function behaves similarly to [`dictGet`](#dictget-dictgetordefault-dictgetornull).
**Syntax**
``` sql
dictGetAll('dict_name', attr_names, id_expr[, limit])
```
**Arguments**
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `attr_names` — Name of the column of the dictionary, [String literal](../../sql-reference/syntax.md#syntax-string-literal), or tuple of column names, [Tuple](../../sql-reference/data-types/tuple.md)([String literal](../../sql-reference/syntax.md#syntax-string-literal)).
- `id_expr` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning array of dictionary key-type value or [Tuple](../../sql-reference/data-types/tuple.md)-type value depending on the dictionary configuration.
- `limit` - Maximum length for each value array returned. When truncating, child nodes are given precedence over parent nodes, and otherwise the defined list order for the regexp tree dictionary is respected. If unspecified, array length is unlimited.
**Returned value**
- If ClickHouse parses the attribute successfully in the attributes data type as defined in the dictionary, returns an array of dictionary attribute values that correspond to `id_expr` for each attribute specified by `attr_names`.
- If there is no key corresponding to `id_expr` in the dictionary, then an empty array is returned.
ClickHouse throws an exception if it cannot parse the value of the attribute or the value does not match the attribute data type.
**Example**
Consider the following regexp tree dictionary:
```sql
CREATE DICTIONARY regexp_dict
(
regexp String,
tag String
)
PRIMARY KEY(regexp)
SOURCE(YAMLRegExpTree(PATH '/var/lib/clickhouse/user_files/regexp_tree.yaml'))
LAYOUT(regexp_tree)
...
```
```yaml
# /var/lib/clickhouse/user_files/regexp_tree.yaml
- regexp: 'foo'
tag: 'foo_attr'
- regexp: 'bar'
tag: 'bar_attr'
- regexp: 'baz'
tag: 'baz_attr'
```
Get all matching values:
```sql
SELECT dictGetAll('regexp_dict', 'tag', 'foobarbaz');
```
```text
┌─dictGetAll('regexp_dict', 'tag', 'foobarbaz')─┐
│ ['foo_attr','bar_attr','baz_attr'] │
└───────────────────────────────────────────────┘
```
Get up to 2 matching values:
```sql
SELECT dictGetAll('regexp_dict', 'tag', 'foobarbaz', 2);
```
```text
┌─dictGetAll('regexp_dict', 'tag', 'foobarbaz', 2)─┐
│ ['foo_attr','bar_attr'] │
└──────────────────────────────────────────────────┘
```
## Other Functions
ClickHouse supports specialized functions that convert dictionary attribute values to a specific data type regardless of the dictionary configuration.

View File

@ -560,77 +560,6 @@ Result:
└───────────────────────────┘
```
## Entropy-learned hashing (experimental)
Entropy-learned hashing is disabled by default, to enable: `SET allow_experimental_hash_functions=1`.
Entropy-learned hashing is not a standalone hash function like `metroHash64`, `cityHash64`, `sipHash64` etc. Instead, it aims to preprocess
the data to be hashed in a way that a standalone hash function can be computed more efficiently while not compromising the hash quality,
i.e. the randomness of the hashes. For that, entropy-based hashing chooses a subset of the bytes in a training data set of Strings which has
the same randomness (entropy) as the original Strings. For example, if the Strings are in average 100 bytes long, and we pick a subset of 5
bytes, then a hash function will be 95% less expensive to evaluate. For details of the method, refer to [Entropy-Learned Hashing: Constant
Time Hashing with Controllable Uniformity](https://doi.org/10.1145/3514221.3517894).
Entropy-learned hashing has two phases:
1. A training phase on a representative but typically small set of Strings to be hashed. Training consists of two steps:
- Function `prepareTrainEntropyLearnedHash(data, id)` caches the training data in a global state under a given `id`. It returns dummy
value `0` on every row.
- Function `trainEntropyLearnedHash(id)` computes a minimal partial sub-key of the training data stored stored under `id` in the global
state. The cached training data in the global state is replaced by the partial key. Dummy value `0` is returned on every row.
2. An evaluation phase where hashes are computed using the previously calculated partial sub-keys. Function `entropyLearnedHash(data, id)`
hashes `data` using the partial subkey stored as `id`. CityHash64 is used as hash function.
The reason that the training phase comprises two steps is that ClickHouse processes data at chunk granularity but entropy-learned hashing
needs to process the entire training set at once.
Since functions `prepareTrainEntropyLearnedHash()` and `trainEntropyLearnedHash()` access global state, they should not be called in
parallel with the same `id`.
**Syntax**
``` sql
prepareTrainEntropyLearnedHash(data, id);
trainEntropyLearnedHash(id);
entropyLearnedHash(data, id);
```
**Example**
```sql
SET allow_experimental_hash_functions=1;
CREATE TABLE tab (col String) ENGINE=Memory;
INSERT INTO tab VALUES ('aa'), ('ba'), ('ca');
SELECT prepareTrainEntropyLearnedHash(col, 'id1') AS prepared FROM tab;
SELECT trainEntropyLearnedHash('id1') AS trained FROM tab;
SELECT entropyLearnedHash(col, 'id1') as hashes FROM tab;
```
Result:
``` response
┌─prepared─┐
│ 0 │
│ 0 │
│ 0 │
└──────────┘
┌─trained─┐
│ 0 │
│ 0 │
│ 0 │
└─────────┘
┌───────────────hashes─┐
│ 2603192927274642682 │
│ 4947675599669400333 │
│ 10783339242466472992 │
└──────────────────────┘
```
## metroHash64
Produces a 64-bit [MetroHash](http://www.jandrewrogers.com/2015/05/27/metrohash/) hash value.

View File

@ -10,7 +10,9 @@ There are at least\* two types of functions - regular functions (they are just c
In this section we discuss regular functions. For aggregate functions, see the section “Aggregate functions”.
\* - There is a third type of function that the arrayJoin function belongs to; table functions can also be mentioned separately.\*
:::note
There is a third type of function that the [arrayJoin function](/docs/en/sql-reference/functions/array-join.md) belongs to. And [table functions](/docs/en/sql-reference/table-functions/index.md) can also be mentioned separately.
:::
## Strong Typing

View File

@ -10,7 +10,7 @@ sidebar_label: INDEX
The following operations are available:
- `ALTER TABLE [db].table_name [ON CLUSTER cluster] ADD INDEX name expression TYPE type GRANULARITY value [FIRST|AFTER name]` - Adds index description to tables metadata.
- `ALTER TABLE [db].table_name [ON CLUSTER cluster] ADD INDEX name expression TYPE type [GRANULARITY value] [FIRST|AFTER name]` - Adds index description to tables metadata.
- `ALTER TABLE [db].table_name [ON CLUSTER cluster] DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations).

View File

@ -119,19 +119,35 @@ The compiled expression cache is enabled/disabled with the query/user/profile-le
Resets the [query cache](../../operations/query-cache.md).
```sql
SYSTEM DROP QUERY CACHE [ON CLUSTER cluster_name]
```
## FLUSH LOGS
Flushes buffered log messages to system tables, e.g. system.query_log. Mainly useful for debugging since most system tables have a default flush interval of 7.5 seconds.
This will also create system tables even if message queue is empty.
```sql
SYSTEM FLUSH LOGS [ON CLUSTER cluster_name]
```
## RELOAD CONFIG
Reloads ClickHouse configuration. Used when configuration is stored in ZooKeeper. Note that `SYSTEM RELOAD CONFIG` does not reload `USER` configuration stored in ZooKeeper, it only reloads `USER` configuration that is stored in `users.xml`. To reload all `USER` config use `SYSTEM RELOAD USERS`
```sql
SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name]
```
## RELOAD USERS
Reloads all access storages, including: users.xml, local disk access storage, replicated (in ZooKeeper) access storage.
```sql
SYSTEM RELOAD USERS [ON CLUSTER cluster_name]
```
## SHUTDOWN
Normally shuts down ClickHouse (like `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`)
@ -149,7 +165,7 @@ ClickHouse can manage [distributed](../../engines/table-engines/special/distribu
Disables background data distribution when inserting data into distributed tables.
``` sql
SYSTEM STOP DISTRIBUTED SENDS [db.]<distributed_table_name>
SYSTEM STOP DISTRIBUTED SENDS [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
### FLUSH DISTRIBUTED
@ -157,7 +173,7 @@ SYSTEM STOP DISTRIBUTED SENDS [db.]<distributed_table_name>
Forces ClickHouse to send data to cluster nodes synchronously. If any nodes are unavailable, ClickHouse throws an exception and stops query execution. You can retry the query until it succeeds, which will happen when all nodes are back online.
``` sql
SYSTEM FLUSH DISTRIBUTED [db.]<distributed_table_name>
SYSTEM FLUSH DISTRIBUTED [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
### START DISTRIBUTED SENDS
@ -165,7 +181,7 @@ SYSTEM FLUSH DISTRIBUTED [db.]<distributed_table_name>
Enables background data distribution when inserting data into distributed tables.
``` sql
SYSTEM START DISTRIBUTED SENDS [db.]<distributed_table_name>
SYSTEM START DISTRIBUTED SENDS [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
## Managing MergeTree Tables
@ -177,7 +193,7 @@ ClickHouse can manage background processes in [MergeTree](../../engines/table-en
Provides possibility to stop background merges for tables in the MergeTree family:
``` sql
SYSTEM STOP MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
SYSTEM STOP MERGES [ON CLUSTER cluster_name] [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
:::note
@ -189,7 +205,7 @@ SYSTEM STOP MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
Provides possibility to start background merges for tables in the MergeTree family:
``` sql
SYSTEM START MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
SYSTEM START MERGES [ON CLUSTER cluster_name] [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
### STOP TTL MERGES
@ -198,7 +214,7 @@ Provides possibility to stop background delete old data according to [TTL expres
Returns `Ok.` even if table does not exist or table has not MergeTree engine. Returns error when database does not exist:
``` sql
SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name]
SYSTEM STOP TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### START TTL MERGES
@ -207,7 +223,7 @@ Provides possibility to start background delete old data according to [TTL expre
Returns `Ok.` even if table does not exist. Returns error when database does not exist:
``` sql
SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name]
SYSTEM START TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### STOP MOVES
@ -216,7 +232,7 @@ Provides possibility to stop background move data according to [TTL table expres
Returns `Ok.` even if table does not exist. Returns error when database does not exist:
``` sql
SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
SYSTEM STOP MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### START MOVES
@ -225,7 +241,7 @@ Provides possibility to start background move data according to [TTL table expre
Returns `Ok.` even if table does not exist. Returns error when database does not exist:
``` sql
SYSTEM START MOVES [[db.]merge_tree_family_table_name]
SYSTEM START MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### SYSTEM UNFREEZE {#query_language-system-unfreeze}
@ -241,7 +257,7 @@ SYSTEM UNFREEZE WITH NAME <backup_name>
Wait until all asynchronously loading data parts of a table (outdated data parts) will became loaded.
``` sql
SYSTEM WAIT LOADING PARTS [db.]merge_tree_family_table_name
SYSTEM WAIT LOADING PARTS [ON CLUSTER cluster_name] [db.]merge_tree_family_table_name
```
## Managing ReplicatedMergeTree Tables
@ -254,7 +270,7 @@ Provides possibility to stop background fetches for inserted parts for tables in
Always returns `Ok.` regardless of the table engine and even if table or database does not exist.
``` sql
SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name]
SYSTEM STOP FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### START FETCHES
@ -263,7 +279,7 @@ Provides possibility to start background fetches for inserted parts for tables i
Always returns `Ok.` regardless of the table engine and even if table or database does not exist.
``` sql
SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name]
SYSTEM START FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### STOP REPLICATED SENDS
@ -271,7 +287,7 @@ SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name]
Provides possibility to stop background sends to other replicas in cluster for new inserted parts for tables in the `ReplicatedMergeTree` family:
``` sql
SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
SYSTEM STOP REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### START REPLICATED SENDS
@ -279,7 +295,7 @@ SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
Provides possibility to start background sends to other replicas in cluster for new inserted parts for tables in the `ReplicatedMergeTree` family:
``` sql
SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
SYSTEM START REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### STOP REPLICATION QUEUES
@ -287,7 +303,7 @@ SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
Provides possibility to stop background fetch tasks from replication queues which stored in Zookeeper for tables in the `ReplicatedMergeTree` family. Possible background tasks types - merges, fetches, mutation, DDL statements with ON CLUSTER clause:
``` sql
SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name]
SYSTEM STOP REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### START REPLICATION QUEUES
@ -295,7 +311,7 @@ SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name]
Provides possibility to start background fetch tasks from replication queues which stored in Zookeeper for tables in the `ReplicatedMergeTree` family. Possible background tasks types - merges, fetches, mutation, DDL statements with ON CLUSTER clause:
``` sql
SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name]
SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### SYNC REPLICA
@ -318,7 +334,7 @@ Provides possibility to reinitialize Zookeeper session's state for `ReplicatedMe
Initialization of replication queue based on ZooKeeper data happens in the same way as for `ATTACH TABLE` statement. For a short time, the table will be unavailable for any operations.
``` sql
SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name
SYSTEM RESTART REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name
```
### RESTORE REPLICA
@ -384,7 +400,7 @@ Provides possibility to reinitialize Zookeeper sessions state for all `Replicate
Allows to drop filesystem cache.
```sql
SYSTEM DROP FILESYSTEM CACHE
SYSTEM DROP FILESYSTEM CACHE [ON CLUSTER cluster_name]
```
### SYNC FILE CACHE
@ -396,5 +412,5 @@ It's too heavy and has potential for misuse.
Will do sync syscall.
```sql
SYSTEM SYNC FILE CACHE
SYSTEM SYNC FILE CACHE [ON CLUSTER cluster_name]
```

View File

@ -387,6 +387,23 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR
Формат CSV поддерживает вывод totals и extremes аналогично `TabSeparated`.
### CSV опции форматирования {#csv-format-settings}
- [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter) - символ, который будет считаться разделителем в CSV данных. Значение по умолчанию - `,`.
- [format_csv_allow_single_quotes](../operations/settings/settings.md#format_csv_allow_single_quotes) - разрешить строки в одинарных кавычках. Значение по умолчанию - `true`.
- [format_csv_allow_double_quotes](../operations/settings/settings.md#format_csv_allow_double_quotes) - разрешить строки в двойных кавычках. Значение по умолчанию - `true`.
- [format_csv_null_representation](../operations/settings/settings.md#format_tsv_null_representation) - пользовательское представление NULL в формате CSV. Значение по умолчанию - `\N`.
- [input_format_csv_empty_as_default](../operations/settings/settings.md#input_format_csv_empty_as_default) - рассматривать пустые поля в CSV в качестве значений по умолчанию. Значение по умолчанию - `true`. Для сложных выражений по умолчанию необходимо также включить [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#input_format_defaults_for_omitted_fields).
- [input_format_csv_enum_as_number](../operations/settings/settings.md#input_format_csv_enum_as_number) - рассматривать вставленные значения enum в форматах CSV как индексы enum. Значение по умолчанию - `false`.
- [input_format_csv_use_best_effort_in_schema_inference](../operations/settings/settings.md#input_format_csv_use_best_effort_in_schema_inference) - использовать некоторые твики и эвристики для вывода схемы в формате CSV. Если параметр отключен, все поля будут определяться как строки. Значение по умолчанию - `true`.
- [input_format_csv_arrays_as_nested_csv](../operations/settings/settings.md#input_format_csv_arrays_as_nested_csv) - при чтении массива из CSV ожидать, что его элементы были сериализованы во вложенный CSV и затем помещены в строку. Значение по умолчанию - `false`.
- [output_format_csv_crlf_end_of_line](../operations/settings/settings.md#output_format_csv_crlf_end_of_line) - если установлено значение true, конец строки в формате вывода CSV будет `\r\n` вместо `\n`. Значение по умолчанию - `false`.
- [input_format_csv_skip_first_lines](../operations/settings/settings.md#input_format_csv_skip_first_lines) - пропустить указанное количество строк в начале данных. Значение по умолчанию - `0`.
- [input_format_csv_detect_header](../operations/settings/settings.md#input_format_csv_detect_header) - обнаружить заголовок с именами и типами в формате CSV. Значение по умолчанию - `true`.
- [input_format_csv_trim_whitespaces](../operations/settings/settings.md#input_format_csv_trim_whitespaces) - удалить пробелы и символы табуляции из строк без кавычек.
Значение по умолчанию - `true`.
## CSVWithNames {#csvwithnames}
Выводит также заголовок, аналогично [TabSeparatedWithNames](#tabseparatedwithnames).

View File

@ -1589,6 +1589,24 @@ SELECT area/period FROM account_orders FORMAT JSON;
Символ, интерпретируемый как разделитель в данных формата CSV. По умолчанию — `,`.
## format_csv_allow_double_quotes {#format_csv_allow_double_quotes}
Если установлено значение true, разрешить строки в двойных кавычках.
Включено по умолчанию.
## input_format_csv_empty_as_default {#input_format_csv_empty_as_default}
Если включено, заменяет пустые поля ввода в CSV значениями по умолчанию. Для сложных выражений по умолчанию `input_format_defaults_for_omitted_fields` также должен быть включен.
Включено по умолчанию.
## input_format_csv_arrays_as_nested_csv {#input_format_csv_arrays_as_nested_csv}
При чтении массива из CSV ожидайте, что его элементы были сериализованы во вложенный CSV, а затем помещены в строку. Пример: "[""Hello"", ""world"", ""42"""" TV""]". Скобки вокруг массива могут быть опущены.
По умолчанию отключены.
## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null}
Для формата CSV включает или выключает парсинг неэкранированной строки `NULL` как литерала (синоним для `\N`)
@ -1665,6 +1683,50 @@ SELECT * FROM table_with_enum_column_for_csv_insert;
Использовать в качестве разделителя строк для CSV формата CRLF (DOS/Windows стиль) вместо LF (Unix стиль).
## input_format_csv_detect_header {#input_format_csv_detect_header}
Обнаружить заголовок с именами и типами в формате CSV.
Значение по умолчанию - `true`.
## input_format_csv_skip_first_lines {#input_format_csv_skip_first_lines}
Количество строк, пропускаемых в начале данных в формате ввода CSV.
Значение по умолчанию: `0`.
## input_format_csv_trim_whitespaces {#input_format_csv_trim_whitespaces}
Удалить пробелы и символы табуляции из строк без кавычек.
Значение по умолчанию: `true`.
**Примеры**
Запрос
```bash
echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true
```
Результат
```text
"string"
```
Запрос
```bash
echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=false
```
Результат
```text
" string "
```
## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line}
Использовать в качестве разделителя строк для TSV формата CRLF (DOC/Windows стиль) вместо LF (Unix стиль).

View File

@ -39,7 +39,7 @@ SELECT name, status FROM system.dictionaries;
**Синтаксис**
```sql
SYSTEM RELOAD MODELS
SYSTEM RELOAD MODELS [ON CLUSTER cluster_name]
```
## RELOAD MODEL {#query_language-system-reload-model}
@ -49,7 +49,7 @@ SYSTEM RELOAD MODELS
**Синтаксис**
```sql
SYSTEM RELOAD MODEL <model_path>
SYSTEM RELOAD MODEL [ON CLUSTER cluster_name] <model_path>
```
## RELOAD FUNCTIONS {#query_language-system-reload-functions}
@ -59,8 +59,8 @@ SYSTEM RELOAD MODEL <model_path>
**Синтаксис**
```sql
RELOAD FUNCTIONS
RELOAD FUNCTION function_name
RELOAD FUNCTIONS [ON CLUSTER cluster_name]
RELOAD FUNCTION function_name [ON CLUSTER cluster_name]
```
## DROP DNS CACHE {#query_language-system-drop-dns-cache}
@ -106,10 +106,18 @@ Cкомпилированные выражения используются ко
Записывает буферы логов в системные таблицы (например system.query_log). Позволяет не ждать 7.5 секунд при отладке.
Если буфер логов пустой, то этот запрос просто создаст системные таблицы.
```sql
SYSTEM FLUSH LOGS [ON CLUSTER cluster_name]
```
## RELOAD CONFIG {#query_language-system-reload-config}
Перечитывает конфигурацию настроек ClickHouse. Используется при хранении конфигурации в zookeeper.
```sql
SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name]
```
## SHUTDOWN {#query_language-system-shutdown}
Штатно завершает работу ClickHouse (аналог `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`)
@ -127,7 +135,7 @@ ClickHouse может оперировать [распределёнными](..
Отключает фоновую отправку при вставке данных в распределённые таблицы.
``` sql
SYSTEM STOP DISTRIBUTED SENDS [db.]<distributed_table_name>
SYSTEM STOP DISTRIBUTED SENDS [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
### FLUSH DISTRIBUTED {#query_language-system-flush-distributed}
@ -135,7 +143,7 @@ SYSTEM STOP DISTRIBUTED SENDS [db.]<distributed_table_name>
В синхронном режиме отправляет все данные на узлы кластера. Если какие-либо узлы недоступны, ClickHouse генерирует исключение и останавливает выполнение запроса. Такой запрос можно повторять до успешного завершения, что будет означать возвращение связанности с остальными узлами кластера.
``` sql
SYSTEM FLUSH DISTRIBUTED [db.]<distributed_table_name>
SYSTEM FLUSH DISTRIBUTED [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
### START DISTRIBUTED SENDS {#query_language-system-start-distributed-sends}
@ -143,7 +151,7 @@ SYSTEM FLUSH DISTRIBUTED [db.]<distributed_table_name>
Включает фоновую отправку при вставке данных в распределенные таблицы.
``` sql
SYSTEM START DISTRIBUTED SENDS [db.]<distributed_table_name>
SYSTEM START DISTRIBUTED SENDS [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
## Managing MergeTree Tables {#query-language-system-mergetree}
@ -155,7 +163,7 @@ ClickHouse может управлять фоновыми процессами
Позволяет остановить фоновые мержи для таблиц семейства MergeTree:
``` sql
SYSTEM STOP MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
SYSTEM STOP MERGES [ON CLUSTER cluster_name] [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
:::note
@ -166,7 +174,7 @@ SYSTEM STOP MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
Включает фоновые мержи для таблиц семейства MergeTree:
``` sql
SYSTEM START MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
SYSTEM START MERGES [ON CLUSTER cluster_name] [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
### STOP TTL MERGES {#query_language-stop-ttl-merges}
@ -175,7 +183,7 @@ SYSTEM START MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name
Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных:
``` sql
SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name]
SYSTEM STOP TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### START TTL MERGES {#query_language-start-ttl-merges}
@ -184,7 +192,7 @@ SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name]
Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных:
``` sql
SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name]
SYSTEM START TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### STOP MOVES {#query_language-stop-moves}
@ -193,7 +201,7 @@ SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name]
Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных:
``` sql
SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
SYSTEM STOP MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### START MOVES {#query_language-start-moves}
@ -202,7 +210,7 @@ SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных:
``` sql
SYSTEM START MOVES [[db.]merge_tree_family_table_name]
SYSTEM START MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### SYSTEM UNFREEZE {#query_language-system-unfreeze}
@ -223,7 +231,7 @@ ClickHouse может управлять фоновыми процессами
Всегда возвращает `Ok.` вне зависимости от типа таблицы и даже если таблица или база данных не существет.
``` sql
SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name]
SYSTEM STOP FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### START FETCHES {#query_language-system-start-fetches}
@ -232,7 +240,7 @@ SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name]
Всегда возвращает `Ok.` вне зависимости от типа таблицы и даже если таблица или база данных не существет.
``` sql
SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name]
SYSTEM START FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### STOP REPLICATED SENDS {#query_language-system-start-replicated-sends}
@ -240,7 +248,7 @@ SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name]
Позволяет остановить фоновые процессы отсылки новых вставленных кусков данных другим репликам в кластере для таблиц семейства `ReplicatedMergeTree`:
``` sql
SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
SYSTEM STOP REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### START REPLICATED SENDS {#query_language-system-start-replicated-sends}
@ -248,7 +256,7 @@ SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
Позволяет запустить фоновые процессы отсылки новых вставленных кусков данных другим репликам в кластере для таблиц семейства `ReplicatedMergeTree`:
``` sql
SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
SYSTEM START REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### STOP REPLICATION QUEUES {#query_language-system-stop-replication-queues}
@ -256,7 +264,7 @@ SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
Останавливает фоновые процессы разбора заданий из очереди репликации которая хранится в Zookeeper для таблиц семейства `ReplicatedMergeTree`. Возможные типы заданий - merges, fetches, mutation, DDL запросы с ON CLUSTER:
``` sql
SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name]
SYSTEM STOP REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### START REPLICATION QUEUES {#query_language-system-start-replication-queues}
@ -264,7 +272,7 @@ SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name]
Запускает фоновые процессы разбора заданий из очереди репликации которая хранится в Zookeeper для таблиц семейства `ReplicatedMergeTree`. Возможные типы заданий - merges, fetches, mutation, DDL запросы с ON CLUSTER:
``` sql
SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name]
SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### SYNC REPLICA {#query_language-system-sync-replica}
@ -287,7 +295,7 @@ SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT | LIGHT
Инициализация очереди репликации на основе данных ZooKeeper происходит так же, как при `ATTACH TABLE`. Некоторое время таблица будет недоступна для любых операций.
``` sql
SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name
SYSTEM RESTART REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name
```
### RESTORE REPLICA {#query_language-system-restore-replica}

View File

@ -71,10 +71,18 @@ SYSTEM DROP REPLICA 'replica_name' FROM ZKPATH '/path/to/table/in/zk';
将日志信息缓冲数据刷入系统表例如system.query_log。调试时允许等待不超过7.5秒。当信息队列为空时,会创建系统表。
```sql
SYSTEM FLUSH LOGS [ON CLUSTER cluster_name]
```
## RELOAD CONFIG {#query_language-system-reload-config}
重新加载ClickHouse的配置。用于当配置信息存放在ZooKeeper时。
```sql
SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name]
```
## SHUTDOWN {#query_language-system-shutdown}
关闭ClickHouse服务类似于 `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`
@ -93,7 +101,7 @@ ClickHouse可以管理 [distribute](../../engines/table-engines/special/distribu
当向分布式表插入数据时,禁用后台的分布式数据分发。
``` sql
SYSTEM STOP DISTRIBUTED SENDS [db.]<distributed_table_name>
SYSTEM STOP DISTRIBUTED SENDS [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
### FLUSH DISTRIBUTED {#query_language-system-flush-distributed}
@ -101,7 +109,7 @@ SYSTEM STOP DISTRIBUTED SENDS [db.]<distributed_table_name>
强制让ClickHouse同步向集群节点同步发送数据。如果有节点失效ClickHouse抛出异常并停止插入操作。当所有节点都恢复上线时你可以重试之前的操作直到成功执行。
``` sql
SYSTEM FLUSH DISTRIBUTED [db.]<distributed_table_name>
SYSTEM FLUSH DISTRIBUTED [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
### START DISTRIBUTED SENDS {#query_language-system-start-distributed-sends}
@ -109,7 +117,7 @@ SYSTEM FLUSH DISTRIBUTED [db.]<distributed_table_name>
当向分布式表插入数据时,允许后台的分布式数据分发。
``` sql
SYSTEM START DISTRIBUTED SENDS [db.]<distributed_table_name>
SYSTEM START DISTRIBUTED SENDS [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
## Managing MergeTree Tables {#query-language-system-mergetree}
@ -121,7 +129,7 @@ ClickHouse可以管理 [MergeTree](../../engines/table-engines/mergetree-family/
为MergeTree系列引擎表停止后台合并操作。
``` sql
SYSTEM STOP MERGES [[db.]merge_tree_family_table_name]
SYSTEM STOP MERGES [ON CLUSTER cluster_name] [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
:::note
@ -133,7 +141,7 @@ SYSTEM STOP MERGES [[db.]merge_tree_family_table_name]
为MergeTree系列引擎表启动后台合并操作。
``` sql
SYSTEM START MERGES [[db.]merge_tree_family_table_name]
SYSTEM START MERGES [ON CLUSTER cluster_name] [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
### STOP TTL MERGES {#query_language-stop-ttl-merges}
@ -142,7 +150,7 @@ SYSTEM START MERGES [[db.]merge_tree_family_table_name]
不管表存在与否,都返回 `OK.`。当数据库不存在时返回错误。
``` sql
SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name]
SYSTEM STOP TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### START TTL MERGES {#query_language-start-ttl-merges}
@ -151,7 +159,7 @@ SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name]
``` sql
SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name]
SYSTEM START TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### STOP MOVES {#query_language-stop-moves}
@ -160,7 +168,7 @@ SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name]
``` sql
SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
SYSTEM STOP MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### START MOVES {#query_language-start-moves}
@ -169,7 +177,7 @@ SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
``` sql
SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
SYSTEM START MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
### SYSTEM UNFREEZE {#query_language-system-unfreeze}
@ -190,7 +198,7 @@ SYSTEM UNFREEZE WITH NAME <backup_name>
不管表引擎类型如何或表/数据库是否存,都返回 `OK.`
``` sql
SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name]
SYSTEM STOP FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### START FETCHES {#query_language-system-start-fetches}
@ -199,7 +207,7 @@ SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name]
不管表引擎类型如何或表/数据库是否存,都返回 `OK.`
``` sql
SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name]
SYSTEM START FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### STOP REPLICATED SENDS {#query_language-system-start-replicated-sends}
@ -207,7 +215,7 @@ SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name]
停止通过后台分发 `ReplicatedMergeTree`系列引擎表中新插入的数据块到集群的其它副本节点。
``` sql
SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
SYSTEM STOP REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### START REPLICATED SENDS {#query_language-system-start-replicated-sends}
@ -215,7 +223,7 @@ SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
启动通过后台分发 `ReplicatedMergeTree`系列引擎表中新插入的数据块到集群的其它副本节点。
``` sql
SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
SYSTEM START REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### STOP REPLICATION QUEUES {#query_language-system-stop-replication-queues}
@ -224,7 +232,7 @@ SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]
停止从Zookeeper中获取 `ReplicatedMergeTree`系列表的复制队列的后台任务。可能的后台任务类型包含merges, fetches, mutation带有 `ON CLUSTER`的ddl语句
``` sql
SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name]
SYSTEM STOP REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### START REPLICATION QUEUES {#query_language-system-start-replication-queues}
@ -232,7 +240,7 @@ SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name]
启动从Zookeeper中获取 `ReplicatedMergeTree`系列表的复制队列的后台任务。可能的后台任务类型包含merges, fetches, mutation带有 `ON CLUSTER`的ddl语句
``` sql
SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name]
SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### SYNC REPLICA {#query_language-system-sync-replica}
@ -250,7 +258,7 @@ SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT | LIGHT
``` sql
SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name
SYSTEM RESTART REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name
```
### RESTART REPLICAS {#query_language-system-restart-replicas}

View File

@ -55,6 +55,8 @@ option (ENABLE_CLICKHOUSE_KEEPER "ClickHouse alternative to ZooKeeper" ${ENABLE_
option (ENABLE_CLICKHOUSE_KEEPER_CONVERTER "Util allows to convert ZooKeeper logs and snapshots into clickhouse-keeper snapshot" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_KEEPER_CLIENT "ClickHouse Keeper Client" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_SU "A tool similar to 'su'" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_DISKS "A tool to manage disks" ${ENABLE_CLICKHOUSE_ALL})
@ -169,6 +171,13 @@ else()
message(STATUS "ClickHouse keeper-converter mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_KEEPER_CLIENT)
message(STATUS "ClickHouse keeper-client mode: ON")
else()
message(STATUS "ClickHouse keeper-client mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_DISKS)
message(STATUS "Clickhouse disks mode: ON")
else()
@ -237,6 +246,10 @@ if (ENABLE_CLICKHOUSE_KEEPER_CONVERTER)
add_subdirectory (keeper-converter)
endif()
if (ENABLE_CLICKHOUSE_KEEPER_CLIENT)
add_subdirectory (keeper-client)
endif()
if (ENABLE_CLICKHOUSE_ODBC_BRIDGE)
add_subdirectory (odbc-bridge)
endif ()
@ -301,6 +314,9 @@ endif()
if (ENABLE_CLICKHOUSE_KEEPER_CONVERTER)
clickhouse_target_link_split_lib(clickhouse keeper-converter)
endif()
if (ENABLE_CLICKHOUSE_KEEPER_CLIENT)
clickhouse_target_link_split_lib(clickhouse keeper-client)
endif()
if (ENABLE_CLICKHOUSE_INSTALL)
clickhouse_target_link_split_lib(clickhouse install)
endif ()
@ -392,6 +408,11 @@ if (ENABLE_CLICKHOUSE_KEEPER_CONVERTER)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-keeper-converter" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-keeper-converter)
endif ()
if (ENABLE_CLICKHOUSE_KEEPER_CLIENT)
add_custom_target (clickhouse-keeper-client ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-keeper-client DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-keeper-client" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-keeper-client)
endif ()
if (ENABLE_CLICKHOUSE_DISKS)
add_custom_target (clickhouse-disks ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-disks DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-disks" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)

View File

@ -17,6 +17,7 @@
#cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE
#cmakedefine01 ENABLE_CLICKHOUSE_LIBRARY_BRIDGE
#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER
#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CLIENT
#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CONVERTER
#cmakedefine01 ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER
#cmakedefine01 ENABLE_CLICKHOUSE_SU

View File

@ -0,0 +1,9 @@
set (CLICKHOUSE_KEEPER_CLIENT_SOURCES KeeperClient.cpp Parser.cpp Commands.cpp)
set (CLICKHOUSE_KEEPER_CLIENT_LINK
PRIVATE
boost::program_options
dbms
)
clickhouse_program_add(keeper-client)

View File

@ -0,0 +1,196 @@
#include "Commands.h"
#include "KeeperClient.h"
namespace DB
{
bool LSCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String arg;
if (!parseKeeperPath(pos, expected, arg))
return true;
node->args.push_back(std::move(arg));
return true;
}
void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
String path;
if (!query->args.empty())
path = client->getAbsolutePath(query->args[0].safeGet<String>());
else
path = client->cwd;
for (const auto & child : client->zookeeper->getChildren(path))
std::cout << child << " ";
std::cout << "\n";
}
bool CDCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String arg;
if (!parseKeeperPath(pos, expected, arg))
return true;
node->args.push_back(std::move(arg));
return true;
}
void CDCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
if (query->args.empty())
return;
auto new_path = client->getAbsolutePath(query->args[0].safeGet<String>());
if (!client->zookeeper->exists(new_path))
std::cerr << "Path " << new_path << " does not exists\n";
else
client->cwd = new_path;
}
bool SetCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String arg;
if (!parseKeeperPath(pos, expected, arg))
return false;
node->args.push_back(std::move(arg));
if (!parseKeeperArg(pos, expected, arg))
return false;
node->args.push_back(std::move(arg));
ASTPtr version;
if (ParserNumber{}.parse(pos, version, expected))
node->args.push_back(version->as<ASTLiteral &>().value);
return true;
}
void SetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
if (query->args.size() == 2)
client->zookeeper->set(client->getAbsolutePath(query->args[0].safeGet<String>()), query->args[1].safeGet<String>());
else
client->zookeeper->set(
client->getAbsolutePath(query->args[0].safeGet<String>()),
query->args[1].safeGet<String>(),
static_cast<Int32>(query->args[2].safeGet<Int64>()));
}
bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String arg;
if (!parseKeeperPath(pos, expected, arg))
return false;
node->args.push_back(std::move(arg));
if (!parseKeeperArg(pos, expected, arg))
return false;
node->args.push_back(std::move(arg));
int mode = zkutil::CreateMode::Persistent;
if (ParserKeyword{"PERSISTENT"}.ignore(pos, expected))
mode = zkutil::CreateMode::Persistent;
else if (ParserKeyword{"EPHEMERAL"}.ignore(pos, expected))
mode = zkutil::CreateMode::Ephemeral;
else if (ParserKeyword{"EPHEMERAL SEQUENTIAL"}.ignore(pos, expected))
mode = zkutil::CreateMode::EphemeralSequential;
else if (ParserKeyword{"PERSISTENT SEQUENTIAL"}.ignore(pos, expected))
mode = zkutil::CreateMode::PersistentSequential;
node->args.push_back(mode);
return true;
}
void CreateCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
client->zookeeper->create(
client->getAbsolutePath(query->args[0].safeGet<String>()),
query->args[1].safeGet<String>(),
static_cast<int>(query->args[2].safeGet<Int64>()));
}
bool GetCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String arg;
if (!parseKeeperPath(pos, expected, arg))
return false;
node->args.push_back(std::move(arg));
return true;
}
void GetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
std::cout << client->zookeeper->get(client->getAbsolutePath(query->args[0].safeGet<String>())) << "\n";
}
bool RMCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String arg;
if (!parseKeeperPath(pos, expected, arg))
return false;
node->args.push_back(std::move(arg));
return true;
}
void RMCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
client->zookeeper->remove(client->getAbsolutePath(query->args[0].safeGet<String>()));
}
bool RMRCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String arg;
if (!parseKeeperPath(pos, expected, arg))
return false;
node->args.push_back(std::move(arg));
return true;
}
void RMRCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
String path = client->getAbsolutePath(query->args[0].safeGet<String>());
client->askConfirmation("You are going to recursively delete path " + path,
[client, path]{ client->zookeeper->removeRecursive(path); });
}
bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptr<ASTKeeperQuery> & /* node */, Expected & /* expected */) const
{
return true;
}
void HelpCommand::execute(const ASTKeeperQuery * /* query */, KeeperClient * /* client */) const
{
for (const auto & pair : KeeperClient::commands)
std::cout << pair.second->getHelpMessage() << "\n";
}
bool FourLetterWordCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
expected.add(pos, "four-letter-word command");
if (pos->type != TokenType::BareWord)
return false;
String cmd(pos->begin, pos->end);
if (cmd.size() != 4)
return false;
++pos;
node->args.push_back(std::move(cmd));
return true;
}
void FourLetterWordCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
std::cout << client->executeFourLetterCommand(query->args[0].safeGet<String>()) << "\n";
}
}

View File

@ -0,0 +1,131 @@
#pragma once
#include "Parser.h"
namespace DB
{
class KeeperClient;
class IKeeperClientCommand
{
public:
static const String name;
virtual bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const = 0;
virtual void execute(const ASTKeeperQuery * query, KeeperClient * client) const = 0;
virtual String getHelpMessage() const = 0;
virtual String getName() const = 0;
virtual ~IKeeperClientCommand() = default;
};
using Command = std::shared_ptr<IKeeperClientCommand>;
class LSCommand : public IKeeperClientCommand
{
String getName() const override { return "ls"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "ls [path] -- Lists the nodes for the given path (default: cwd)"; }
};
class CDCommand : public IKeeperClientCommand
{
String getName() const override { return "cd"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "cd [path] -- Change the working path (default `.`)"; }
};
class SetCommand : public IKeeperClientCommand
{
String getName() const override { return "set"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override
{
return "set <path> <value> [version] -- Updates the node's value. Only update if version matches (default: -1)";
}
};
class CreateCommand : public IKeeperClientCommand
{
String getName() const override { return "create"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "create <path> <value> -- Creates new node"; }
};
class GetCommand : public IKeeperClientCommand
{
String getName() const override { return "get"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "get <path> -- Returns the node's value"; }
};
class RMCommand : public IKeeperClientCommand
{
String getName() const override { return "rm"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "remove <path> -- Remove the node"; }
};
class RMRCommand : public IKeeperClientCommand
{
String getName() const override { return "rmr"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "rmr <path> -- Recursively deletes path. Confirmation required"; }
};
class HelpCommand : public IKeeperClientCommand
{
String getName() const override { return "help"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "help -- Prints this message"; }
};
class FourLetterWordCommand : public IKeeperClientCommand
{
String getName() const override { return "flwc"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "flwc <command> -- Executes four-letter-word command"; }
};
}

View File

@ -0,0 +1,343 @@
#include "KeeperClient.h"
#include "Commands.h"
#include <Client/ReplxxLineReader.h>
#include <Client/ClientBase.h>
#include <Common/EventNotifier.h>
#include <Common/filesystemHelpers.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Parsers/parseQuery.h>
#include <Poco/Util/HelpFormatter.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
String KeeperClient::executeFourLetterCommand(const String & command)
{
/// We need to create a new socket every time because ZooKeeper forcefully shuts down the connection after a four-letter-word command.
Poco::Net::StreamSocket socket;
socket.connect(Poco::Net::SocketAddress{zk_args.hosts[0]}, zk_args.connection_timeout_ms * 1000);
socket.setReceiveTimeout(zk_args.operation_timeout_ms * 1000);
socket.setSendTimeout(zk_args.operation_timeout_ms * 1000);
socket.setNoDelay(true);
ReadBufferFromPocoSocket in(socket);
WriteBufferFromPocoSocket out(socket);
out.write(command.data(), command.size());
out.next();
String result;
readStringUntilEOF(result, in);
in.next();
return result;
}
std::vector<String> KeeperClient::getCompletions(const String & prefix) const
{
Tokens tokens(prefix.data(), prefix.data() + prefix.size(), 0, false);
IParser::Pos pos(tokens, 0);
if (pos->type != TokenType::BareWord)
return registered_commands_and_four_letter_words;
++pos;
if (pos->isEnd())
return registered_commands_and_four_letter_words;
ParserToken{TokenType::Whitespace}.ignore(pos);
std::vector<String> result;
String string_path;
Expected expected;
if (!parseKeeperPath(pos, expected, string_path))
string_path = cwd;
if (!pos->isEnd())
return result;
fs::path path = string_path;
String parent_path;
if (string_path.ends_with("/"))
parent_path = getAbsolutePath(string_path);
else
parent_path = getAbsolutePath(path.parent_path());
try
{
for (const auto & child : zookeeper->getChildren(parent_path))
result.push_back(child);
}
catch (Coordination::Exception &) {}
std::sort(result.begin(), result.end());
return result;
}
void KeeperClient::askConfirmation(const String & prompt, std::function<void()> && callback)
{
std::cout << prompt << " Continue?\n";
need_confirmation = true;
confirmation_callback = callback;
}
fs::path KeeperClient::getAbsolutePath(const String & relative) const
{
String result;
if (relative.starts_with('/'))
result = fs::weakly_canonical(relative);
else
result = fs::weakly_canonical(cwd / relative);
if (result.ends_with('/') && result.size() > 1)
result.pop_back();
return result;
}
void KeeperClient::loadCommands(std::vector<Command> && new_commands)
{
for (const auto & command : new_commands)
{
String name = command->getName();
commands.insert({name, command});
registered_commands_and_four_letter_words.push_back(std::move(name));
}
for (const auto & command : four_letter_word_commands)
registered_commands_and_four_letter_words.push_back(command);
std::sort(registered_commands_and_four_letter_words.begin(), registered_commands_and_four_letter_words.end());
}
void KeeperClient::defineOptions(Poco::Util::OptionSet & options)
{
Poco::Util::Application::defineOptions(options);
options.addOption(
Poco::Util::Option("help", "", "show help and exit")
.binding("help"));
options.addOption(
Poco::Util::Option("host", "h", "server hostname. default `localhost`")
.argument("host")
.binding("host"));
options.addOption(
Poco::Util::Option("port", "p", "server port. default `2181`")
.argument("port")
.binding("port"));
options.addOption(
Poco::Util::Option("query", "q", "will execute given query, then exit.")
.argument("query")
.binding("query"));
options.addOption(
Poco::Util::Option("connection-timeout", "", "set connection timeout in seconds. default 10s.")
.argument("connection-timeout")
.binding("connection-timeout"));
options.addOption(
Poco::Util::Option("session-timeout", "", "set session timeout in seconds. default 10s.")
.argument("session-timeout")
.binding("session-timeout"));
options.addOption(
Poco::Util::Option("operation-timeout", "", "set operation timeout in seconds. default 10s.")
.argument("operation-timeout")
.binding("operation-timeout"));
options.addOption(
Poco::Util::Option("history-file", "", "set path of history file. default `~/.keeper-client-history`")
.argument("history-file")
.binding("history-file"));
options.addOption(
Poco::Util::Option("log-level", "", "set log level")
.argument("log-level")
.binding("log-level"));
}
void KeeperClient::initialize(Poco::Util::Application & /* self */)
{
suggest.setCompletionsCallback(
[&](const String & prefix, size_t /* prefix_length */) { return getCompletions(prefix); });
loadCommands({
std::make_shared<LSCommand>(),
std::make_shared<CDCommand>(),
std::make_shared<SetCommand>(),
std::make_shared<CreateCommand>(),
std::make_shared<GetCommand>(),
std::make_shared<RMCommand>(),
std::make_shared<RMRCommand>(),
std::make_shared<HelpCommand>(),
std::make_shared<FourLetterWordCommand>(),
});
String home_path;
const char * home_path_cstr = getenv("HOME"); // NOLINT(concurrency-mt-unsafe)
if (home_path_cstr)
home_path = home_path_cstr;
if (config().has("history-file"))
history_file = config().getString("history-file");
else
history_file = home_path + "/.keeper-client-history";
if (!history_file.empty() && !fs::exists(history_file))
{
try
{
FS::createFile(history_file);
}
catch (const ErrnoException & e)
{
if (e.getErrno() != EEXIST)
throw;
}
}
Poco::Logger::root().setLevel(config().getString("log-level", "error"));
EventNotifier::init();
}
void KeeperClient::executeQuery(const String & query)
{
std::vector<String> queries;
boost::algorithm::split(queries, query, boost::is_any_of(";"));
for (const auto & query_text : queries)
{
if (!query_text.empty())
processQueryText(query_text);
}
}
bool KeeperClient::processQueryText(const String & text)
{
if (exit_strings.find(text) != exit_strings.end())
return false;
try
{
if (need_confirmation)
{
need_confirmation = false;
if (text.size() == 1 && (text == "y" || text == "Y"))
confirmation_callback();
return true;
}
KeeperParser parser;
String message;
const char * begin = text.data();
ASTPtr res = tryParseQuery(parser, begin, begin + text.size(), message, true, "", false, 0, 0, false);
if (!res)
{
std::cerr << message << "\n";
return true;
}
auto * query = res->as<ASTKeeperQuery>();
auto command = KeeperClient::commands.find(query->command);
command->second->execute(query, this);
}
catch (Coordination::Exception & err)
{
std::cerr << err.message() << "\n";
}
return true;
}
void KeeperClient::runInteractive()
{
LineReader::Patterns query_extenders = {"\\"};
LineReader::Patterns query_delimiters = {};
ReplxxLineReader lr(suggest, history_file, false, query_extenders, query_delimiters, {});
lr.enableBracketedPaste();
while (true)
{
String prompt;
if (need_confirmation)
prompt = "[y/n] ";
else
prompt = cwd.string() + " :) ";
auto input = lr.readLine(prompt, ":-] ");
if (input.empty())
break;
if (!processQueryText(input))
break;
}
}
int KeeperClient::main(const std::vector<String> & /* args */)
{
if (config().hasOption("help"))
{
Poco::Util::HelpFormatter help_formatter(KeeperClient::options());
auto header_str = fmt::format("{} [OPTION]\n", commandName());
help_formatter.setHeader(header_str);
help_formatter.format(std::cout);
return 0;
}
auto host = config().getString("host", "localhost");
auto port = config().getString("port", "2181");
zk_args.hosts = {host + ":" + port};
zk_args.connection_timeout_ms = config().getInt("connection-timeout", 10) * 1000;
zk_args.session_timeout_ms = config().getInt("session-timeout", 10) * 1000;
zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000;
zookeeper = std::make_unique<zkutil::ZooKeeper>(zk_args);
if (config().has("query"))
executeQuery(config().getString("query"));
else
runInteractive();
return 0;
}
}
int mainEntryClickHouseKeeperClient(int argc, char ** argv)
{
try
{
DB::KeeperClient client;
client.init(argc, argv);
return client.run();
}
catch (const DB::Exception & e)
{
std::cerr << DB::getExceptionMessage(e, false) << std::endl;
return 1;
}
catch (const boost::program_options::error & e)
{
std::cerr << "Bad arguments: " << e.what() << std::endl;
return DB::ErrorCodes::BAD_ARGUMENTS;
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << std::endl;
return 1;
}
}

View File

@ -0,0 +1,69 @@
#pragma once
#include "Parser.h"
#include "Commands.h"
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Client/LineReader.h>
#include <IO/ReadBufferFromPocoSocket.h>
#include <IO/WriteBufferFromPocoSocket.h>
#include <Parsers/ASTLiteral.h>
#include <Poco/Net/StreamSocket.h>
#include <Poco/Util/Application.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
static const NameSet four_letter_word_commands
{
"ruok", "mntr", "srvr", "stat", "srst", "conf",
"cons", "crst", "envi", "dirs", "isro", "wchs",
"wchc", "wchp", "dump", "csnp", "lgif", "rqld",
};
class KeeperClient: public Poco::Util::Application
{
public:
KeeperClient() = default;
void initialize(Poco::Util::Application & self) override;
int main(const std::vector<String> & args) override;
void defineOptions(Poco::Util::OptionSet & options) override;
fs::path getAbsolutePath(const String & relative) const;
void askConfirmation(const String & prompt, std::function<void()> && callback);
String executeFourLetterCommand(const String & command);
zkutil::ZooKeeperPtr zookeeper;
std::filesystem::path cwd = "/";
std::function<void()> confirmation_callback;
inline static std::map<String, Command> commands;
protected:
void runInteractive();
bool processQueryText(const String & text);
void executeQuery(const String & query);
void loadCommands(std::vector<Command> && new_commands);
std::vector<String> getCompletions(const String & prefix) const;
String history_file;
LineReader::Suggest suggest;
zkutil::ZooKeeperArgs zk_args;
bool need_confirmation = false;
std::vector<String> registered_commands_and_four_letter_words;
};
}

View File

@ -0,0 +1,94 @@
#include "Parser.h"
#include "KeeperClient.h"
namespace DB
{
bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result)
{
expected.add(pos, getTokenName(TokenType::BareWord));
if (pos->type == TokenType::BareWord)
{
result = String(pos->begin, pos->end);
++pos;
ParserToken{TokenType::Whitespace}.ignore(pos);
return true;
}
bool status = parseIdentifierOrStringLiteral(pos, expected, result);
ParserToken{TokenType::Whitespace}.ignore(pos);
return status;
}
bool parseKeeperPath(IParser::Pos & pos, Expected & expected, String & path)
{
expected.add(pos, "path");
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
return parseIdentifierOrStringLiteral(pos, expected, path);
String result;
while (pos->type == TokenType::BareWord || pos->type == TokenType::Slash || pos->type == TokenType::Dot)
{
result.append(pos->begin, pos->end);
++pos;
}
ParserToken{TokenType::Whitespace}.ignore(pos);
if (result.empty())
return false;
path = result;
return true;
}
bool KeeperParser::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto query = std::make_shared<ASTKeeperQuery>();
for (const auto & pair : KeeperClient::commands)
expected.add(pos, pair.first.data());
for (const auto & flwc : four_letter_word_commands)
expected.add(pos, flwc.data());
if (pos->type != TokenType::BareWord)
return false;
String command_name(pos->begin, pos->end);
Command command;
auto iter = KeeperClient::commands.find(command_name);
if (iter == KeeperClient::commands.end())
{
if (command_name.size() == 4)
{
/// Treat it like four-letter command
/// Since keeper server can potentially have different version we don't want to match this command with embedded list
command = std::make_shared<FourLetterWordCommand>();
command_name = command->getName();
/// We also don't move the position, so the command will be parsed as an argument
}
else
return false;
}
else
{
command = iter->second;
++pos;
ParserToken{TokenType::Whitespace}.ignore(pos);
}
query->command = command_name;
if (!command->parse(pos, query, expected))
return false;
ParserToken{TokenType::Whitespace}.ignore(pos);
node = query;
return true;
}
}

View File

@ -0,0 +1,36 @@
#pragma once
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/IAST.h>
#include <Parsers/IParserBase.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
namespace DB
{
bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result);
bool parseKeeperPath(IParser::Pos & pos, Expected & expected, String & path);
class ASTKeeperQuery : public IAST
{
public:
String getID(char) const override { return "KeeperQuery"; }
ASTPtr clone() const override { return std::make_shared<ASTKeeperQuery>(*this); }
String command;
std::vector<Field> args;
};
class KeeperParser : public IParserBase
{
protected:
const char * getName() const override { return "Keeper client query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -130,15 +130,31 @@ void LocalServer::initialize(Poco::Util::Application & self)
});
#endif
IOThreadPool::initialize(
getIOThreadPool().initialize(
config().getUInt("max_io_thread_pool_size", 100),
config().getUInt("max_io_thread_pool_free_size", 0),
config().getUInt("io_thread_pool_queue_size", 10000));
OutdatedPartsLoadingThreadPool::initialize(
config().getUInt("max_outdated_parts_loading_thread_pool_size", 16),
const size_t active_parts_loading_threads = config().getUInt("max_active_parts_loading_thread_pool_size", 64);
getActivePartsLoadingThreadPool().initialize(
active_parts_loading_threads,
0, // We don't need any threads one all the parts will be loaded
config().getUInt("max_outdated_parts_loading_thread_pool_size", 16));
active_parts_loading_threads);
const size_t outdated_parts_loading_threads = config().getUInt("max_outdated_parts_loading_thread_pool_size", 32);
getOutdatedPartsLoadingThreadPool().initialize(
outdated_parts_loading_threads,
0, // We don't need any threads one all the parts will be loaded
outdated_parts_loading_threads);
getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(active_parts_loading_threads);
const size_t cleanup_threads = config().getUInt("max_parts_cleaning_thread_pool_size", 128);
getPartsCleaningThreadPool().initialize(
cleanup_threads,
0, // We don't need any threads one all the parts will be deleted
cleanup_threads);
}

View File

@ -62,6 +62,9 @@ int mainEntryClickHouseKeeper(int argc, char ** argv);
#if ENABLE_CLICKHOUSE_KEEPER_CONVERTER
int mainEntryClickHouseKeeperConverter(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_KEEPER_CLIENT
int mainEntryClickHouseKeeperClient(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER
int mainEntryClickHouseStaticFilesDiskUploader(int argc, char ** argv);
#endif
@ -133,6 +136,9 @@ std::pair<const char *, MainFunc> clickhouse_applications[] =
#if ENABLE_CLICKHOUSE_KEEPER_CONVERTER
{"keeper-converter", mainEntryClickHouseKeeperConverter},
#endif
#if ENABLE_CLICKHOUSE_KEEPER_CLIENT
{"keeper-client", mainEntryClickHouseKeeperClient},
#endif
#if ENABLE_CLICKHOUSE_INSTALL
{"install", mainEntryClickHouseInstall},
{"start", mainEntryClickHouseStart},

View File

@ -683,21 +683,36 @@ try
});
#endif
IOThreadPool::initialize(
getIOThreadPool().initialize(
server_settings.max_io_thread_pool_size,
server_settings.max_io_thread_pool_free_size,
server_settings.io_thread_pool_queue_size);
BackupsIOThreadPool::initialize(
getBackupsIOThreadPool().initialize(
server_settings.max_backups_io_thread_pool_size,
server_settings.max_backups_io_thread_pool_free_size,
server_settings.backups_io_thread_pool_queue_size);
OutdatedPartsLoadingThreadPool::initialize(
getActivePartsLoadingThreadPool().initialize(
server_settings.max_active_parts_loading_thread_pool_size,
0, // We don't need any threads once all the parts will be loaded
server_settings.max_active_parts_loading_thread_pool_size);
getOutdatedPartsLoadingThreadPool().initialize(
server_settings.max_outdated_parts_loading_thread_pool_size,
0, // We don't need any threads one all the parts will be loaded
0, // We don't need any threads once all the parts will be loaded
server_settings.max_outdated_parts_loading_thread_pool_size);
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(
server_settings.max_active_parts_loading_thread_pool_size
);
getPartsCleaningThreadPool().initialize(
server_settings.max_parts_cleaning_thread_pool_size,
0, // We don't need any threads one all the parts will be deleted
server_settings.max_parts_cleaning_thread_pool_size);
/// Initialize global local cache for remote filesystem.
if (config().has("local_cache_for_remote_fs"))
{
@ -1226,6 +1241,36 @@ try
global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size);
global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size);
getIOThreadPool().reloadConfiguration(
server_settings.max_io_thread_pool_size,
server_settings.max_io_thread_pool_free_size,
server_settings.io_thread_pool_queue_size);
getBackupsIOThreadPool().reloadConfiguration(
server_settings.max_backups_io_thread_pool_size,
server_settings.max_backups_io_thread_pool_free_size,
server_settings.backups_io_thread_pool_queue_size);
getActivePartsLoadingThreadPool().reloadConfiguration(
server_settings.max_active_parts_loading_thread_pool_size,
0, // We don't need any threads once all the parts will be loaded
server_settings.max_active_parts_loading_thread_pool_size);
getOutdatedPartsLoadingThreadPool().reloadConfiguration(
server_settings.max_outdated_parts_loading_thread_pool_size,
0, // We don't need any threads once all the parts will be loaded
server_settings.max_outdated_parts_loading_thread_pool_size);
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(
server_settings.max_active_parts_loading_thread_pool_size
);
getPartsCleaningThreadPool().reloadConfiguration(
server_settings.max_parts_cleaning_thread_pool_size,
0, // We don't need any threads one all the parts will be deleted
server_settings.max_parts_cleaning_thread_pool_size);
if (config->has("resources"))
{
global_context->getResourceManager()->updateConfiguration(*config);

View File

@ -1,3 +1,10 @@
[env]
CFLAGS = "@RUST_CFLAGS@"
CXXFLAGS = "@RUST_CXXFLAGS@"
[build]
rustflags = @RUSTFLAGS@
rustdocflags = @RUSTFLAGS@
[unstable]
@RUST_CARGO_BUILD_STD@

View File

@ -8,8 +8,6 @@ extern "C" {
char *blake3_apply_shim(const char *begin, uint32_t _size, uint8_t *out_char_data);
char *blake3_apply_shim_msan_compat(const char *begin, uint32_t size, uint8_t *out_char_data);
void blake3_free_char_pointer(char *ptr_to_free);
} // extern "C"

View File

@ -3,7 +3,6 @@ extern crate libc;
use std::ffi::{CStr, CString};
use std::os::raw::c_char;
use std::mem;
#[no_mangle]
pub unsafe extern "C" fn blake3_apply_shim(
@ -24,30 +23,6 @@ pub unsafe extern "C" fn blake3_apply_shim(
std::ptr::null_mut()
}
#[no_mangle]
pub unsafe extern "C" fn blake3_apply_shim_msan_compat(
mut begin: *const c_char,
size: u32,
out_char_data: *mut u8,
) -> *mut c_char {
if begin.is_null() {
let err_str = CString::new("input was a null pointer").unwrap();
return err_str.into_raw();
}
libc::memset(out_char_data as *mut libc::c_void, 0, mem::size_of::<u8>());
let mut hasher = blake3::Hasher::new();
let mut vec = Vec::<u8>::new();
for _ in 0..size {
vec.push(*begin as u8);
begin = begin.add(1);
}
let input_res = vec.as_mut_slice();
hasher.update(input_res);
let mut reader = hasher.finalize_xof();
reader.fill(std::slice::from_raw_parts_mut(out_char_data, blake3::OUT_LEN));
std::ptr::null_mut()
}
// Freeing memory according to docs: https://doc.rust-lang.org/std/ffi/struct.CString.html#method.into_raw
#[no_mangle]
pub unsafe extern "C" fn blake3_free_char_pointer(ptr_to_free: *mut c_char) {

View File

@ -14,8 +14,18 @@ macro(configure_rustc)
set(RUST_CFLAGS "${RUST_CFLAGS} --sysroot ${CMAKE_SYSROOT}")
endif()
set(RUSTFLAGS "[]")
set(RUST_CARGO_BUILD_STD "")
# For more info: https://doc.rust-lang.org/beta/unstable-book/compiler-flags/sanitizer.html#memorysanitizer
if (SANITIZE STREQUAL "memory")
set(RUST_CARGO_BUILD_STD "build-std = [\"std\", \"panic_abort\", \"core\", \"alloc\"]")
set(RUSTFLAGS "[\"-Zsanitizer=memory\", \"-Zsanitizer-memory-track-origins\"]")
endif()
message(STATUS "RUST_CFLAGS: ${RUST_CFLAGS}")
message(STATUS "RUST_CXXFLAGS: ${RUST_CXXFLAGS}")
message(STATUS "RUSTFLAGS: ${RUSTFLAGS}")
message(STATUS "RUST_CARGO_BUILD_STD: ${RUST_CARGO_BUILD_STD}")
# NOTE: requires RW access for the source dir
configure_file("${CMAKE_CURRENT_SOURCE_DIR}/.cargo/config.toml.in" "${CMAKE_CURRENT_SOURCE_DIR}/.cargo/config.toml" @ONLY)

View File

@ -2,6 +2,7 @@
#include <Access/Quota.h>
#include <Access/RowPolicy.h>
#include <Access/User.h>
#include <Access/Role.h>
#include <Access/SettingsProfile.h>
#include <Access/AccessControl.h>
#include <Access/resolveSetting.h>
@ -13,6 +14,7 @@
#include <Core/Settings.h>
#include <Interpreters/executeQuery.h>
#include <Parsers/Access/ASTGrantQuery.h>
#include <Parsers/Access/ASTRolesOrUsersSet.h>
#include <Parsers/Access/ParserGrantQuery.h>
#include <Parsers/parseQuery.h>
#include <Poco/Util/AbstractConfiguration.h>
@ -52,11 +54,64 @@ namespace
UUID generateID(const IAccessEntity & entity) { return generateID(entity.getType(), entity.getName()); }
template <typename T>
void parseGrant(T & entity, const String & string_query, const std::unordered_set<UUID> & allowed_role_ids)
{
ParserGrantQuery parser;
parser.setParseWithoutGrantees();
String error_message;
const char * pos = string_query.data();
auto ast = tryParseQuery(parser, pos, pos + string_query.size(), error_message, false, "", false, 0, 0);
if (!ast)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse grant query. Error: {}", error_message);
auto & query = ast->as<ASTGrantQuery &>();
if (query.roles && query.is_revoke)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Roles can't be revoked in config file");
if (!query.cluster.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Can't grant on cluster using config file");
if (query.grantees)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "You can't specify grantees in query using config file");
for (auto & element : query.access_rights_elements)
{
if (query.is_revoke)
entity.access.revoke(element);
else
entity.access.grant(element);
}
if (query.roles)
{
std::vector<UUID> roles_to_grant;
roles_to_grant.reserve(query.roles->size());
for (const auto & role_name : query.roles->names)
{
auto role_id = generateID(AccessEntityType::ROLE, role_name);
if (!allowed_role_ids.contains(role_id))
throw Exception(ErrorCodes::THERE_IS_NO_PROFILE, "Role {} was not found", role_name);
roles_to_grant.push_back(role_id);
}
if (query.admin_option)
entity.granted_roles.grantWithAdminOption(roles_to_grant);
else
entity.granted_roles.grant(roles_to_grant);
}
}
UserPtr parseUser(
const Poco::Util::AbstractConfiguration & config,
const String & user_name,
const std::unordered_set<UUID> & allowed_profile_ids,
const std::unordered_set<UUID> & allowed_role_ids,
bool allow_no_password,
bool allow_plaintext_password)
{
@ -241,37 +296,8 @@ namespace
if (grant_queries)
{
ParserGrantQuery parser;
parser.parseWithoutGrantees();
for (const auto & string_query : *grant_queries)
{
String error_message;
const char * pos = string_query.data();
auto ast = tryParseQuery(parser, pos, pos + string_query.size(), error_message, false, "", false, 0, 0);
if (!ast)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse grant query. Error: {}", error_message);
auto & query = ast->as<ASTGrantQuery &>();
if (query.roles)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Roles can't be granted in config file");
if (!query.cluster.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Can't grant on cluster using config file");
if (query.grantees)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "You can't specify grantees in query using config file");
for (auto & element : query.access_rights_elements)
{
if (query.is_revoke)
user->access.revoke(element);
else
user->access.grant(element);
}
}
parseGrant(*user, string_query, allowed_role_ids);
}
else
{
@ -321,6 +347,7 @@ namespace
std::vector<AccessEntityPtr> parseUsers(
const Poco::Util::AbstractConfiguration & config,
const std::unordered_set<UUID> & allowed_profile_ids,
const std::unordered_set<UUID> & allowed_role_ids,
bool allow_no_password,
bool allow_plaintext_password)
{
@ -333,7 +360,7 @@ namespace
{
try
{
users.push_back(parseUser(config, user_name, allowed_profile_ids, allow_no_password, allow_plaintext_password));
users.push_back(parseUser(config, user_name, allowed_profile_ids, allowed_role_ids, allow_no_password, allow_plaintext_password));
}
catch (Exception & e)
{
@ -345,6 +372,55 @@ namespace
return users;
}
RolePtr parseRole(
const Poco::Util::AbstractConfiguration & config,
const String & role_name,
const std::unordered_set<UUID> & allowed_role_ids)
{
auto role = std::make_shared<Role>();
role->setName(role_name);
String role_config = "roles." + role_name;
const auto grants_config = role_config + ".grants";
if (config.has(grants_config))
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(grants_config, keys);
for (const auto & key : keys)
{
const auto query = config.getString(grants_config + "." + key);
parseGrant(*role, query, allowed_role_ids);
}
}
return role;
}
std::vector<AccessEntityPtr> parseRoles(
const Poco::Util::AbstractConfiguration & config,
const std::unordered_set<UUID> & allowed_role_ids)
{
Poco::Util::AbstractConfiguration::Keys role_names;
config.keys("roles", role_names);
std::vector<AccessEntityPtr> roles;
roles.reserve(role_names.size());
for (const auto & role_name : role_names)
{
try
{
roles.push_back(parseRole(config, role_name, allowed_role_ids));
}
catch (Exception & e)
{
e.addMessage(fmt::format("while parsing roles '{}' in users configuration file", role_name));
throw;
}
}
return roles;
}
QuotaPtr parseQuota(const Poco::Util::AbstractConfiguration & config, const String & quota_name, const std::vector<UUID> & user_ids)
{
@ -635,14 +711,16 @@ namespace
return profiles;
}
std::unordered_set<UUID> getAllowedSettingsProfileIDs(const Poco::Util::AbstractConfiguration & config)
std::unordered_set<UUID> getAllowedIDs(
const Poco::Util::AbstractConfiguration & config,
const String & configuration_key,
const AccessEntityType type)
{
Poco::Util::AbstractConfiguration::Keys profile_names;
config.keys("profiles", profile_names);
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(configuration_key, keys);
std::unordered_set<UUID> ids;
for (const auto & profile_name : profile_names)
ids.emplace(generateID(AccessEntityType::SETTINGS_PROFILE, profile_name));
for (const auto & key : keys)
ids.emplace(generateID(type, key));
return ids;
}
}
@ -693,12 +771,13 @@ void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfigu
{
try
{
auto allowed_profile_ids = getAllowedSettingsProfileIDs(config);
auto allowed_profile_ids = getAllowedIDs(config, "profiles", AccessEntityType::SETTINGS_PROFILE);
auto allowed_role_ids = getAllowedIDs(config, "roles", AccessEntityType::ROLE);
bool no_password_allowed = access_control.isNoPasswordAllowed();
bool plaintext_password_allowed = access_control.isPlaintextPasswordAllowed();
std::vector<std::pair<UUID, AccessEntityPtr>> all_entities;
for (const auto & entity : parseUsers(config, allowed_profile_ids, no_password_allowed, plaintext_password_allowed))
for (const auto & entity : parseUsers(config, allowed_profile_ids, allowed_role_ids, no_password_allowed, plaintext_password_allowed))
all_entities.emplace_back(generateID(*entity), entity);
for (const auto & entity : parseQuotas(config))
all_entities.emplace_back(generateID(*entity), entity);
@ -706,6 +785,8 @@ void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfigu
all_entities.emplace_back(generateID(*entity), entity);
for (const auto & entity : parseSettingsProfiles(config, allowed_profile_ids, access_control))
all_entities.emplace_back(generateID(*entity), entity);
for (const auto & entity : parseRoles(config, allowed_role_ids))
all_entities.emplace_back(generateID(*entity), entity);
memory_storage.setAll(all_entities);
}
catch (Exception & e)

View File

@ -141,6 +141,13 @@ public:
nested_func->merge(place, rhs, arena);
}
bool isAbleToParallelizeMerge() const override { return nested_func->isAbleToParallelizeMerge(); }
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override
{
nested_func->merge(place, rhs, thread_pool, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> version) const override
{
nested_func->serialize(place, buf, version);

View File

@ -110,6 +110,13 @@ public:
nested_func->merge(place, rhs, arena);
}
bool isAbleToParallelizeMerge() const override { return nested_func->isAbleToParallelizeMerge(); }
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override
{
nested_func->merge(place, rhs, thread_pool, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> version) const override
{
nested_func->serialize(place, buf, version);

View File

@ -148,6 +148,13 @@ public:
nested_function->merge(nestedPlace(place), nestedPlace(rhs), arena);
}
bool isAbleToParallelizeMerge() const override { return nested_function->isAbleToParallelizeMerge(); }
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override
{
nested_function->merge(nestedPlace(place), nestedPlace(rhs), thread_pool, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> version) const override
{
bool flag = getFlag(place);

View File

@ -91,6 +91,13 @@ public:
nested_func->merge(place, rhs, arena);
}
bool isAbleToParallelizeMerge() const override { return nested_func->isAbleToParallelizeMerge(); }
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override
{
nested_func->merge(place, rhs, thread_pool, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> version) const override
{
nested_func->serialize(place, buf, version);

View File

@ -117,7 +117,10 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const
else
{
const auto & table_storage_id = table_node->getStorageID();
column_identifier_parts = { table_storage_id.getDatabaseName(), table_storage_id.getTableName() };
if (table_storage_id.hasDatabase() && options.qualify_indentifiers_with_database)
column_identifier_parts = { table_storage_id.getDatabaseName(), table_storage_id.getTableName() };
else
column_identifier_parts = { table_storage_id.getTableName() };
}
}
}

View File

@ -187,10 +187,13 @@ public:
/// Identifiers are fully qualified (`database.table.column`), otherwise names are just column names (`column`)
bool fully_qualified_identifiers = true;
/// Identifiers are qualified but database name is not added (`table.column`) if set to false.
bool qualify_indentifiers_with_database = true;
};
/// Convert query tree to AST
ASTPtr toAST(const ConvertToASTOptions & options = { .add_cast_for_constants = true, .fully_qualified_identifiers = true }) const;
ASTPtr toAST(const ConvertToASTOptions & options = { .add_cast_for_constants = true, .fully_qualified_identifiers = true, .qualify_indentifiers_with_database = true }) const;
/// Convert query tree to AST and then format it for error message.
String formatConvertedASTForErrorMessage() const;

View File

@ -10,9 +10,10 @@
namespace DB
{
LambdaNode::LambdaNode(Names argument_names_, QueryTreeNodePtr expression_)
LambdaNode::LambdaNode(Names argument_names_, QueryTreeNodePtr expression_, DataTypePtr result_type_)
: IQueryTreeNode(children_size)
, argument_names(std::move(argument_names_))
, result_type(std::move(result_type_))
{
auto arguments_list_node = std::make_shared<ListNode>();
auto & nodes = arguments_list_node->getNodes();
@ -63,7 +64,7 @@ void LambdaNode::updateTreeHashImpl(HashState & state) const
QueryTreeNodePtr LambdaNode::cloneImpl() const
{
return std::make_shared<LambdaNode>(argument_names, getExpression());
return std::make_shared<LambdaNode>(argument_names, getExpression(), result_type);
}
ASTPtr LambdaNode::toASTImpl(const ConvertToASTOptions & options) const

View File

@ -35,7 +35,7 @@ class LambdaNode final : public IQueryTreeNode
{
public:
/// Initialize lambda with argument names and lambda body expression
explicit LambdaNode(Names argument_names_, QueryTreeNodePtr expression_);
explicit LambdaNode(Names argument_names_, QueryTreeNodePtr expression_, DataTypePtr result_type_ = {});
/// Get argument names
const Names & getArgumentNames() const

View File

@ -4767,13 +4767,14 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
auto * table_node = in_second_argument->as<TableNode>();
auto * table_function_node = in_second_argument->as<TableFunctionNode>();
if (table_node && dynamic_cast<StorageSet *>(table_node->getStorage().get()) != nullptr)
if (table_node)
{
/// If table is already prepared set, we do not replace it with subquery
/// If table is already prepared set, we do not replace it with subquery.
/// If table is not a StorageSet, we'll create plan to build set in the Planner.
}
else if (table_node || table_function_node)
else if (table_function_node)
{
const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot();
const auto & storage_snapshot = table_function_node->getStorageSnapshot();
auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary));
size_t columns_to_select_size = columns_to_select.size();

View File

@ -202,15 +202,16 @@ bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const
return is_subquery == rhs_typed.is_subquery &&
is_cte == rhs_typed.is_cte &&
cte_name == rhs_typed.cte_name &&
projection_columns == rhs_typed.projection_columns &&
is_distinct == rhs_typed.is_distinct &&
is_limit_with_ties == rhs_typed.is_limit_with_ties &&
is_group_by_with_totals == rhs_typed.is_group_by_with_totals &&
is_group_by_with_rollup == rhs_typed.is_group_by_with_rollup &&
is_group_by_with_cube == rhs_typed.is_group_by_with_cube &&
is_group_by_with_grouping_sets == rhs_typed.is_group_by_with_grouping_sets &&
is_group_by_all == rhs_typed.is_group_by_all;
is_group_by_all == rhs_typed.is_group_by_all &&
cte_name == rhs_typed.cte_name &&
projection_columns == rhs_typed.projection_columns &&
settings_changes == rhs_typed.settings_changes;
}
void QueryNode::updateTreeHashImpl(HashState & state) const
@ -239,6 +240,18 @@ void QueryNode::updateTreeHashImpl(HashState & state) const
state.update(is_group_by_with_cube);
state.update(is_group_by_with_grouping_sets);
state.update(is_group_by_all);
state.update(settings_changes.size());
for (const auto & setting_change : settings_changes)
{
state.update(setting_change.name.size());
state.update(setting_change.name);
auto setting_change_value_dump = setting_change.value.dump();
state.update(setting_change_value_dump.size());
state.update(setting_change_value_dump);
}
}
QueryTreeNodePtr QueryNode::cloneImpl() const
@ -256,6 +269,7 @@ QueryTreeNodePtr QueryNode::cloneImpl() const
result_query_node->is_group_by_all = is_group_by_all;
result_query_node->cte_name = cte_name;
result_query_node->projection_columns = projection_columns;
result_query_node->settings_changes = settings_changes;
return result_query_node;
}

View File

@ -91,6 +91,11 @@ ASTPtr TableNode::toASTImpl(const ConvertToASTOptions & /* options */) const
if (!temporary_table_name.empty())
return std::make_shared<ASTTableIdentifier>(temporary_table_name);
// In case of cross-replication we don't know what database is used for the table.
// `storage_id.hasDatabase()` can return false only on the initiator node.
// Each shard will use the default database (in the case of cross-replication shards may have different defaults).
if (!storage_id.hasDatabase())
return std::make_shared<ASTTableIdentifier>(storage_id.getTableName());
return std::make_shared<ASTTableIdentifier>(storage_id.getDatabaseName(), storage_id.getTableName());
}

View File

@ -161,7 +161,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s
/* dest_key= */ blob_path[0],
request_settings,
object_attributes,
threadPoolCallbackRunner<void>(BackupsIOThreadPool::get(), "BackupReaderS3"),
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupReaderS3"),
/* for_disk_s3= */ true);
return file_size;
@ -212,7 +212,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src
fs::path(s3_uri.key) / path_in_backup,
request_settings,
{},
threadPoolCallbackRunner<void>(BackupsIOThreadPool::get(), "BackupWriterS3"));
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWriterS3"));
return; /// copied!
}
}
@ -224,7 +224,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src
void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length)
{
copyDataToS3File(create_read_buffer, start_pos, length, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, {},
threadPoolCallbackRunner<void>(BackupsIOThreadPool::get(), "BackupWriterS3"));
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWriterS3"));
}
BackupWriterS3::~BackupWriterS3() = default;
@ -258,7 +258,7 @@ std::unique_ptr<WriteBuffer> BackupWriterS3::writeFile(const String & file_name)
DBMS_DEFAULT_BUFFER_SIZE,
request_settings,
std::nullopt,
threadPoolCallbackRunner<void>(BackupsIOThreadPool::get(), "BackupWriterS3"),
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWriterS3"),
write_settings);
}

View File

@ -346,6 +346,7 @@ target_link_libraries(clickhouse_common_io
PUBLIC
boost::program_options
boost::system
boost::context
ch_contrib::cityhash
ch_contrib::re2
ch_contrib::re2_st

View File

@ -90,14 +90,6 @@ namespace CurrentMetrics
namespace DB
{
static const NameSet exit_strings
{
"exit", "quit", "logout", "учше", "йгше", "дщпщге",
"exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж",
"q", "й", "\\q", "\\Q", "\\й", "\\Й", ":q", "Жй"
};
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
@ -286,7 +278,7 @@ public:
static Int32 cancelled_status() { return exit_after_signals.load(); }
};
/// This signal handler is set only for SIGINT.
/// This signal handler is set for SIGINT and SIGQUIT.
void interruptSignalHandler(int signum)
{
if (QueryInterruptHandler::try_stop())
@ -325,6 +317,9 @@ void ClientBase::setupSignalHandler()
if (sigaction(SIGINT, &new_act, nullptr))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
if (sigaction(SIGQUIT, &new_act, nullptr))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
}

View File

@ -24,6 +24,14 @@ namespace po = boost::program_options;
namespace DB
{
static const NameSet exit_strings
{
"exit", "quit", "logout", "учше", "йгше", "дщпщге",
"exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж",
"q", "й", "\\q", "\\Q", "\\й", "\\Й", ":q", "Жй"
};
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;

View File

@ -138,11 +138,6 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
socket->impl()->error(err); // Throws an exception
socket->setBlocking(true);
#if USE_SSL
if (static_cast<bool>(secure))
static_cast<Poco::Net::SecureStreamSocket *>(socket.get())->completeHandshake();
#endif
}
else
{

View File

@ -81,18 +81,36 @@ replxx::Replxx::completions_t LineReader::Suggest::getCompletions(const String &
std::lock_guard lock(mutex);
Words to_search;
bool no_case = false;
/// Only perform case sensitive completion when the prefix string contains any uppercase characters
if (std::none_of(prefix.begin(), prefix.end(), [](char32_t x) { return iswupper(static_cast<wint_t>(x)); }))
{
to_search = words_no_case;
no_case = true;
}
else
to_search = words;
if (custom_completions_callback)
{
auto new_words = custom_completions_callback(prefix, prefix_length);
assert(std::is_sorted(new_words.begin(), new_words.end()));
addNewWords(to_search, new_words, std::less<std::string>{});
}
if (no_case)
range = std::equal_range(
words_no_case.begin(), words_no_case.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched)
to_search.begin(), to_search.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched)
{
return strncasecmp(s.data(), prefix_searched.data(), prefix_length) < 0;
});
else
range = std::equal_range(words.begin(), words.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched)
{
return strncmp(s.data(), prefix_searched.data(), prefix_length) < 0;
});
range = std::equal_range(
to_search.begin(), to_search.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched)
{
return strncmp(s.data(), prefix_searched.data(), prefix_length) < 0;
});
return replxx::Replxx::completions_t(range.first, range.second);
}

View File

@ -18,15 +18,20 @@ public:
struct Suggest
{
using Words = std::vector<std::string>;
using Callback = std::function<Words(const String & prefix, size_t prefix_length)>;
/// Get vector for the matched range of words if any.
replxx::Replxx::completions_t getCompletions(const String & prefix, size_t prefix_length);
void addWords(Words && new_words);
void setCompletionsCallback(Callback && callback) { custom_completions_callback = callback; }
private:
Words words TSA_GUARDED_BY(mutex);
Words words_no_case TSA_GUARDED_BY(mutex);
Callback custom_completions_callback = nullptr;
std::mutex mutex;
};

View File

@ -313,6 +313,11 @@ MutableColumnPtr ColumnLowCardinality::cloneResized(size_t size) const
MutableColumnPtr ColumnLowCardinality::cloneNullable() const
{
auto res = cloneFinalized();
/* Compact required not to share dictionary.
* If `shared` flag is not set `cloneFinalized` will return shallow copy
* and `nestedToNullable` will mutate source column.
*/
assert_cast<ColumnLowCardinality &>(*res).compactInplace();
assert_cast<ColumnLowCardinality &>(*res).nestedToNullable();
return res;
}

View File

@ -48,3 +48,16 @@ TEST(ColumnLowCardinality, Insert)
testLowCardinalityNumberInsert<Float32>(std::make_shared<DataTypeFloat32>());
testLowCardinalityNumberInsert<Float64>(std::make_shared<DataTypeFloat64>());
}
TEST(ColumnLowCardinality, Clone)
{
auto data_type = std::make_shared<DataTypeInt32>();
auto low_cardinality_type = std::make_shared<DataTypeLowCardinality>(data_type);
auto column = low_cardinality_type->createColumn();
ASSERT_FALSE(assert_cast<const ColumnLowCardinality &>(*column).nestedIsNullable());
auto nullable_column = assert_cast<const ColumnLowCardinality &>(*column).cloneNullable();
ASSERT_TRUE(assert_cast<const ColumnLowCardinality &>(*nullable_column).nestedIsNullable());
ASSERT_FALSE(assert_cast<const ColumnLowCardinality &>(*column).nestedIsNullable());
}

View File

@ -69,13 +69,23 @@ AsynchronousMetrics::AsynchronousMetrics(
/// CGroups v2
openFileIfExists("/sys/fs/cgroup/memory.max", cgroupmem_limit_in_bytes);
openFileIfExists("/sys/fs/cgroup/memory.current", cgroupmem_usage_in_bytes);
if (cgroupmem_limit_in_bytes)
{
openFileIfExists("/sys/fs/cgroup/memory.current", cgroupmem_usage_in_bytes);
}
openFileIfExists("/sys/fs/cgroup/cpu.max", cgroupcpu_max);
/// CGroups v1
if (!cgroupmem_limit_in_bytes)
{
openFileIfExists("/sys/fs/cgroup/memory/memory.limit_in_bytes", cgroupmem_limit_in_bytes);
if (!cgroupmem_usage_in_bytes)
openFileIfExists("/sys/fs/cgroup/memory/memory.usage_in_bytes", cgroupmem_usage_in_bytes);
}
if (!cgroupcpu_max)
{
openFileIfExists("/sys/fs/cgroup/cpu/cpu.cfs_period_us", cgroupcpu_cfs_period);
openFileIfExists("/sys/fs/cgroup/cpu/cpu.cfs_quota_us", cgroupcpu_cfs_quota);
}
openSensors();
openBlockDevices();
@ -926,6 +936,61 @@ void AsynchronousMetrics::update(TimePoint update_time)
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
if (cgroupcpu_max)
{
try {
cgroupcpu_max->rewind();
uint64_t quota = 0;
uint64_t period = 0;
std::string line;
readText(line, *cgroupcpu_max);
auto space = line.find(' ');
if (line.rfind("max", space) == std::string::npos)
{
auto field1 = line.substr(0, space);
quota = std::stoull(field1);
}
if (space != std::string::npos)
{
auto field2 = line.substr(space + 1);
period = std::stoull(field2);
}
new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."};
new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."};
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
else if (cgroupcpu_cfs_quota && cgroupcpu_cfs_period)
{
try {
cgroupcpu_cfs_quota->rewind();
cgroupcpu_cfs_period->rewind();
uint64_t quota = 0;
uint64_t period = 0;
tryReadText(quota, *cgroupcpu_cfs_quota);
tryReadText(period, *cgroupcpu_cfs_period);
new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."};
new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."};
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
if (meminfo)
{
try

View File

@ -110,6 +110,9 @@ private:
std::optional<ReadBufferFromFilePRead> cgroupmem_limit_in_bytes;
std::optional<ReadBufferFromFilePRead> cgroupmem_usage_in_bytes;
std::optional<ReadBufferFromFilePRead> cgroupcpu_cfs_period;
std::optional<ReadBufferFromFilePRead> cgroupcpu_cfs_quota;
std::optional<ReadBufferFromFilePRead> cgroupcpu_max;
std::vector<std::unique_ptr<ReadBufferFromFilePRead>> thermal;

View File

@ -131,8 +131,14 @@
M(DistributedInsertThreadsActive, "Number of threads used for INSERT into Distributed running a task.") \
M(StorageS3Threads, "Number of threads in the StorageS3 thread pool.") \
M(StorageS3ThreadsActive, "Number of threads in the StorageS3 thread pool running a task.") \
M(ObjectStorageS3Threads, "Number of threads in the S3ObjectStorage thread pool.") \
M(ObjectStorageS3ThreadsActive, "Number of threads in the S3ObjectStorage thread pool running a task.") \
M(ObjectStorageAzureThreads, "Number of threads in the AzureObjectStorage thread pool.") \
M(ObjectStorageAzureThreadsActive, "Number of threads in the AzureObjectStorage thread pool running a task.") \
M(MergeTreePartsLoaderThreads, "Number of threads in the MergeTree parts loader thread pool.") \
M(MergeTreePartsLoaderThreadsActive, "Number of threads in the MergeTree parts loader thread pool running a task.") \
M(MergeTreeOutdatedPartsLoaderThreads, "Number of threads in the threadpool for loading Outdated data parts.") \
M(MergeTreeOutdatedPartsLoaderThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \
M(MergeTreePartsCleanerThreads, "Number of threads in the MergeTree parts cleaner thread pool.") \
M(MergeTreePartsCleanerThreadsActive, "Number of threads in the MergeTree parts cleaner thread pool running a task.") \
M(SystemReplicasThreads, "Number of threads in the system.replicas thread pool.") \
@ -145,8 +151,10 @@
M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool.") \
M(OutdatedPartsLoadingThreads, "Number of threads in the threadpool for loading Outdated data parts.") \
M(OutdatedPartsLoadingThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \
M(DistributedBytesToInsert, "Number of pending bytes to process for asynchronous insertion into Distributed tables. Number of bytes for every shard is summed.") \
M(BrokenDistributedBytesToInsert, "Number of bytes for asynchronous insertion into Distributed tables that has been marked as broken. Number of bytes for every shard is summed.") \
M(DistributedFilesToInsert, "Number of pending files to process for asynchronous insertion into Distributed tables. Number of files for every shard is summed.") \
M(BrokenDistributedFilesToInsert, "Number of files for asynchronous insertion into Distributed tables that has been marked as broken. This metric will starts from 0 on start. Number of files for every shard is summed.") \
M(BrokenDistributedFilesToInsert, "Number of files for asynchronous insertion into Distributed tables that has been marked as broken. Number of files for every shard is summed.") \
M(TablesToDropQueueSize, "Number of dropped tables, that are waiting for background data removal.") \
M(MaxDDLEntryID, "Max processed DDL entry of DDLWorker.") \
M(MaxPushedDDLEntryID, "Max DDL entry of DDLWorker that pushed to zookeeper.") \

View File

@ -124,7 +124,7 @@ SpanHolder::SpanHolder(std::string_view _operation_name, SpanKind _kind)
this->start_time_us
= std::chrono::duration_cast<std::chrono::microseconds>(std::chrono::system_clock::now().time_since_epoch()).count();
/// Add new initialization here
this->addAttribute("clickhouse.thread_id", getThreadId());
}
catch (...)
{

View File

@ -381,6 +381,9 @@ The server successfully detected this situation and will download merged part fr
M(CachedWriteBufferCacheWriteBytes, "Bytes written from source (remote fs, etc) to filesystem cache") \
M(CachedWriteBufferCacheWriteMicroseconds, "Time spent writing data into filesystem cache") \
\
M(FilesystemCacheEvictedBytes, "Number of bytes evicted from filesystem cache") \
M(FilesystemCacheEvictedFileSegments, "Number of file segments evicted from filesystem cache") \
\
M(RemoteFSSeeks, "Total number of seeks for async buffer") \
M(RemoteFSPrefetches, "Number of prefetches made with asynchronous reading from remote filesystem") \
M(RemoteFSCancelledPrefetches, "Number of cancelled prefecthes (because of seek)") \
@ -429,10 +432,10 @@ The server successfully detected this situation and will download merged part fr
M(AggregationPreallocatedElementsInHashTables, "How many elements were preallocated in hash tables for aggregation.") \
M(AggregationHashTablesInitializedAsTwoLevel, "How many hash tables were inited as two-level for aggregation.") \
\
M(MergeTreeMetadataCacheGet, "Number of rocksdb reads(used for merge tree metadata cache)") \
M(MergeTreeMetadataCachePut, "Number of rocksdb puts(used for merge tree metadata cache)") \
M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \
M(MergeTreeMetadataCacheSeek, "Number of rocksdb seeks(used for merge tree metadata cache)") \
M(MergeTreeMetadataCacheGet, "Number of rocksdb reads (used for merge tree metadata cache)") \
M(MergeTreeMetadataCachePut, "Number of rocksdb puts (used for merge tree metadata cache)") \
M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes (used for merge tree metadata cache)") \
M(MergeTreeMetadataCacheSeek, "Number of rocksdb seeks (used for merge tree metadata cache)") \
M(MergeTreeMetadataCacheHit, "Number of times the read of meta file was done from MergeTree metadata cache") \
M(MergeTreeMetadataCacheMiss, "Number of times the read of meta file was not done from MergeTree metadata cache") \
\

View File

@ -0,0 +1,29 @@
#include <Common/checkSSLReturnCode.h>
#include "config.h"
#if USE_SSL
#include <Poco/Net/SecureStreamSocket.h>
#endif
namespace DB
{
bool checkSSLWantRead([[maybe_unused]] ssize_t ret)
{
#if USE_SSL
return ret == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ;
#else
return false;
#endif
}
bool checkSSLWantWrite([[maybe_unused]] ssize_t ret)
{
#if USE_SSL
return ret == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE;
#else
return false;
#endif
}
}

View File

@ -0,0 +1,13 @@
#pragma once
#include <sys/types.h>
namespace DB
{
/// Check if ret is ERR_SSL_WANT_READ.
bool checkSSLWantRead(ssize_t ret);
/// CHeck if ret is ERR_SSL_WANT_WRITE.
bool checkSSLWantWrite(ssize_t ret);
}

View File

@ -25,17 +25,10 @@ namespace DB
*/
template <typename To, typename From>
requires std::is_reference_v<To>
To typeid_cast(From & from)
To typeid_cast(From & from) noexcept(false)
{
try
{
if ((typeid(From) == typeid(To)) || (typeid(from) == typeid(To)))
return static_cast<To>(from);
}
catch (const std::exception & e)
{
throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR);
}
if ((typeid(From) == typeid(To)) || (typeid(from) == typeid(To)))
return static_cast<To>(from);
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Bad cast from type {} to {}",
demangle(typeid(from).name()), demangle(typeid(To).name()));
@ -44,19 +37,12 @@ To typeid_cast(From & from)
template <typename To, typename From>
requires std::is_pointer_v<To>
To typeid_cast(From * from)
To typeid_cast(From * from) noexcept
{
try
{
if ((typeid(From) == typeid(std::remove_pointer_t<To>)) || (from && typeid(*from) == typeid(std::remove_pointer_t<To>)))
return static_cast<To>(from);
else
return nullptr;
}
catch (const std::exception & e)
{
throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR);
}
if ((typeid(From) == typeid(std::remove_pointer_t<To>)) || (from && typeid(*from) == typeid(std::remove_pointer_t<To>)))
return static_cast<To>(from);
else
return nullptr;
}
namespace detail
@ -79,17 +65,10 @@ inline constexpr bool is_shared_ptr_v = is_shared_ptr<T>::value;
template <typename To, typename From>
requires detail::is_shared_ptr_v<To>
To typeid_cast(const std::shared_ptr<From> & from)
To typeid_cast(const std::shared_ptr<From> & from) noexcept
{
try
{
if ((typeid(From) == typeid(typename To::element_type)) || (from && typeid(*from) == typeid(typename To::element_type)))
return std::static_pointer_cast<typename To::element_type>(from);
else
return nullptr;
}
catch (const std::exception & e)
{
throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR);
}
if ((typeid(From) == typeid(typename To::element_type)) || (from && typeid(*from) == typeid(typename To::element_type)))
return std::static_pointer_cast<typename To::element_type>(from);
else
return nullptr;
}

View File

@ -364,6 +364,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
else if (s.get_last_log_idx() < latest_snapshot_meta->get_last_log_idx())
{
LOG_INFO(log, "A snapshot with a larger last log index ({}) was created, skipping applying this snapshot", latest_snapshot_meta->get_last_log_idx());
return true;
}
latest_snapshot_ptr = latest_snapshot_buf;
@ -373,6 +374,10 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
std::lock_guard lock(storage_and_responses_lock);
auto snapshot_deserialization_result
= snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx()));
/// maybe some logs were preprocessed with log idx larger than the snapshot idx
/// we have to apply them to the new storage
storage->applyUncommittedState(*snapshot_deserialization_result.storage, s.get_last_log_idx());
storage = std::move(snapshot_deserialization_result.storage);
latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta;
cluster_config = snapshot_deserialization_result.cluster_config;

View File

@ -375,23 +375,26 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta)
delta.operation);
}
void KeeperStorage::UncommittedState::addDelta(Delta new_delta)
{
const auto & added_delta = deltas.emplace_back(std::move(new_delta));
if (!added_delta.path.empty())
{
deltas_for_path[added_delta.path].push_back(&added_delta);
applyDelta(added_delta);
}
else if (const auto * auth_delta = std::get_if<AddAuthDelta>(&added_delta.operation))
{
auto & uncommitted_auth = session_and_auth[auth_delta->session_id];
uncommitted_auth.emplace_back(&auth_delta->auth_id);
}
}
void KeeperStorage::UncommittedState::addDeltas(std::vector<Delta> new_deltas)
{
for (auto & delta : new_deltas)
{
const auto & added_delta = deltas.emplace_back(std::move(delta));
if (!added_delta.path.empty())
{
deltas_for_path[added_delta.path].push_back(&added_delta);
applyDelta(added_delta);
}
else if (const auto * auth_delta = std::get_if<AddAuthDelta>(&added_delta.operation))
{
auto & uncommitted_auth = session_and_auth[auth_delta->session_id];
uncommitted_auth.emplace_back(&auth_delta->auth_id);
}
}
addDelta(std::move(delta));
}
void KeeperStorage::UncommittedState::commit(int64_t commit_zxid)
@ -602,6 +605,26 @@ namespace
}
void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_zxid)
{
for (const auto & transaction : uncommitted_transactions)
{
if (transaction.zxid <= last_zxid)
continue;
other.uncommitted_transactions.push_back(transaction);
}
auto it = uncommitted_state.deltas.begin();
for (; it != uncommitted_state.deltas.end(); ++it)
{
if (it->zxid <= last_zxid)
continue;
other.uncommitted_state.addDelta(*it);
}
}
Coordination::Error KeeperStorage::commit(int64_t commit_zxid)
{
// Deltas are added with increasing ZXIDs

View File

@ -222,6 +222,7 @@ public:
{
explicit UncommittedState(KeeperStorage & storage_) : storage(storage_) { }
void addDelta(Delta new_delta);
void addDeltas(std::vector<Delta> new_deltas);
void commit(int64_t commit_zxid);
void rollback(int64_t rollback_zxid);
@ -310,6 +311,10 @@ public:
UncommittedState uncommitted_state{*this};
// Apply uncommitted state to another storage using only transactions
// with zxid > last_zxid
void applyUncommittedState(KeeperStorage & other, int64_t last_zxid);
Coordination::Error commit(int64_t zxid);
// Create node in the storage

View File

@ -2524,6 +2524,83 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest)
}
}
TEST_P(CoordinationTest, TestReapplyingDeltas)
{
using namespace DB;
using namespace Coordination;
static constexpr int64_t initial_zxid = 100;
const auto create_request = std::make_shared<ZooKeeperCreateRequest>();
create_request->path = "/test/data";
create_request->is_sequential = true;
const auto process_create = [](KeeperStorage & storage, const auto & request, int64_t zxid)
{
storage.preprocessRequest(request, 1, 0, zxid);
auto responses = storage.processRequest(request, 1, zxid);
EXPECT_GE(responses.size(), 1);
EXPECT_EQ(responses[0].response->error, Error::ZOK);
};
const auto commit_initial_data = [&](auto & storage)
{
int64_t zxid = 1;
const auto root_create = std::make_shared<ZooKeeperCreateRequest>();
root_create->path = "/test";
process_create(storage, root_create, zxid);
++zxid;
for (; zxid <= initial_zxid; ++zxid)
process_create(storage, create_request, zxid);
};
KeeperStorage storage1{500, "", keeper_context};
commit_initial_data(storage1);
for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid)
storage1.preprocessRequest(create_request, 1, 0, zxid);
/// create identical new storage
KeeperStorage storage2{500, "", keeper_context};
commit_initial_data(storage2);
storage1.applyUncommittedState(storage2, initial_zxid);
const auto commit_unprocessed = [&](KeeperStorage & storage)
{
for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid)
{
auto responses = storage.processRequest(create_request, 1, zxid);
EXPECT_GE(responses.size(), 1);
EXPECT_EQ(responses[0].response->error, Error::ZOK);
}
};
commit_unprocessed(storage1);
commit_unprocessed(storage2);
const auto get_children = [&](KeeperStorage & storage)
{
const auto list_request = std::make_shared<ZooKeeperListRequest>();
list_request->path = "/test";
auto responses = storage.processRequest(list_request, 1, std::nullopt, /*check_acl=*/true, /*is_local=*/true);
EXPECT_EQ(responses.size(), 1);
const auto * list_response = dynamic_cast<const ListResponse *>(responses[0].response.get());
EXPECT_TRUE(list_response);
return list_response->names;
};
auto children1 = get_children(storage1);
std::unordered_set<std::string> children1_set(children1.begin(), children1.end());
auto children2 = get_children(storage2);
std::unordered_set<std::string> children2_set(children2.begin(), children2.end());
ASSERT_TRUE(children1_set == children2_set);
}
INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite,
CoordinationTest,
::testing::ValuesIn(std::initializer_list<CompressionParam>{

View File

@ -167,7 +167,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header,
auto temporary_table = TemporaryTableHolder(getContext(), ColumnsDescription{columns}, {});
auto storage = temporary_table.getTable();
getContext()->addExternalTable(data->table_name, std::move(temporary_table));
auto sink = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), getContext());
auto sink = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), getContext(), /*async_insert=*/false);
/// Write data
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*data->pipe));

View File

@ -202,6 +202,8 @@ void preparePostgreSQLArrayInfo(
parser = [](std::string & field) -> Field { return pqxx::from_string<float>(field); };
else if (which.isFloat64())
parser = [](std::string & field) -> Field { return pqxx::from_string<double>(field); };
else if (which.isUUID())
parser = [](std::string & field) -> Field { return parse<UUID>(field); };
else if (which.isString() || which.isFixedString())
parser = [](std::string & field) -> Field { return field; };
else if (which.isDate())

View File

@ -21,7 +21,9 @@ namespace DB
M(UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0) \
M(UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0) \
M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \
M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The maximum number of threads that would be used for loading outdated data parts on startup", 0) \
M(UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0) \
M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0) \
M(UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0) \
M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \
M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \
M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \

View File

@ -160,6 +160,7 @@ class IColumn;
M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \
M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \
M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \
M(UInt64, parallel_replicas_min_number_of_granules_to_enable, 0, "If the number of marks to read is less than the value of this setting - parallel replicas will be disabled", 0) \
\
M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \
\
@ -201,6 +202,8 @@ class IColumn;
M(Bool, force_primary_key, false, "Throw an exception if there is primary key in a table, and it is not used.", 0) \
M(Bool, use_skip_indexes, true, "Use data skipping indexes during query execution.", 0) \
M(Bool, use_skip_indexes_if_final, false, "If query has FINAL, then skipping data based on indexes may produce incorrect result, hence disabled by default.", 0) \
M(String, ignore_data_skipping_indices, "", "Comma separated list of strings or literals with the name of the data skipping indices that should be excluded during query execution.", 0) \
\
M(String, force_data_skipping_indices, "", "Comma separated list of strings or literals with the name of the data skipping indices that should be used during query execution, otherwise an exception will be thrown.", 0) \
\
M(Float, max_streams_to_max_threads_ratio, 1, "Allows you to use more sources than the number of threads - to more evenly distribute work across threads. It is assumed that this is a temporary solution, since it will be possible in the future to make the number of sources equal to the number of threads, but for each source to dynamically select available work for itself.", 0) \
@ -510,7 +513,7 @@ class IColumn;
M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \
M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \
M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \
M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) \
M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \
M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \
M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \
M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \
@ -560,7 +563,7 @@ class IColumn;
M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \
M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \
M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \
M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) \
M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) ALIAS(allow_experimental_projection_optimization) \
M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \
M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \
M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \
@ -651,6 +654,8 @@ class IColumn;
M(UInt64, merge_tree_min_rows_for_concurrent_read_for_remote_filesystem, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \
M(UInt64, merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \
M(UInt64, remote_read_min_bytes_for_seek, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes required for remote read (url, s3) to do seek, instead of read with ignore.", 0) \
M(UInt64, merge_tree_min_bytes_per_task_for_remote_reading, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes to read per task.", 0) \
M(Bool, merge_tree_use_const_size_tasks_for_remote_reading, true, "Whether to use constant size tasks for reading from a remote table.", 0) \
\
M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \
M(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background. If wait_for_async_insert is false, INSERT query is processed almost instantly, otherwise client will wait until data will be flushed to table", 0) \
@ -717,7 +722,6 @@ class IColumn;
\
M(Bool, parallelize_output_from_storages, true, "Parallelize output for reading step from storage. It allows parallelizing query processing right after reading from storage if possible", 0) \
M(String, insert_deduplication_token, "", "If not empty, used for duplicate detection instead of data digest", 0) \
M(String, ann_index_select_query_params, "", "Parameters passed to ANN indexes in SELECT queries, the format is 'param1=x, param2=y, ...'", 0) \
M(Bool, count_distinct_optimization, false, "Rewrite count distinct to subquery of group by", 0) \
M(Bool, throw_if_no_data_to_insert, true, "Enables or disables empty INSERTs, enabled by default", 0) \
M(Bool, compatibility_ignore_auto_increment_in_create_table, false, "Ignore AUTO_INCREMENT keyword in column declaration if true, otherwise return error. It simplifies migration from MySQL", 0) \
@ -740,7 +744,8 @@ class IColumn;
M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions (hashid, etc)", 0) \
M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \
M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \
M(UInt64, max_limit_for_ann_queries, 1000000, "Maximum limit value for using ANN indexes is used to prevent memory overflow in search queries for indexes", 0) \
M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \
M(Int64, annoy_index_search_k_nodes, -1, "SELECT queries search up to this many nodes in Annoy indexes.", 0) \
M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \
M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \
M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \
@ -768,7 +773,6 @@ class IColumn;
MAKE_OBSOLETE(M, Bool, allow_experimental_database_atomic, true) \
MAKE_OBSOLETE(M, Bool, allow_experimental_bigint_types, true) \
MAKE_OBSOLETE(M, Bool, allow_experimental_window_functions, true) \
MAKE_OBSOLETE(M, Bool, allow_experimental_lightweight_delete, true) \
MAKE_OBSOLETE(M, Bool, allow_experimental_geo_types, true) \
\
MAKE_OBSOLETE(M, Milliseconds, async_insert_stale_timeout_ms, 0) \
@ -781,7 +785,6 @@ class IColumn;
MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \
MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \
MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \
MAKE_OBSOLETE(M, Bool, allow_experimental_projection_optimization, true) \
MAKE_OBSOLETE(M, Bool, allow_experimental_query_cache, true) \
/* moved to config.xml: see also src/Core/ServerSettings.h */ \
MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_buffer_flush_schedule_pool_size, 16) \
@ -847,6 +850,7 @@ class IColumn;
M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \
M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \
M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \
M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \
M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \
M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \
M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \

View File

@ -51,19 +51,11 @@ void SerializationUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & ist
{
assertChar('\'', istr);
char * next_pos = find_first_symbols<'\\', '\''>(istr.position(), istr.buffer().end());
size_t len = next_pos - istr.position();
if ((len == 32) && (istr.position()[32] == '\''))
const size_t len = next_pos - istr.position();
if ((len == 32 || len == 36) && istr.position()[len] == '\'')
{
parseUUIDWithoutSeparator(
reinterpret_cast<const UInt8 *>(istr.position()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
istr.ignore(33);
fast = true;
}
else if ((len == 36) && (istr.position()[36] == '\''))
{
parseUUID(
reinterpret_cast<const UInt8 *>(istr.position()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
istr.ignore(37);
uuid = parseUUID(std::span(reinterpret_cast<const UInt8 *>(istr.position()), len));
istr.ignore(len + 1);
fast = true;
}
else

View File

@ -207,6 +207,51 @@ public:
return result;
}
/**
* Analogous to getColumn, but for dictGetAll
*/
virtual ColumnPtr getColumnAllValues(
const std::string & attribute_name [[maybe_unused]],
const DataTypePtr & result_type [[maybe_unused]],
const Columns & key_columns [[maybe_unused]],
const DataTypes & key_types [[maybe_unused]],
const ColumnPtr & default_values_column [[maybe_unused]],
size_t limit [[maybe_unused]]) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Method getColumnAllValues is not supported for {} dictionary.",
getDictionaryID().getNameForLogs());
}
/**
* Analogous to getColumns, but for dictGetAll
*/
virtual Columns getColumnsAllValues(
const Strings & attribute_names,
const DataTypes & result_types,
const Columns & key_columns,
const DataTypes & key_types,
const Columns & default_values_columns,
size_t limit) const
{
size_t attribute_names_size = attribute_names.size();
Columns result;
result.reserve(attribute_names_size);
for (size_t i = 0; i < attribute_names_size; ++i)
{
const auto & attribute_name = attribute_names[i];
const auto & result_type = result_types[i];
const auto & default_values_column = default_values_columns[i];
result.emplace_back(getColumnAllValues(
attribute_name, result_type, key_columns, key_types, default_values_column, limit));
}
return result;
}
/** Subclass must validate key columns and key types and return ColumnUInt8 that
* is bitmask representation of is key in dictionary or not.
* If key is in dictionary then value of associated row will be 1, otherwise 0.

View File

@ -70,7 +70,7 @@ namespace
explicit StringPiece(int ref_) : ref_num(ref_) {}
};
Field parseStringToField(const String & raw, DataTypePtr data_type)
Field parseStringToField(const String & raw, const DataTypePtr data_type)
try
{
ReadBufferFromString buffer(raw);
@ -129,17 +129,6 @@ struct RegExpTreeDictionary::RegexTreeNode
return searcher.Match(haystack, 0, size, re2_st::RE2::Anchor::UNANCHORED, nullptr, 0);
}
/// check if this node can cover all the attributes from the query.
bool containsAll(const std::unordered_map<String, const DictionaryAttribute &> & matching_attributes) const
{
for (const auto & [key, value] : matching_attributes)
{
if (!attributes.contains(key))
return false;
}
return true;
}
struct AttributeValue
{
Field field;
@ -419,6 +408,65 @@ RegExpTreeDictionary::RegExpTreeDictionary(
calculateBytesAllocated();
}
// Thin wrapper around unordered_map<String, Field> that manages the collection of attribute values subject to the
// behavior specified by collect_values_limit
class RegExpTreeDictionary::AttributeCollector : public std::unordered_map<String, Field>
{
private:
std::optional<size_t> collect_values_limit; // std::nullopt means single-value mode, i.e. don't collect
size_t n_full_attributes;
public:
explicit AttributeCollector(std::optional<size_t> collect_values_limit_)
: collect_values_limit(collect_values_limit_), n_full_attributes(0)
{
}
constexpr bool collecting() const { return collect_values_limit != std::nullopt; }
// Add a name-value pair to the collection if there's space
void add(const String & attr_name, Field field)
{
if (collect_values_limit)
{
if (!this->contains(attr_name))
(*this)[attr_name] = Array();
Array & values = (*this)[attr_name].safeGet<Array &>();
if (values.size() < *collect_values_limit)
{
values.push_back(std::move(field));
if (values.size() == *collect_values_limit)
n_full_attributes++;
}
}
else if (!this->contains(attr_name))
{
(*this)[attr_name] = std::move(field);
n_full_attributes++;
}
}
// Checks if no more values can be added for a given attribute
inline bool full(const String & attr_name) const
{
if (collect_values_limit)
{
auto it = this->find(attr_name);
if (it == this->end())
return false;
return it->second.safeGet<const Array &>().size() >= *collect_values_limit;
}
else
{
return this->contains(attr_name);
}
}
// Returns the number of full attributes
inline size_t attributesFull() const { return n_full_attributes; }
};
std::pair<String, bool> processBackRefs(const String & data, const re2_st::RE2 & searcher, const std::vector<StringPiece> & pieces)
{
re2_st::StringPiece haystack(data.data(), data.size());
@ -442,7 +490,7 @@ std::pair<String, bool> processBackRefs(const String & data, const re2_st::RE2 &
// The return value means whether we finish collecting.
bool RegExpTreeDictionary::setAttributes(
UInt64 id,
std::unordered_map<String, Field> & attributes_to_set,
AttributeCollector & attributes_to_set,
const String & data,
std::unordered_set<UInt64> & visited_nodes,
const std::unordered_map<String, const DictionaryAttribute &> & attributes,
@ -451,34 +499,43 @@ bool RegExpTreeDictionary::setAttributes(
{
if (visited_nodes.contains(id))
return attributes_to_set.size() == attributes.size();
return attributes_to_set.attributesFull() == attributes.size();
visited_nodes.emplace(id);
const auto & node_attributes = regex_nodes.at(id)->attributes;
for (const auto & [name_, value] : node_attributes)
{
if (!attributes.contains(name_) || attributes_to_set.contains(name_))
if (!attributes.contains(name_) || attributes_to_set.full(name_))
continue;
if (value.containsBackRefs())
{
auto [updated_str, use_default] = processBackRefs(data, regex_nodes.at(id)->searcher, value.pieces);
if (use_default)
{
DefaultValueProvider default_value(attributes.at(name_).null_value, defaults.at(name_));
attributes_to_set[name_] = default_value.getDefaultValue(key_index);
// Back-ref processing failed.
// - If not collecting values, set the default value immediately while we're still on this node.
// Otherwise, a value from a different node could take its place before we set it to the default value post-walk.
// - If collecting values, don't add anything. If we find no other matches for this attribute,
// then we'll set its value to the default Array value later.
if (!attributes_to_set.collecting())
{
DefaultValueProvider default_value(attributes.at(name_).null_value, defaults.at(name_));
attributes_to_set.add(name_, default_value.getDefaultValue(key_index));
}
}
else
attributes_to_set[name_] = parseStringToField(updated_str, attributes.at(name_).type);
attributes_to_set.add(name_, parseStringToField(updated_str, attributes.at(name_).type));
}
else
attributes_to_set[name_] = value.field;
attributes_to_set.add(name_, value.field);
}
auto parent_id = regex_nodes.at(id)->parent_id;
if (parent_id > 0)
setAttributes(parent_id, attributes_to_set, data, visited_nodes, attributes, defaults, key_index);
/// if all the attributes have set, the walking through can be stopped.
return attributes_to_set.size() == attributes.size();
/// if all attributes are full, we can stop walking the tree
return attributes_to_set.attributesFull() == attributes.size();
}
/// a temp struct to store all the matched result.
@ -550,7 +607,8 @@ std::unordered_map<String, ColumnPtr> RegExpTreeDictionary::match(
const ColumnString::Chars & keys_data,
const ColumnString::Offsets & keys_offsets,
const std::unordered_map<String, const DictionaryAttribute &> & attributes,
const std::unordered_map<String, ColumnPtr> & defaults) const
const std::unordered_map<String, ColumnPtr> & defaults,
std::optional<size_t> collect_values_limit) const
{
#if USE_VECTORSCAN
@ -573,7 +631,7 @@ std::unordered_map<String, ColumnPtr> RegExpTreeDictionary::match(
/// initialize columns
for (const auto & [name_, attr] : attributes)
{
auto col_ptr = attr.type->createColumn();
auto col_ptr = (collect_values_limit ? std::make_shared<DataTypeArray>(attr.type) : attr.type)->createColumn();
col_ptr->reserve(keys_offsets.size());
columns[name_] = std::move(col_ptr);
}
@ -622,19 +680,16 @@ std::unordered_map<String, ColumnPtr> RegExpTreeDictionary::match(
if (node_ptr->match(reinterpret_cast<const char *>(keys_data.data()) + offset, length))
{
match_result.insertNodeID(node_ptr->id);
/// When this node is leaf and contains all the required attributes, it means a match.
if (node_ptr->containsAll(attributes) && node_ptr->children.empty())
break;
}
}
match_result.sort();
/// Walk through the regex tree util all attributes are set;
std::unordered_map<String, Field> attributes_to_set;
AttributeCollector attributes_to_set{collect_values_limit};
std::unordered_set<UInt64> visited_nodes;
/// Some node matches but its parents cannot match. In this case we must regard this node unmatched.
auto is_invalid = [&](UInt64 id)
auto is_valid = [&](UInt64 id)
{
while (id)
{
@ -650,7 +705,7 @@ std::unordered_map<String, ColumnPtr> RegExpTreeDictionary::match(
for (auto item : match_result.matched_idx_sorted_list)
{
UInt64 id = item.second;
if (!is_invalid(id))
if (!is_valid(id))
continue;
if (visited_nodes.contains(id))
continue;
@ -663,7 +718,8 @@ std::unordered_map<String, ColumnPtr> RegExpTreeDictionary::match(
if (attributes_to_set.contains(name_))
continue;
DefaultValueProvider default_value(attr.null_value, defaults.at(name_));
DefaultValueProvider default_value(
collect_values_limit ? DataTypeArray(attr.type).getDefault() : attr.null_value, defaults.at(name_));
columns[name_]->insert(default_value.getDefaultValue(key_idx));
}
@ -727,12 +783,13 @@ Pipe RegExpTreeDictionary::read(const Names & , size_t max_block_size, size_t) c
return Pipe(std::make_shared<BlocksListSource>(std::move(result)));
}
Columns RegExpTreeDictionary::getColumns(
Columns RegExpTreeDictionary::getColumnsImpl(
const Strings & attribute_names,
const DataTypes & result_types,
const Columns & key_columns,
const DataTypes & key_types,
const Columns & default_values_columns) const
const Columns & default_values_columns,
std::optional<size_t> collect_values_limit) const
{
/// valid check
if (key_columns.size() != 1)
@ -746,7 +803,17 @@ Columns RegExpTreeDictionary::getColumns(
for (size_t i = 0; i < attribute_names.size(); i++)
{
const auto & attribute = structure.getAttribute(attribute_names[i], result_types[i]);
DataTypePtr attribute_type = result_types[i];
if (collect_values_limit)
{
if (!WhichDataType(attribute_type).isArray())
throw Exception(
ErrorCodes::LOGICAL_ERROR, "Expected Array result type for attribute `{}`, got `{}`",
attribute_names[i],
attribute_type->getName());
attribute_type = assert_cast<const DataTypeArray &>(*attribute_type).getNestedType();
}
const auto & attribute = structure.getAttribute(attribute_names[i], attribute_type);
attributes.emplace(attribute.name, attribute);
defaults[attribute.name] = default_values_columns[i];
}
@ -757,7 +824,8 @@ Columns RegExpTreeDictionary::getColumns(
key_column->getChars(),
key_column->getOffsets(),
attributes,
defaults);
defaults,
collect_values_limit);
Columns result;
for (const String & name_ : attribute_names)

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