Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into disk-over-web-server

This commit is contained in:
kssenii 2021-08-24 07:29:50 +00:00
commit b536653851
219 changed files with 5823 additions and 651 deletions

View File

@ -3,7 +3,7 @@ I hereby agree to the terms of the CLA available at: https://yandex.ru/legal/cla
Changelog category (leave one):
- New Feature
- Improvement
- Bug Fix
- Bug Fix (user-visible misbehaviour in official stable or prestable release)
- Performance Improvement
- Backward Incompatible Change
- Build/Testing/Packaging Improvement

View File

@ -1,9 +1,11 @@
# Security Policy
## Supported Versions
## Security Announcements
Security fixes will be announced by posting them in the [security changelog](https://clickhouse.tech/docs/en/whats-new/security-changelog/)
The following versions of ClickHouse server are
currently being supported with security updates:
## Scope and Supported Versions
The following versions of ClickHouse server are currently being supported with security updates:
| Version | Supported |
| ------- | ------------------ |
@ -11,18 +13,49 @@ currently being supported with security updates:
| 18.x | :x: |
| 19.x | :x: |
| 20.1 | :x: |
| 20.3 | :white_check_mark: |
| 20.3 | :x: |
| 20.4 | :x: |
| 20.5 | :x: |
| 20.6 | :x: |
| 20.7 | :x: |
| 20.8 | :white_check_mark: |
| 20.8 | :x: |
| 20.9 | :x: |
| 20.10 | :x: |
| 20.11 | :white_check_mark: |
| 20.12 | :white_check_mark: |
| 21.1 | :white_check_mark: |
| 20.11 | :x: |
| 20.12 | :x: |
| 21.1 | :x: |
| 21.2 | :x: |
| 21.3 | ✅ |
| 21.4 | :x: |
| 21.5 | :x: |
| 21.6 | ✅ |
| 21.7 | ✅ |
| 21.8 | ✅ |
## Reporting a Vulnerability
We're extremely grateful for security researchers and users that report vulnerabilities to the ClickHouse Open Source Community. All reports are thoroughly investigated by developers.
To report a potential vulnerability in ClickHouse please send the details about it to [clickhouse-feedback@yandex-team.com](mailto:clickhouse-feedback@yandex-team.com).
### When Should I Report a Vulnerability?
- You think you discovered a potential security vulnerability in ClickHouse
- You are unsure how a vulnerability affects ClickHouse
### When Should I NOT Report a Vulnerability?
- You need help tuning ClickHouse components for security
- You need help applying security related updates
- Your issue is not security related
## Security Vulnerability Response
Each report is acknowledged and analyzed by ClickHouse maintainers within 5 working days.
As the security issue moves from triage, to identified fix, to release planning we will keep the reporter updated.
## Public Disclosure Timing
A public disclosure date is negotiated by the ClickHouse maintainers and the bug submitter. We prefer to fully disclose the bug as soon as possible once a user mitigation is available. It is reasonable to delay disclosure when the bug or the fix is not yet fully understood, the solution is not well-tested, or for vendor coordination. The timeframe for disclosure is from immediate (especially if it's already publicly known) to 90 days. For a vulnerability with a straightforward mitigation, we expect report date to disclosure date to be on the order of 7 days.

28
base/common/insertAtEnd.h Normal file
View File

@ -0,0 +1,28 @@
#pragma once
#include <vector>
/// Appends a specified vector with elements of another vector.
template <typename T>
void insertAtEnd(std::vector<T> & dest, const std::vector<T> & src)
{
if (src.empty())
return;
dest.reserve(dest.size() + src.size());
dest.insert(dest.end(), src.begin(), src.end());
}
template <typename T>
void insertAtEnd(std::vector<T> & dest, std::vector<T> && src)
{
if (src.empty())
return;
if (dest.empty())
{
dest.swap(src);
return;
}
dest.reserve(dest.size() + src.size());
dest.insert(dest.end(), std::make_move_iterator(src.begin()), std::make_move_iterator(src.end()));
src.clear();
}

10
base/common/unit.h Normal file
View File

@ -0,0 +1,10 @@
#pragma once
#include <cstddef>
constexpr size_t KiB = 1024;
constexpr size_t MiB = 1024 * KiB;
constexpr size_t GiB = 1024 * MiB;
constexpr size_t operator"" _KiB(unsigned long long val) { return val * KiB; }
constexpr size_t operator"" _MiB(unsigned long long val) { return val * MiB; }
constexpr size_t operator"" _GiB(unsigned long long val) { return val * GiB; }

View File

@ -9,6 +9,7 @@
#include <cmath>
#include <cfloat>
#include <cassert>
#include <tuple>
#include <limits>
@ -39,6 +40,18 @@ static constexpr bool IntegralConcept() noexcept
return std::is_integral_v<T> || IsWideInteger<T>::value;
}
template <typename T>
class IsTupleLike
{
template <typename U>
static auto check(U * p) -> decltype(std::tuple_size<U>::value, int());
template <typename>
static void check(...);
public:
static constexpr const bool value = !std::is_void<decltype(check<T>(nullptr))>::value;
};
}
namespace std
@ -227,6 +240,19 @@ struct integer<Bits, Signed>::_impl
self.items[i] = 0;
}
template <typename TupleLike, size_t i = 0>
constexpr static void wide_integer_from_tuple_like(integer<Bits, Signed> & self, const TupleLike & tuple) noexcept
{
if constexpr (i < item_count)
{
if constexpr (i < std::tuple_size_v<TupleLike>)
self.items[i] = std::get<i>(tuple);
else
self.items[i] = 0;
wide_integer_from_tuple_like<TupleLike, i + 1>(self, tuple);
}
}
/**
* N.B. t is constructed from double, so max(t) = max(double) ~ 2^310
* the recursive call happens when t / 2^64 > 2^64, so there won't be more than 5 of them.
@ -966,6 +992,8 @@ constexpr integer<Bits, Signed>::integer(T rhs) noexcept
{
if constexpr (IsWideInteger<T>::value)
_impl::wide_integer_from_wide_integer(*this, rhs);
else if constexpr (IsTupleLike<T>::value)
_impl::wide_integer_from_tuple_like(*this, rhs);
else
_impl::wide_integer_from_builtin(*this, rhs);
}
@ -979,6 +1007,8 @@ constexpr integer<Bits, Signed>::integer(std::initializer_list<T> il) noexcept
{
if constexpr (IsWideInteger<T>::value)
_impl::wide_integer_from_wide_integer(*this, *il.begin());
else if constexpr (IsTupleLike<T>::value)
_impl::wide_integer_from_tuple_like(*this, *il.begin());
else
_impl::wide_integer_from_builtin(*this, *il.begin());
}
@ -1007,7 +1037,10 @@ template <size_t Bits, typename Signed>
template <typename T>
constexpr integer<Bits, Signed> & integer<Bits, Signed>::operator=(T rhs) noexcept
{
_impl::wide_integer_from_builtin(*this, rhs);
if constexpr (IsTupleLike<T>::value)
_impl::wide_integer_from_tuple_like(*this, rhs);
else
_impl::wide_integer_from_builtin(*this, rhs);
return *this;
}

View File

@ -2,6 +2,8 @@ FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update \
&& apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \
--yes --no-install-recommends --verbose-versions \

View File

@ -3,6 +3,8 @@ FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.10.1.*
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update \
&& apt-get install --yes --no-install-recommends \
apt-transport-https \

View File

@ -3,6 +3,8 @@ FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update \
&& apt-get install \
apt-transport-https \

View File

@ -3,6 +3,8 @@ FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update \
&& apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \
--yes --no-install-recommends --verbose-versions \

View File

@ -5,6 +5,8 @@ RUN export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \
&& wget -nv -O /tmp/arrow-keyring.deb "https://apache.jfrog.io/artifactory/arrow/ubuntu/apache-arrow-apt-source-latest-${CODENAME}.deb" \
&& dpkg -i /tmp/arrow-keyring.deb
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
# Libraries from OS are only needed to test the "unbundled" build (that is not used in production).
RUN apt-get update \
&& apt-get install \

View File

@ -26,6 +26,8 @@ ARG DEBIAN_FRONTEND=noninteractive
# installed to prevent picking those uid / gid by some unrelated software.
# The same uid / gid (101) is used both for alpine and ubuntu.
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN groupadd -r clickhouse --gid=101 \
&& useradd -r -g clickhouse --uid=101 --home-dir=/var/lib/clickhouse --shell=/bin/bash clickhouse \
&& apt-get update \

View File

@ -3,6 +3,8 @@ FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update \
&& apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \
--yes --no-install-recommends --verbose-versions \

View File

@ -2,6 +2,8 @@
# docker run --volume=path_to_repo:/repo_folder --volume=path_to_result:/test_output yandex/clickhouse-codebrowser
FROM yandex/clickhouse-binary-builder
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update && apt-get --yes --allow-unauthenticated install clang-9 libllvm9 libclang-9-dev
# repo versions doesn't work correctly with C++17

View File

@ -3,6 +3,8 @@ FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update \
&& apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \
--yes --no-install-recommends --verbose-versions \

View File

@ -303,6 +303,7 @@ function run_tests
01683_codec_encrypted # Depends on OpenSSL
01776_decrypt_aead_size_check # Depends on OpenSSL
01811_filter_by_null # Depends on OpenSSL
02012_sha512_fixedstring # Depends on OpenSSL
01281_unsucceeded_insert_select_queries_counter
01292_create_user
01294_lazy_database_concurrent

View File

@ -5,6 +5,8 @@ ENV LANG=C.UTF-8
ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update \
&& DEBIAN_FRONTEND=noninteractive apt-get install --yes --no-install-recommends \
ca-certificates \

View File

@ -1,6 +1,8 @@
# docker build -t yandex/clickhouse-integration-tests-runner .
FROM ubuntu:20.04
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update \
&& env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
ca-certificates \

View File

@ -5,6 +5,8 @@ ENV LANG=C.UTF-8
ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update \
&& DEBIAN_FRONTEND=noninteractive apt-get install --yes --no-install-recommends \
bash \

View File

@ -1,6 +1,8 @@
# docker build -t yandex/clickhouse-sqlancer-test .
FROM ubuntu:20.04
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update --yes && env DEBIAN_FRONTEND=noninteractive apt-get install wget unzip git openjdk-14-jdk maven python3 --yes --no-install-recommends
RUN wget https://github.com/sqlancer/sqlancer/archive/master.zip -O /sqlancer.zip
RUN mkdir /sqlancer && \

View File

@ -1,6 +1,8 @@
# docker build -t yandex/clickhouse-style-test .
FROM ubuntu:20.04
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
shellcheck \
libxml2-utils \

View File

@ -1,6 +1,8 @@
# docker build -t yandex/clickhouse-testflows-runner .
FROM ubuntu:20.04
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
RUN apt-get update \
&& env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
ca-certificates \

View File

@ -23,3 +23,5 @@ You can also use the following database engines:
- [PostgreSQL](../../engines/database-engines/postgresql.md)
- [Replicated](../../engines/database-engines/replicated.md)
- [SQLite](../../engines/database-engines/sqlite.md)

View File

@ -1,6 +1,6 @@
---
toc_priority: 29
toc_title: "[experimental] MaterializedMySQL"
toc_title: MaterializedMySQL
---
# [experimental] MaterializedMySQL {#materialized-mysql}

View File

@ -0,0 +1,80 @@
---
toc_priority: 32
toc_title: SQLite
---
# SQLite {#sqlite}
Allows to connect to [SQLite](https://www.sqlite.org/index.html) database and perform `INSERT` and `SELECT` queries to exchange data between ClickHouse and SQLite.
## Creating a Database {#creating-a-database}
``` sql
CREATE DATABASE sqlite_database
ENGINE = SQLite('db_path')
```
**Engine Parameters**
- `db_path` — Path to a file with SQLite database.
## Data Types Support {#data_types-support}
| SQLite | ClickHouse |
|---------------|---------------------------------------------------------|
| INTEGER | [Int32](../../sql-reference/data-types/int-uint.md) |
| REAL | [Float32](../../sql-reference/data-types/float.md) |
| TEXT | [String](../../sql-reference/data-types/string.md) |
| BLOB | [String](../../sql-reference/data-types/string.md) |
## Specifics and Recommendations {#specifics-and-recommendations}
SQLite stores the entire database (definitions, tables, indices, and the data itself) as a single cross-platform file on a host machine. During writing SQLite locks the entire database file, therefore write operations are performed sequentially. Read operations can be multitasked.
SQLite does not require service management (such as startup scripts) or access control based on `GRANT` and passwords. Access control is handled by means of file-system permissions given to the database file itself.
## Usage Example {#usage-example}
Database in ClickHouse, connected to the SQLite:
``` sql
CREATE DATABASE sqlite_db ENGINE = SQLite('sqlite.db');
SHOW TABLES FROM sqlite_db;
```
``` text
┌──name───┐
│ table1 │
│ table2 │
└─────────┘
```
Shows the tables:
``` sql
SELECT * FROM sqlite_db.table1;
```
``` text
┌─col1──┬─col2─┐
│ line1 │ 1 │
│ line2 │ 2 │
│ line3 │ 3 │
└───────┴──────┘
```
Inserting data into SQLite table from ClickHouse table:
``` sql
CREATE TABLE clickhouse_table(`col1` String,`col2` Int16) ENGINE = MergeTree() ORDER BY col2;
INSERT INTO clickhouse_table VALUES ('text',10);
INSERT INTO sqlite_db.table1 SELECT * FROM clickhouse_table;
SELECT * FROM sqlite_db.table1;
```
``` text
┌─col1──┬─col2─┐
│ line1 │ 1 │
│ line2 │ 2 │
│ line3 │ 3 │
│ text │ 10 │
└───────┴──────┘
```

View File

@ -19,3 +19,4 @@ List of supported integrations:
- [EmbeddedRocksDB](../../../engines/table-engines/integrations/embedded-rocksdb.md)
- [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md)
- [PostgreSQL](../../../engines/table-engines/integrations/postgresql.md)
- [SQLite](../../../engines/table-engines/integrations/sqlite.md)

View File

@ -0,0 +1,59 @@
---
toc_priority: 7
toc_title: SQLite
---
# SQLite {#sqlite}
The engine allows to import and export data to SQLite and supports queries to SQLite tables directly from ClickHouse.
## Creating a Table {#creating-a-table}
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name
(
name1 [type1],
name2 [type2], ...
) ENGINE = SQLite('db_path', 'table')
```
**Engine Parameters**
- `db_path` — Path to SQLite file with a database.
- `table` — Name of a table in the SQLite database.
## Usage Example {#usage-example}
Shows a query creating the SQLite table:
```sql
SHOW CREATE TABLE sqlite_db.table2;
```
``` text
CREATE TABLE SQLite.table2
(
`col1` Nullable(Int32),
`col2` Nullable(String)
)
ENGINE = SQLite('sqlite.db','table2');
```
Returns the data from the table:
``` sql
SELECT * FROM sqlite_db.table2 ORDER BY col1;
```
```text
┌─col1─┬─col2──┐
│ 1 │ text1 │
│ 2 │ text2 │
│ 3 │ text3 │
└──────┴───────┘
```
**See Also**
- [SQLite](../../../engines/database-engines/sqlite.md) engine
- [sqlite](../../../sql-reference/table-functions/sqlite.md) table function

View File

@ -99,7 +99,9 @@ For a description of parameters, see the [CREATE query description](../../../sql
- `use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information, see the [setting description](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) in “Server configuration parameters”.
- `min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes.
<a name="mergetree_setting-merge_with_ttl_timeout"></a>
- `merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with TTL. Default value: 86400 (1 day).
- `merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with delete TTL. Default value: `14400` seconds (4 hours).
- `merge_with_recompression_ttl_timeout` — Minimum delay in seconds before repeating a merge with recompression TTL. Default value: `14400` seconds (4 hours).
- `try_fetch_recompressed_part_timeout` — Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. Default value: `7200` seconds (2 hours).
- `write_final_mark` — Enables or disables writing the final index mark at the end of data part (after the last byte). Default value: 1. Dont turn it off.
- `merge_max_block_size` — Maximum number of rows in block for merge operations. Default value: 8192.
- `storage_policy` — Storage policy. See [Using Multiple Block Devices for Data Storage](#table_engine-mergetree-multiple-volumes).
@ -333,7 +335,7 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
The optional `false_positive` parameter is the probability of receiving a false positive response from the filter. Possible values: (0, 1). Default value: 0.025.
Supported data types: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`.
Supported data types: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`, `UUID`.
The following functions can use it: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md).
@ -416,18 +418,20 @@ Reading from a table is automatically parallelized.
Determines the lifetime of values.
The `TTL` clause can be set for the whole table and for each individual column. Table-level TTL can also specify logic of automatic move of data between disks and volumes.
The `TTL` clause can be set for the whole table and for each individual column. Table-level `TTL` can also specify the logic of automatic moving data between disks and volumes, or recompressing parts where all the data has been expired.
Expressions must evaluate to [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md) data type.
Example:
**Syntax**
Setting time-to-live for a column:
``` sql
TTL time_column
TTL time_column + interval
```
To define `interval`, use [time interval](../../../sql-reference/operators/index.md#operators-datetime) operators.
To define `interval`, use [time interval](../../../sql-reference/operators/index.md#operators-datetime) operators, for example:
``` sql
TTL date_time + INTERVAL 1 MONTH
@ -440,9 +444,9 @@ When the values in the column expire, ClickHouse replaces them with the default
The `TTL` clause cant be used for key columns.
Examples:
**Examples**
Creating a table with TTL
Creating a table with `TTL`:
``` sql
CREATE TABLE example_table
@ -475,11 +479,11 @@ ALTER TABLE example_table
### Table TTL {#mergetree-table-ttl}
Table can have an expression for removal of expired rows, and multiple expressions for automatic move of parts between [disks or volumes](#table_engine-mergetree-multiple-volumes). When rows in the table expire, ClickHouse deletes all corresponding rows. For parts moving feature, all rows of a part must satisfy the movement expression criteria.
Table can have an expression for removal of expired rows, and multiple expressions for automatic move of parts between [disks or volumes](#table_engine-mergetree-multiple-volumes). When rows in the table expire, ClickHouse deletes all corresponding rows. For parts moving or recompressing, all rows of a part must satisfy the `TTL` expression criteria.
``` sql
TTL expr
[DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'] ...
[DELETE|RECOMPRESS codec_name1|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|RECOMPRESS codec_name2|TO DISK 'aaa'|TO VOLUME 'bbb'] ...
[WHERE conditions]
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ]
```
@ -487,11 +491,12 @@ TTL expr
Type of TTL rule may follow each TTL expression. It affects an action which is to be done once the expression is satisfied (reaches current time):
- `DELETE` - delete expired rows (default action);
- `RECOMPRESS codec_name` - recompress data part with the `codec_name`;
- `TO DISK 'aaa'` - move part to the disk `aaa`;
- `TO VOLUME 'bbb'` - move part to the disk `bbb`;
- `GROUP BY` - aggregate expired rows.
With `WHERE` clause you may specify which of the expired rows to delete or aggregate (it cannot be applied to moves).
With `WHERE` clause you may specify which of the expired rows to delete or aggregate (it cannot be applied to moves or recompression).
`GROUP BY` expression must be a prefix of the table primary key.
@ -499,7 +504,7 @@ If a column is not part of the `GROUP BY` expression and is not set explicitly i
**Examples**
Creating a table with TTL:
Creating a table with `TTL`:
``` sql
CREATE TABLE example_table
@ -515,7 +520,7 @@ TTL d + INTERVAL 1 MONTH [DELETE],
d + INTERVAL 2 WEEK TO DISK 'bbb';
```
Altering TTL of the table:
Altering `TTL` of the table:
``` sql
ALTER TABLE example_table
@ -536,6 +541,21 @@ ORDER BY d
TTL d + INTERVAL 1 MONTH DELETE WHERE toDayOfWeek(d) = 1;
```
Creating a table, where expired rows are recompressed:
```sql
CREATE TABLE table_for_recompression
(
d DateTime,
key UInt64,
value String
) ENGINE MergeTree()
ORDER BY tuple()
PARTITION BY key
TTL d + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), d + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10))
SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0;
```
Creating a table, where expired rows are aggregated. In result rows `x` contains the maximum value accross the grouped rows, `y` — the minimum value, and `d` — any occasional value from grouped rows.
``` sql
@ -552,14 +572,19 @@ ORDER BY (k1, k2)
TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y);
```
**Removing Data**
### Removing Expired Data {#mergetree-removing-expired-data}
Data with an expired TTL is removed when ClickHouse merges data parts.
Data with an expired `TTL` is removed when ClickHouse merges data parts.
When ClickHouse see that data is expired, it performs an off-schedule merge. To control the frequency of such merges, you can set `merge_with_ttl_timeout`. If the value is too low, it will perform many off-schedule merges that may consume a lot of resources.
When ClickHouse detects that data is expired, it performs an off-schedule merge. To control the frequency of such merges, you can set `merge_with_ttl_timeout`. If the value is too low, it will perform many off-schedule merges that may consume a lot of resources.
If you perform the `SELECT` query between merges, you may get expired data. To avoid it, use the [OPTIMIZE](../../../sql-reference/statements/optimize.md) query before `SELECT`.
**See Also**
- [ttl_only_drop_parts](../../../operations/settings/settings.md#ttl_only_drop_parts) setting
## Using Multiple Block Devices for Data Storage {#table_engine-mergetree-multiple-volumes}
### Introduction {#introduction}

View File

@ -2041,10 +2041,25 @@ Default value: 0.
## input_format_parallel_parsing {#input-format-parallel-parsing}
- Type: bool
- Default value: True
Enables or disables order-preserving parallel parsing of data formats. Supported only for [TSV](../../interfaces/formats.md#tabseparated), [TKSV](../../interfaces/formats.md#tskv), [CSV](../../interfaces/formats.md#csv) and [JSONEachRow](../../interfaces/formats.md#jsoneachrow) formats.
Enable order-preserving parallel parsing of data formats. Supported only for TSV, TKSV, CSV, and JSONEachRow formats.
Possible values:
- 1 — Enabled.
- 0 — Disabled.
Default value: `0`.
## output_format_parallel_formatting {#output-format-parallel-formatting}
Enables or disables parallel formatting of data formats. Supported only for [TSV](../../interfaces/formats.md#tabseparated), [TKSV](../../interfaces/formats.md#tskv), [CSV](../../interfaces/formats.md#csv) and [JSONEachRow](../../interfaces/formats.md#jsoneachrow) formats.
Possible values:
- 1 — Enabled.
- 0 — Disabled.
Default value: `0`.
## min_chunk_bytes_for_parallel_parsing {#min-chunk-bytes-for-parallel-parsing}

View File

@ -255,7 +255,7 @@ windowFunnel(window, [mode, [mode, ... ]])(timestamp, cond1, cond2, ..., condN)
- `window` — Length of the sliding window, it is the time interval between the first and the last condition. The unit of `window` depends on the `timestamp` itself and varies. Determined using the expression `timestamp of cond1 <= timestamp of cond2 <= ... <= timestamp of condN <= timestamp of cond1 + window`.
- `mode` — It is an optional argument. One or more modes can be set.
- `'strict'` — If same condition holds for sequence of events then such non-unique events would be skipped.
- `'strict_deduplication'` — If the same condition holds for the sequence of events, then such repeating event interrupts further processing.
- `'strict_order'` — Don't allow interventions of other events. E.g. in the case of `A->B->D->C`, it stops finding `A->B->C` at the `D` and the max event level is 2.
- `'strict_increase'` — Apply conditions only to events with strictly increasing timestamps.

View File

@ -143,7 +143,9 @@ It works faster than intHash32. Average quality.
## SHA256 {#sha256}
Calculates SHA-1, SHA-224, or SHA-256 from a string and returns the resulting set of bytes as FixedString(20), FixedString(28), or FixedString(32).
## SHA512 {#sha512}
Calculates SHA-1, SHA-224, SHA-256 or SHA-512 from a string and returns the resulting set of bytes as FixedString(20), FixedString(28), FixedString(32), or FixedString(64).
The function works fairly slowly (SHA-1 processes about 5 million short strings per second per processor core, while SHA-224 and SHA-256 process about 2.2 million).
We recommend using this function only in cases when you need a specific hash function and you cant select it.
Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in SELECTS.

View File

@ -6,7 +6,7 @@ toc_title: JOIN
Join produces a new table by combining columns from one or multiple tables by using values common to each. It is a common operation in databases with SQL support, which corresponds to [relational algebra](https://en.wikipedia.org/wiki/Relational_algebra#Joins_and_join-like_operators) join. The special case of one table join is often referred to as “self-join”.
Syntax:
**Syntax**
``` sql
SELECT <expr_list>
@ -38,7 +38,7 @@ Additional join types available in ClickHouse:
## Settings {#join-settings}
The default join type can be overriden using [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) setting.
The default join type can be overridden using [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) setting.
The behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) setting.
@ -52,6 +52,61 @@ The behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_
- [join_on_disk_max_files_to_merge](../../../operations/settings/settings.md#join_on_disk_max_files_to_merge)
- [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys)
## ON Section Conditions {on-section-conditions}
An `ON` section can contain several conditions combined using the `AND` operator. Conditions specifying join keys must refer both left and right tables and must use the equality operator. Other conditions may use other logical operators but they must refer either the left or the right table of a query.
Rows are joined if the whole complex condition is met. If the conditions are not met, still rows may be included in the result depending on the `JOIN` type. Note that if the same conditions are placed in a `WHERE` section and they are not met, then rows are always filtered out from the result.
!!! note "Note"
The `OR` operator inside an `ON` section is not supported yet.
!!! note "Note"
If a condition refers columns from different tables, then only the equality operator (`=`) is supported so far.
**Example**
Consider `table_1` and `table_2`:
```
┌─Id─┬─name─┐ ┌─Id─┬─text───────────┬─scores─┐
│ 1 │ A │ │ 1 │ Text A │ 10 │
│ 2 │ B │ │ 1 │ Another text A │ 12 │
│ 3 │ C │ │ 2 │ Text B │ 15 │
└────┴──────┘ └────┴────────────────┴────────┘
```
Query with one join key condition and an additional condition for `table_2`:
``` sql
SELECT name, text FROM table_1 LEFT OUTER JOIN table_2
ON table_1.Id = table_2.Id AND startsWith(table_2.text, 'Text');
```
Note that the result contains the row with the name `C` and the empty text column. It is included into the result because an `OUTER` type of a join is used.
```
┌─name─┬─text───┐
│ A │ Text A │
│ B │ Text B │
│ C │ │
└──────┴────────┘
```
Query with `INNER` type of a join and multiple conditions:
``` sql
SELECT name, text, scores FROM table_1 INNER JOIN table_2
ON table_1.Id = table_2.Id AND table_2.scores > 10 AND startsWith(table_2.text, 'Text');
```
Result:
```
┌─name─┬─text───┬─scores─┐
│ B │ Text B │ 15 │
└──────┴────────┴────────┘
```
## ASOF JOIN Usage {#asof-join-usage}
`ASOF JOIN` is useful when you need to join records that have no exact match.
@ -59,7 +114,7 @@ The behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_
Algorithm requires the special column in tables. This column:
- Must contain an ordered sequence.
- Can be one of the following types: [Int*, UInt*](../../../sql-reference/data-types/int-uint.md), [Float\*](../../../sql-reference/data-types/float.md), [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md), [Decimal\*](../../../sql-reference/data-types/decimal.md).
- Can be one of the following types: [Int, UInt](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md), [Decimal](../../../sql-reference/data-types/decimal.md).
- Cant be the only column in the `JOIN` clause.
Syntax `ASOF JOIN ... ON`:
@ -84,7 +139,7 @@ ASOF JOIN table_2
USING (equi_column1, ... equi_columnN, asof_column)
```
`ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table_2.asof_column` condition. The `asof_column` column always the last one in the `USING` clause.
`ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table_2.asof_column` condition. The `asof_column` column is always the last one in the `USING` clause.
For example, consider the following tables:

View File

@ -34,5 +34,6 @@ You can use table functions in:
| [odbc](../../sql-reference/table-functions/odbc.md) | Creates a [ODBC](../../engines/table-engines/integrations/odbc.md)-engine table. |
| [hdfs](../../sql-reference/table-functions/hdfs.md) | Creates a [HDFS](../../engines/table-engines/integrations/hdfs.md)-engine table. |
| [s3](../../sql-reference/table-functions/s3.md) | Creates a [S3](../../engines/table-engines/integrations/s3.md)-engine table. |
| [sqlite](../../sql-reference/table-functions/sqlite.md) | Creates a [sqlite](../../engines/table-engines/integrations/sqlite.md)-engine table. |
[Original article](https://clickhouse.tech/docs/en/sql-reference/table-functions/) <!--hide-->

View File

@ -0,0 +1,45 @@
---
toc_priority: 55
toc_title: sqlite
---
## sqlite {#sqlite}
Allows to perform queries on a data stored in an [SQLite](../../engines/database-engines/sqlite.md) database.
**Syntax**
``` sql
sqlite('db_path', 'table_name')
```
**Arguments**
- `db_path` — Path to a file with an SQLite database. [String](../../sql-reference/data-types/string.md).
- `table_name` — Name of a table in the SQLite database. [String](../../sql-reference/data-types/string.md).
**Returned value**
- A table object with the same columns as in the original `SQLite` table.
**Example**
Query:
``` sql
SELECT * FROM sqlite('sqlite.db', 'table1') ORDER BY col2;
```
Result:
``` text
┌─col1──┬─col2─┐
│ line1 │ 1 │
│ line2 │ 2 │
│ line3 │ 3 │
└───────┴──────┘
```
**See Also**
- [SQLite](../../engines/table-engines/integrations/sqlite.md) table engine

View File

@ -0,0 +1,79 @@
---
toc_priority: 32
toc_title: SQLite
---
# SQLite {#sqlite}
Движок баз данных позволяет подключаться к базе [SQLite](https://www.sqlite.org/index.html) и выполнять запросы `INSERT` и `SELECT` для обмена данными между ClickHouse и SQLite.
## Создание базы данных {#creating-a-database}
``` sql
CREATE DATABASE sqlite_database
ENGINE = SQLite('db_path')
```
**Параметры движка**
- `db_path` — путь к файлу с базой данных SQLite.
## Поддерживаемые типы данных {#data_types-support}
| SQLite | ClickHouse |
|---------------|---------------------------------------------------------|
| INTEGER | [Int32](../../sql-reference/data-types/int-uint.md) |
| REAL | [Float32](../../sql-reference/data-types/float.md) |
| TEXT | [String](../../sql-reference/data-types/string.md) |
| BLOB | [String](../../sql-reference/data-types/string.md) |
## Особенности и рекомендации {#specifics-and-recommendations}
SQLite хранит всю базу данных (определения, таблицы, индексы и сами данные) в виде единого кроссплатформенного файла на хост-машине. Во время записи SQLite блокирует весь файл базы данных, поэтому операции записи выполняются последовательно. Операции чтения могут быть многозадачными.
SQLite не требует управления службами (например, сценариями запуска) или контроля доступа на основе `GRANT` и паролей. Контроль доступа осуществляется с помощью разрешений файловой системы, предоставляемых самому файлу базы данных.
## Примеры использования {#usage-example}
Отобразим список таблиц базы данных в ClickHouse, подключенной к SQLite:
``` sql
CREATE DATABASE sqlite_db ENGINE = SQLite('sqlite.db');
SHOW TABLES FROM sqlite_db;
```
``` text
┌──name───┐
│ table1 │
│ table2 │
└─────────┘
```
Отобразим содержимое таблицы:
``` sql
SELECT * FROM sqlite_db.table1;
```
``` text
┌─col1──┬─col2─┐
│ line1 │ 1 │
│ line2 │ 2 │
│ line3 │ 3 │
└───────┴──────┘
```
Вставим данные в таблицу SQLite из таблицы ClickHouse:
``` sql
CREATE TABLE clickhouse_table(`col1` String,`col2` Int16) ENGINE = MergeTree() ORDER BY col2;
INSERT INTO clickhouse_table VALUES ('text',10);
INSERT INTO sqlite_db.table1 SELECT * FROM clickhouse_table;
SELECT * FROM sqlite_db.table1;
```
``` text
┌─col1──┬─col2─┐
│ line1 │ 1 │
│ line2 │ 2 │
│ line3 │ 3 │
│ text │ 10 │
└───────┴──────┘
```

View File

@ -0,0 +1,59 @@
---
toc_priority: 7
toc_title: SQLite
---
# SQLite {#sqlite}
Движок позволяет импортировать и экспортировать данные из SQLite, а также поддерживает отправку запросов к таблицам SQLite напрямую из ClickHouse.
## Создание таблицы {#creating-a-table}
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name
(
name1 [type1],
name2 [type2], ...
) ENGINE = SQLite('db_path', 'table')
```
**Параметры движка**
- `db_path` — путь к файлу с базой данных SQLite.
- `table` — имя таблицы в базе данных SQLite.
## Примеры использования {#usage-example}
Отобразим запрос, с помощью которого была создана таблица SQLite:
```sql
SHOW CREATE TABLE sqlite_db.table2;
```
``` text
CREATE TABLE SQLite.table2
(
`col1` Nullable(Int32),
`col2` Nullable(String)
)
ENGINE = SQLite('sqlite.db','table2');
```
Получим данные из таблицы:
``` sql
SELECT * FROM sqlite_db.table2 ORDER BY col1;
```
```text
┌─col1─┬─col2──┐
│ 1 │ text1 │
│ 2 │ text2 │
│ 3 │ text3 │
└──────┴───────┘
```
**См. также**
- [SQLite](../../../engines/database-engines/sqlite.md) движок баз данных
- [sqlite](../../../sql-reference/table-functions/sqlite.md) табличная функция

View File

@ -86,7 +86,9 @@ ORDER BY expr
- `enable_mixed_granularity_parts` — включает или выключает переход к ограничению размера гранул с помощью настройки `index_granularity_bytes`. Настройка `index_granularity_bytes` улучшает производительность ClickHouse при выборке данных из таблиц с большими (десятки и сотни мегабайтов) строками. Если у вас есть таблицы с большими строками, можно включить эту настройку, чтобы повысить эффективность запросов `SELECT`.
- `use_minimalistic_part_header_in_zookeeper` — Способ хранения заголовков кусков данных в ZooKeeper. Если `use_minimalistic_part_header_in_zookeeper = 1`, то ZooKeeper хранит меньше данных. Подробнее читайте в [описании настройки](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) в разделе "Конфигурационные параметры сервера".
- `min_merge_bytes_to_use_direct_io` — минимальный объём данных при слиянии, необходимый для прямого (небуферизованного) чтения/записи (direct I/O) на диск. При слиянии частей данных ClickHouse вычисляет общий объём хранения всех данных, подлежащих слиянию. Если общий объём хранения всех данных для чтения превышает `min_bytes_to_use_direct_io` байт, тогда ClickHouse использует флаг `O_DIRECT` при чтении данных с диска. Если `min_merge_bytes_to_use_direct_io = 0`, тогда прямой ввод-вывод отключен. Значение по умолчанию: `10 * 1024 * 1024 * 1024` байтов.
- <a name="mergetree_setting-merge_with_ttl_timeout"></a>`merge_with_ttl_timeout` — минимальное время в секундах перед повторным слиянием с TTL. По умолчанию — 86400 (1 день).
- `merge_with_ttl_timeout` — минимальное время в секундах перед повторным слиянием для удаления данных с истекшим TTL. По умолчанию: `14400` секунд (4 часа).
- `merge_with_recompression_ttl_timeout` — минимальное время в секундах перед повторным слиянием для повторного сжатия данных с истекшим TTL. По умолчанию: `14400` секунд (4 часа).
- `try_fetch_recompressed_part_timeout` — время ожидания (в секундах) перед началом слияния с повторным сжатием. В течение этого времени ClickHouse пытается извлечь сжатую часть из реплики, которая назначила это слияние. Значение по умолчанию: `7200` секунд (2 часа).
- `write_final_mark` — включает или отключает запись последней засечки индекса в конце куска данных, указывающей за последний байт. По умолчанию — 1. Не отключайте её.
- `merge_max_block_size` — максимальное количество строк в блоке для операций слияния. Значение по умолчанию: 8192.
- `storage_policy` — политика хранения данных. Смотрите [Хранение данных таблицы на нескольких блочных устройствах](#table_engine-mergetree-multiple-volumes).
@ -401,20 +403,22 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT
## TTL для столбцов и таблиц {#table_engine-mergetree-ttl}
Определяет время жизни значений, а также правила перемещения данных на другой диск или том.
Определяет время жизни значений.
Секция `TTL` может быть установлена как для всей таблицы, так и для каждого отдельного столбца. Правила `TTL` для таблицы позволяют указать целевые диски или тома для фонового перемещения на них частей данных.
Секция `TTL` может быть установлена как для всей таблицы, так и для каждого отдельного столбца. Для таблиц можно установить правила `TTL` для фонового перемещения кусков данных на целевые диски или тома, или правила повторного сжатия кусков данных.
Выражения должны возвращать тип [Date](../../../engines/table-engines/mergetree-family/mergetree.md) или [DateTime](../../../engines/table-engines/mergetree-family/mergetree.md).
Для задания времени жизни столбца, например:
**Синтаксис**
Для задания времени жизни столбца:
``` sql
TTL time_column
TTL time_column + interval
```
Чтобы задать `interval`, используйте операторы [интервала времени](../../../engines/table-engines/mergetree-family/mergetree.md#operators-datetime).
Чтобы задать `interval`, используйте операторы [интервала времени](../../../engines/table-engines/mergetree-family/mergetree.md#operators-datetime), например:
``` sql
TTL date_time + INTERVAL 1 MONTH
@ -423,13 +427,13 @@ TTL date_time + INTERVAL 15 HOUR
### TTL столбца {#mergetree-column-ttl}
Когда срок действия значений в столбце истечет, ClickHouse заменит их значениями по умолчанию для типа данных столбца. Если срок действия всех значений столбцов в части данных истек, ClickHouse удаляет столбец из куска данных в файловой системе.
Когда срок действия значений в столбце истечёт, ClickHouse заменит их значениями по умолчанию для типа данных столбца. Если срок действия всех значений столбцов в части данных истек, ClickHouse удаляет столбец из куска данных в файловой системе.
Секцию `TTL` нельзя использовать для ключевых столбцов.
Примеры:
**Примеры**
Создание таблицы с TTL
Создание таблицы с `TTL`:
``` sql
CREATE TABLE example_table
@ -444,7 +448,7 @@ PARTITION BY toYYYYMM(d)
ORDER BY d;
```
Добавление TTL на колонку существующей таблицы
Добавление `TTL` на колонку существующей таблицы:
``` sql
ALTER TABLE example_table
@ -452,7 +456,7 @@ ALTER TABLE example_table
c String TTL d + INTERVAL 1 DAY;
```
Изменение TTL у колонки
Изменение `TTL` у колонки:
``` sql
ALTER TABLE example_table
@ -462,23 +466,24 @@ ALTER TABLE example_table
### TTL таблицы {#mergetree-table-ttl}
Для таблицы можно задать одно выражение для устаревания данных, а также несколько выражений, по срабатывании которых данные переместятся на [некоторый диск или том](#table_engine-mergetree-multiple-volumes). Когда некоторые данные в таблице устаревают, ClickHouse удаляет все соответствующие строки.
Для таблицы можно задать одно выражение для устаревания данных, а также несколько выражений, при срабатывании которых данные будут перемещены на [некоторый диск или том](#table_engine-mergetree-multiple-volumes). Когда некоторые данные в таблице устаревают, ClickHouse удаляет все соответствующие строки. Операции перемещения или повторного сжатия данных выполняются только когда устаревают все данные в куске.
``` sql
TTL expr
[DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'] ...
[DELETE|RECOMPRESS codec_name1|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|RECOMPRESS codec_name2|TO DISK 'aaa'|TO VOLUME 'bbb'] ...
[WHERE conditions]
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ]
```
За каждым TTL выражением может следовать тип действия, которое выполняется после достижения времени, соответствующего результату TTL выражения:
За каждым `TTL` выражением может следовать тип действия, которое выполняется после достижения времени, соответствующего результату `TTL` выражения:
- `DELETE` - удалить данные (действие по умолчанию);
- `RECOMPRESS codec_name` - повторно сжать данные с помощью кодека `codec_name`;
- `TO DISK 'aaa'` - переместить данные на диск `aaa`;
- `TO VOLUME 'bbb'` - переместить данные на том `bbb`;
- `GROUP BY` - агрегировать данные.
В секции `WHERE` можно задать условие удаления или агрегирования устаревших строк (для перемещения условие `WHERE` не применимо).
В секции `WHERE` можно задать условие удаления или агрегирования устаревших строк (для перемещения и сжатия условие `WHERE` не применимо).
Колонки, по которым агрегируются данные в `GROUP BY`, должны являться префиксом первичного ключа таблицы.
@ -486,7 +491,7 @@ TTL expr
**Примеры**
Создание таблицы с TTL:
Создание таблицы с `TTL`:
``` sql
CREATE TABLE example_table
@ -502,7 +507,7 @@ TTL d + INTERVAL 1 MONTH [DELETE],
d + INTERVAL 2 WEEK TO DISK 'bbb';
```
Изменение TTL:
Изменение `TTL`:
``` sql
ALTER TABLE example_table
@ -523,6 +528,21 @@ ORDER BY d
TTL d + INTERVAL 1 MONTH DELETE WHERE toDayOfWeek(d) = 1;
```
Создание таблицы, в которой куски с устаревшими данными повторно сжимаются:
```sql
CREATE TABLE table_for_recompression
(
d DateTime,
key UInt64,
value String
) ENGINE MergeTree()
ORDER BY tuple()
PARTITION BY key
TTL d + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), d + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10))
SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0;
```
Создание таблицы, где устаревшие строки агрегируются. В результирующих строках колонка `x` содержит максимальное значение по сгруппированным строкам, `y` — минимальное значение, а `d` — случайное значение из одной из сгуппированных строк.
``` sql
@ -539,14 +559,18 @@ ORDER BY (k1, k2)
TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y);
```
**Удаление данных**
### Удаление устаревших данных {#mergetree-removing-expired-data}
Данные с истекшим TTL удаляются, когда ClickHouse мёржит куски данных.
Данные с истекшим `TTL` удаляются, когда ClickHouse мёржит куски данных.
Когда ClickHouse видит, что некоторые данные устарели, он выполняет внеплановые мёржи. Для управление частотой подобных мёржей, можно задать настройку `merge_with_ttl_timeout`. Если её значение слишком низкое, придется выполнять много внеплановых мёржей, которые могут начать потреблять значительную долю ресурсов сервера.
Если вы выполните запрос `SELECT` между слияниями вы можете получить устаревшие данные. Чтобы избежать этого используйте запрос [OPTIMIZE](../../../engines/table-engines/mergetree-family/mergetree.md#misc_operations-optimize) перед `SELECT`.
**См. также**
- настройку [ttl_only_drop_parts](../../../operations/settings/settings.md#ttl_only_drop_parts)
## Хранение данных таблицы на нескольких блочных устройствах {#table_engine-mergetree-multiple-volumes}
### Введение {#introduction}

View File

@ -1865,10 +1865,25 @@ ClickHouse генерирует исключение
## input_format_parallel_parsing {#input-format-parallel-parsing}
- Тип: bool
- Значение по умолчанию: True
Включает или отключает режим, при котором входящие данные разбиваются на части, парсинг каждой из которых осуществляется параллельно с сохранением исходного порядка. Поддерживается только для форматов [TSV](../../interfaces/formats.md#tabseparated), [TKSV](../../interfaces/formats.md#tskv), [CSV](../../interfaces/formats.md#csv) и [JSONEachRow](../../interfaces/formats.md#jsoneachrow).
Включает режим, при котором входящие данные парсятся параллельно, но с сохранением исходного порядка следования. Поддерживается только для форматов TSV, TKSV, CSV и JSONEachRow.
Возможные значения:
- 1 — включен режим параллельного разбора.
- 0 — отключен режим параллельного разбора.
Значение по умолчанию: `0`.
## output_format_parallel_formatting {#output-format-parallel-formatting}
Включает или отключает режим, при котором исходящие данные форматируются параллельно с сохранением исходного порядка. Поддерживается только для форматов [TSV](../../interfaces/formats.md#tabseparated), [TKSV](../../interfaces/formats.md#tskv), [CSV](../../interfaces/formats.md#csv) и [JSONEachRow](../../interfaces/formats.md#jsoneachrow).
Возможные значения:
- 1 — включен режим параллельного форматирования.
- 0 — отключен режим параллельного форматирования.
Значение по умолчанию: `0`.
## min_chunk_bytes_for_parallel_parsing {#min-chunk-bytes-for-parallel-parsing}

View File

@ -6,7 +6,7 @@ toc_title: JOIN
`JOIN` создаёт новую таблицу путем объединения столбцов из одной или нескольких таблиц с использованием общих для каждой из них значений. Это обычная операция в базах данных с поддержкой SQL, которая соответствует join из [реляционной алгебры](https://en.wikipedia.org/wiki/Relational_algebra#Joins_and_join-like_operators). Частный случай соединения одной таблицы часто называют self-join.
Синтаксис:
**Синтаксис**
``` sql
SELECT <expr_list>
@ -19,7 +19,7 @@ FROM <left_table>
## Поддерживаемые типы соединения {#select-join-types}
Все типы из стандартого [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) поддерживаются:
Все типы из стандартного [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) поддерживаются:
- `INNER JOIN`, возвращаются только совпадающие строки.
- `LEFT OUTER JOIN`, не совпадающие строки из левой таблицы возвращаются в дополнение к совпадающим строкам.
@ -33,7 +33,7 @@ FROM <left_table>
- `LEFT SEMI JOIN` и `RIGHT SEMI JOIN`, белый список по ключам соединения, не производит декартово произведение.
- `LEFT ANTI JOIN` и `RIGHT ANTI JOIN`, черный список по ключам соединения, не производит декартово произведение.
- `LEFT ANY JOIN`, `RIGHT ANY JOIN` и `INNER ANY JOIN`, Частично (для противоположных сторон `LEFT` и `RIGHT`) или полностью (для `INNER` и `FULL`) отключает декартово произведение для стандартых видов `JOIN`.
- `LEFT ANY JOIN`, `RIGHT ANY JOIN` и `INNER ANY JOIN`, Частично (для противоположных сторон `LEFT` и `RIGHT`) или полностью (для `INNER` и `FULL`) отключает декартово произведение для стандартных видов `JOIN`.
- `ASOF JOIN` и `LEFT ASOF JOIN`, Для соединения последовательностей по нечеткому совпадению. Использование `ASOF JOIN` описано ниже.
## Настройки {#join-settings}
@ -52,6 +52,61 @@ FROM <left_table>
- [join_on_disk_max_files_to_merge](../../../operations/settings/settings.md#join_on_disk_max_files_to_merge)
- [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys)
## Условия в секции ON {on-section-conditions}
Секция `ON` может содержать несколько условий, связанных оператором `AND`. Условия, задающие ключи соединения, должны содержать столбцы левой и правой таблицы и должны использовать оператор равенства. Прочие условия могут использовать другие логические операторы, но в отдельном условии могут использоваться столбцы либо только левой, либо только правой таблицы.
Строки объединяются только тогда, когда всё составное условие выполнено. Если оно не выполнено, то строки могут попасть в результат в зависимости от типа `JOIN`. Обратите внимание, что если то же самое условие поместить в секцию `WHERE`, то строки, для которых оно не выполняется, никогда не попаду в результат.
!!! note "Примечание"
Оператор `OR` внутри секции `ON` пока не поддерживается.
!!! note "Примечание"
Если в условии использованы столбцы из разных таблиц, то пока поддерживается только оператор равенства (`=`).
**Пример**
Рассмотрим `table_1` и `table_2`:
```
┌─Id─┬─name─┐ ┌─Id─┬─text───────────┬─scores─┐
│ 1 │ A │ │ 1 │ Text A │ 10 │
│ 2 │ B │ │ 1 │ Another text A │ 12 │
│ 3 │ C │ │ 2 │ Text B │ 15 │
└────┴──────┘ └────┴────────────────┴────────┘
```
Запрос с одним условием, задающим ключ соединения, и дополнительным условием для `table_2`:
``` sql
SELECT name, text FROM table_1 LEFT OUTER JOIN table_2
ON table_1.Id = table_2.Id AND startsWith(table_2.text, 'Text');
```
Обратите внимание, что результат содержит строку с именем `C` и пустым текстом. Строка включена в результат, потому что использован тип соединения `OUTER`.
```
┌─name─┬─text───┐
│ A │ Text A │
│ B │ Text B │
│ C │ │
└──────┴────────┘
```
Запрос с типом соединения `INNER` и несколькими условиями:
``` sql
SELECT name, text, scores FROM table_1 INNER JOIN table_2
ON table_1.Id = table_2.Id AND table_2.scores > 10 AND startsWith(table_2.text, 'Text');
```
Результат:
```
┌─name─┬─text───┬─scores─┐
│ B │ Text B │ 15 │
└──────┴────────┴────────┘
```
## Использование ASOF JOIN {#asof-join-usage}
`ASOF JOIN` применим в том случае, когда необходимо объединять записи, которые не имеют точного совпадения.
@ -59,7 +114,7 @@ FROM <left_table>
Для работы алгоритма необходим специальный столбец в таблицах. Этот столбец:
- Должен содержать упорядоченную последовательность.
- Может быть одного из следующих типов: [Int*, UInt*](../../data-types/int-uint.md), [Float*](../../data-types/float.md), [Date](../../data-types/date.md), [DateTime](../../data-types/datetime.md), [Decimal*](../../data-types/decimal.md).
- Может быть одного из следующих типов: [Int, UInt](../../data-types/int-uint.md), [Float](../../data-types/float.md), [Date](../../data-types/date.md), [DateTime](../../data-types/datetime.md), [Decimal](../../data-types/decimal.md).
- Не может быть единственным столбцом в секции `JOIN`.
Синтаксис `ASOF JOIN ... ON`:

View File

@ -0,0 +1,45 @@
---
toc_priority: 55
toc_title: sqlite
---
## sqlite {#sqlite}
Позволяет выполнять запросы к данным, хранящимся в базе данных [SQLite](../../engines/database-engines/sqlite.md).
**Синтаксис**
``` sql
sqlite('db_path', 'table_name')
```
**Аргументы**
- `db_path` — путь к файлу с базой данных SQLite. [String](../../sql-reference/data-types/string.md).
- `table_name` — имя таблицы в базе данных SQLite. [String](../../sql-reference/data-types/string.md).
**Возвращаемое значение**
- Объект таблицы с теми же столбцами, что и в исходной таблице `SQLite`.
**Пример**
Запрос:
``` sql
SELECT * FROM sqlite('sqlite.db', 'table1') ORDER BY col2;
```
Результат:
``` text
┌─col1──┬─col2─┐
│ line1 │ 1 │
│ line2 │ 2 │
│ line3 │ 3 │
└───────┴──────┘
```
**См. также**
- [SQLite](../../engines/table-engines/integrations/sqlite.md) движок таблиц

View File

@ -38,13 +38,13 @@ ENGINE = MySQL('host:port', ['database' | database], 'user', 'password')
| BIGINT | [Int64](../../sql-reference/data-types/int-uint.md) |
| FLOAT | [Float32](../../sql-reference/data-types/float.md) |
| DOUBLE | [Float64](../../sql-reference/data-types/float.md) |
| DATE | [日期](../../sql-reference/data-types/date.md) |
| DATETIME, TIMESTAMP | [日期时间](../../sql-reference/data-types/datetime.md) |
| BINARY | [固定字符串](../../sql-reference/data-types/fixedstring.md) |
| DATE | [Date](../../sql-reference/data-types/date.md) |
| DATETIME, TIMESTAMP | [DateTime](../../sql-reference/data-types/datetime.md) |
| BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) |
其他的MySQL数据类型将全部都转换为[字符串](../../sql-reference/data-types/string.md)。
其他的MySQL数据类型将全部都转换为[String](../../sql-reference/data-types/string.md)。
同时以上的所有类型都支持[可为空](../../sql-reference/data-types/nullable.md)。
同时以上的所有类型都支持[Nullable](../../sql-reference/data-types/nullable.md)。
## 使用示例 {#shi-yong-shi-li}

View File

@ -1,8 +1,8 @@
# 功能与Yandex的工作。梅特里卡词典 {#functions-for-working-with-yandex-metrica-dictionaries}
# 使用 Yandex.Metrica 字典函数 {#functions-for-working-with-yandex-metrica-dictionaries}
为了使下面的功能正常工作,服务器配置必须指定获取所有Yandex的路径和地址。梅特里卡字典. 字典在任何这些函数的第一次调用时加载。 如果无法加载引用列表,则会引发异常。
为了使下面的功能正常工作,服务器配置必须指定获取所有 Yandex.Metrica 字典的路径和地址。Yandex.Metrica 字典在任何这些函数的第一次调用时加载。 如果无法加载引用列表,则会引发异常。
For information about creating reference lists, see the section «Dictionaries».
有关创建引用列表的信息,请参阅 «字典» 部分.
## 多个地理基 {#multiple-geobases}
@ -17,18 +17,18 @@ ClickHouse支持同时使用多个备选地理基区域层次结构
所有字典都在运行时重新加载每隔一定数量的秒重新加载一次如builtin_dictionaries_reload_interval config参数中定义或默认情况下每小时一次。 但是,可用字典列表在服务器启动时定义一次。
All functions for working with regions have an optional argument at the end the dictionary key. It is referred to as the geobase.
所有处理区域的函数都在末尾有一个可选参数—字典键。它被称为地基。
示例:
regionToCountry(RegionID) Uses the default dictionary: /opt/geo/regions_hierarchy.txt
regionToCountry(RegionID, '') Uses the default dictionary: /opt/geo/regions_hierarchy.txt
regionToCountry(RegionID, 'ua') Uses the dictionary for the 'ua' key: /opt/geo/regions_hierarchy_ua.txt
regionToCountry(RegionID) 使用默认路径: /opt/geo/regions_hierarchy.txt
regionToCountry(RegionID, '') 使用默认路径: /opt/geo/regions_hierarchy.txt
regionToCountry(RegionID, 'ua') 使用字典中的'ua' 键: /opt/geo/regions_hierarchy_ua.txt
### ツ环板(ョツ嘉ッツ偲青regionシツ氾カツ鉄ツ工ツ渉\]) {#regiontocityid-geobase}
### regionToCity(id[, geobase]) {#regiontocityid-geobase}
Accepts a UInt32 number the region ID from the Yandex geobase. If this region is a city or part of a city, it returns the region ID for the appropriate city. Otherwise, returns 0.
从 Yandex geobase 接收一个 UInt32 数字类型的区域ID 。如果该区域是一个城市或城市的一部分它将返回相应城市的区域ID。否则,返回0。
### 虏茅驴麓卤戮碌禄路戮鲁拢\]) {#regiontoareaid-geobase}
### regionToArea(id[, geobase]) {#regiontoareaid-geobase}
将区域转换为区域地理数据库中的类型5。 在所有其他方式,这个功能是一样的 regionToCity.
@ -84,36 +84,58 @@ LIMIT 15
│ Federation of Bosnia and Herzegovina │
└──────────────────────────────────────────────────────────┘
### 虏茅驴麓卤戮碌禄路戮鲁拢(陆毛隆隆(803)888-8325\]) {#regiontocountryid-geobase}
### regionToCountry(id[, geobase]) {#regiontocountryid-geobase}
将区域转换为国家。 在所有其他方式,这个功能是一样的 regionToCity.
示例: `regionToCountry(toUInt32(213)) = 225` 转换莫斯科213到俄罗斯225
### 掳胫((禄脢鹿脷露胫鲁隆鹿((酶-11-16""\[脪陆,ase\]) {#regiontocontinentid-geobase}
### regionToContinent(id[, geobase]) {#regiontocontinentid-geobase}
将区域转换为大陆。 在所有其他方式,这个功能是一样的 regionToCity.
示例: `regionToContinent(toUInt32(213)) = 10001` 将莫斯科213转换为欧亚大陆10001
### ツ环板(ョツ嘉ッツ偲青regionャツ静ャツ青サツ催ャツ渉\]) {#regiontopopulationid-geobase}
### regionToTopContinent (#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent}
查找该区域层次结构中最高的大陆。
**语法**
``` sql
regionToTopContinent(id[, geobase])
```
**参数**
- `id` — Yandex geobase 的区域 ID. [UInt32](../../sql-reference/data-types/int-uint.md).
- `geobase` — 字典的建. 参阅 [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). 可选.
**返回值**
- 顶级大陆的标识符(当您在区域层次结构中攀爬时,是后者)。
- 0如果没有。
类型: `UInt32`.
### regionToPopulation(id\[, geobase\]) {#regiontopopulationid-geobase}
获取区域的人口。
The population can be recorded in files with the geobase. See the section «External dictionaries».
人口可以记录在文件与地球基。请参阅«外部词典»部分。
如果没有为该区域记录人口则返回0。
在Yandex地理数据库中可能会为子区域记录人口但不会为父区域记录人口。
### regionIn(lhs,rhs\[,地理数据库\]) {#regioninlhs-rhs-geobase}
检查是否 lhs 属于一个区域 rhs 区域。 如果属于UInt8则返回等于1的数字如果不属于则返回0。
The relationship is reflexive any region also belongs to itself.
这种关系是反射的——任何地区也属于自己。
### ツ暗ェツ氾环催ツ団ツ法ツ人\]) {#regionhierarchyid-geobase}
### regionHierarchy(id\[, geobase\]) {#regionhierarchyid-geobase}
Accepts a UInt32 number the region ID from the Yandex geobase. Returns an array of region IDs consisting of the passed region and all parents along the chain.
从 Yandex geobase 接收一个 UInt32 数字类型的区域ID。返回一个区域ID数组由传递的区域和链上的所有父节点组成。
示例: `regionHierarchy(toUInt32(213)) = [213,1,3,225,10001,10000]`.
### 地区名称(id\[,郎\]) {#regiontonameid-lang}
### regionToName(id\[, lang\]) {#regiontonameid-lang}
Accepts a UInt32 number the region ID from the Yandex geobase. A string with the name of the language can be passed as a second argument. Supported languages are: ru, en, ua, uk, by, kz, tr. If the second argument is omitted, the language ru is used. If the language is not supported, an exception is thrown. Returns a string the name of the region in the corresponding language. If the region with the specified ID doesnt exist, an empty string is returned.
从 Yandex geobase 接收一个 UInt32 数字类型的区域ID。带有语言名称的字符串可以作为第二个参数传递。支持的语言有:ru, en, ua, uk, by, kz, tr。如果省略第二个参数则使用' ru '语言。如果不支持该语言,则抛出异常。返回一个字符串-对应语言的区域名称。如果指定ID的区域不存在则返回一个空字符串。
`ua``uk` 都意味着乌克兰。

View File

@ -129,6 +129,7 @@ namespace ErrorCodes
extern const int UNRECOGNIZED_ARGUMENTS;
extern const int SYNTAX_ERROR;
extern const int TOO_DEEP_RECURSION;
extern const int AUTHENTICATION_FAILED;
}
@ -773,31 +774,50 @@ private:
<< connection_parameters.host << ":" << connection_parameters.port
<< (!connection_parameters.user.empty() ? " as user " + connection_parameters.user : "") << "." << std::endl;
connection = std::make_unique<Connection>(
connection_parameters.host,
connection_parameters.port,
connection_parameters.default_database,
connection_parameters.user,
connection_parameters.password,
"", /* cluster */
"", /* cluster_secret */
"client",
connection_parameters.compression,
connection_parameters.security);
String server_name;
UInt64 server_version_major = 0;
UInt64 server_version_minor = 0;
UInt64 server_version_patch = 0;
if (max_client_network_bandwidth)
try
{
ThrottlerPtr throttler = std::make_shared<Throttler>(max_client_network_bandwidth, 0, "");
connection->setThrottler(throttler);
}
connection = std::make_unique<Connection>(
connection_parameters.host,
connection_parameters.port,
connection_parameters.default_database,
connection_parameters.user,
connection_parameters.password,
"", /* cluster */
"", /* cluster_secret */
"client",
connection_parameters.compression,
connection_parameters.security);
connection->getServerVersion(
connection_parameters.timeouts, server_name, server_version_major, server_version_minor, server_version_patch, server_revision);
if (max_client_network_bandwidth)
{
ThrottlerPtr throttler = std::make_shared<Throttler>(max_client_network_bandwidth, 0, "");
connection->setThrottler(throttler);
}
connection->getServerVersion(
connection_parameters.timeouts, server_name, server_version_major, server_version_minor, server_version_patch, server_revision);
}
catch (const Exception & e)
{
/// It is typical when users install ClickHouse, type some password and instantly forget it.
if ((connection_parameters.user.empty() || connection_parameters.user == "default")
&& e.code() == DB::ErrorCodes::AUTHENTICATION_FAILED)
{
std::cerr << std::endl
<< "If you have installed ClickHouse and forgot password you can reset it in the configuration file." << std::endl
<< "The password for default user is typically located at /etc/clickhouse-server/users.d/default-password.xml" << std::endl
<< "and deleting this file will reset the password." << std::endl
<< "See also /etc/clickhouse-server/users.xml on the server where ClickHouse is installed." << std::endl
<< std::endl;
}
throw;
}
server_version = toString(server_version_major) + "." + toString(server_version_minor) + "." + toString(server_version_patch);

View File

@ -376,8 +376,8 @@ void LocalServer::processQueries()
throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR);
/// Authenticate and create a context to execute queries.
Session session{global_context, ClientInfo::Interface::TCP};
session.authenticate("default", "", Poco::Net::SocketAddress{});
Session session{global_context, ClientInfo::Interface::LOCAL};
session.authenticate("default", "", {});
/// Use the same context for all queries.
auto context = session.makeQueryContext();

View File

@ -357,6 +357,7 @@ void Server::createServer(const std::string & listen_host, const char * port_nam
try
{
func(port);
global_context->registerServerPort(port_name, port);
}
catch (const Poco::Exception &)
{
@ -735,6 +736,10 @@ if (ThreadFuzzer::instance().isEffective())
setupTmpPath(log, disk->getPath());
}
/// Storage keeping all the backups.
fs::create_directories(path / "backups");
global_context->setBackupsVolume(config().getString("backups_path", path / "backups"), config().getString("backups_policy", ""));
/** Directory with 'flags': files indicating temporary settings for the server set by system administrator.
* Flags may be cleared automatically after being applied by the server.
* Examples: do repair of local data; clone all replicated tables from replica.

View File

@ -1,5 +1,5 @@
#include <AggregateFunctions/AggregateFunctionIf.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <AggregateFunctions/AggregateFunctionIf.h>
#include "AggregateFunctionNull.h"
@ -11,6 +11,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_AGGREGATION;
}
class AggregateFunctionCombinatorIf final : public IAggregateFunctionCombinator
@ -37,6 +38,10 @@ public:
const DataTypes & arguments,
const Array & params) const override
{
if (nested_function->getName().find(getName()) != String::npos)
{
throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, "nested function for {0}-combinator must not have {0}-combinator", getName());
}
return std::make_shared<AggregateFunctionIf>(nested_function, arguments, params);
}
};

View File

@ -56,6 +56,8 @@ template <typename Value, bool float_return> using FuncQuantilesTDigestWeighted
template <typename Value, bool float_return> using FuncQuantileBFloat16 = AggregateFunctionQuantile<Value, QuantileBFloat16Histogram<Value>, NameQuantileBFloat16, false, std::conditional_t<float_return, Float64, void>, false>;
template <typename Value, bool float_return> using FuncQuantilesBFloat16 = AggregateFunctionQuantile<Value, QuantileBFloat16Histogram<Value>, NameQuantilesBFloat16, false, std::conditional_t<float_return, Float64, void>, true>;
template <typename Value, bool float_return> using FuncQuantileBFloat16Weighted = AggregateFunctionQuantile<Value, QuantileBFloat16Histogram<Value>, NameQuantileBFloat16Weighted, true, std::conditional_t<float_return, Float64, void>, false>;
template <typename Value, bool float_return> using FuncQuantilesBFloat16Weighted = AggregateFunctionQuantile<Value, QuantileBFloat16Histogram<Value>, NameQuantilesBFloat16Weighted, true, std::conditional_t<float_return, Float64, void>, true>;
template <template <typename, bool> class Function>
static constexpr bool supportDecimal()
@ -167,6 +169,9 @@ void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory)
factory.registerFunction(NameQuantileBFloat16::name, createAggregateFunctionQuantile<FuncQuantileBFloat16>);
factory.registerFunction(NameQuantilesBFloat16::name, { createAggregateFunctionQuantile<FuncQuantilesBFloat16>, properties });
factory.registerFunction(NameQuantileBFloat16Weighted::name, createAggregateFunctionQuantile<FuncQuantileBFloat16Weighted>);
factory.registerFunction(NameQuantilesBFloat16Weighted::name, createAggregateFunctionQuantile<FuncQuantilesBFloat16Weighted>);
/// 'median' is an alias for 'quantile'
factory.registerAlias("median", NameQuantile::name);
factory.registerAlias("medianDeterministic", NameQuantileDeterministic::name);
@ -179,6 +184,7 @@ void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory)
factory.registerAlias("medianTDigest", NameQuantileTDigest::name);
factory.registerAlias("medianTDigestWeighted", NameQuantileTDigestWeighted::name);
factory.registerAlias("medianBFloat16", NameQuantileBFloat16::name);
factory.registerAlias("medianBFloat16Weighted", NameQuantileBFloat16Weighted::name);
}
}

View File

@ -237,5 +237,7 @@ struct NameQuantilesTDigestWeighted { static constexpr auto name = "quantilesTDi
struct NameQuantileBFloat16 { static constexpr auto name = "quantileBFloat16"; };
struct NameQuantilesBFloat16 { static constexpr auto name = "quantilesBFloat16"; };
struct NameQuantileBFloat16Weighted { static constexpr auto name = "quantileBFloat16Weighted"; };
struct NameQuantilesBFloat16Weighted { static constexpr auto name = "quantilesBFloat16Weighted"; };
}

View File

@ -137,8 +137,8 @@ class AggregateFunctionWindowFunnel final
private:
UInt64 window;
UInt8 events_size;
/// When the 'strict' is set, it applies conditions only for the not repeating values.
bool strict;
/// When the 'strict_deduplication' is set, it applies conditions only for the not repeating values.
bool strict_deduplication;
/// When the 'strict_order' is set, it doesn't allow interventions of other events.
/// In the case of 'A->B->D->C', it stops finding 'A->B->C' at the 'D' and the max event level is 2.
@ -150,7 +150,7 @@ private:
/// Loop through the entire events_list, update the event timestamp value
/// The level path must be 1---2---3---...---check_events_size, find the max event level that satisfied the path in the sliding window.
/// If found, returns the max event level, else return 0.
/// The Algorithm complexity is O(n).
/// The algorithm works in O(n) time, but the overall function works in O(n * log(n)) due to sorting.
UInt8 getEventLevel(Data & data) const
{
if (data.size() == 0)
@ -163,10 +163,10 @@ private:
/// events_timestamp stores the timestamp of the first and previous i-th level event happen within time window
std::vector<std::optional<std::pair<UInt64, UInt64>>> events_timestamp(events_size);
bool first_event = false;
for (const auto & pair : data.events_list)
for (size_t i = 0; i < data.events_list.size(); ++i)
{
const T & timestamp = pair.first;
const auto & event_idx = pair.second - 1;
const T & timestamp = data.events_list[i].first;
const auto & event_idx = data.events_list[i].second - 1;
if (strict_order && event_idx == -1)
{
if (first_event)
@ -179,9 +179,9 @@ private:
events_timestamp[0] = std::make_pair(timestamp, timestamp);
first_event = true;
}
else if (strict && events_timestamp[event_idx].has_value())
else if (strict_deduplication && events_timestamp[event_idx].has_value())
{
return event_idx + 1;
return data.events_list[i - 1].second;
}
else if (strict_order && first_event && !events_timestamp[event_idx - 1].has_value())
{
@ -226,18 +226,20 @@ public:
events_size = arguments.size() - 1;
window = params.at(0).safeGet<UInt64>();
strict = false;
strict_deduplication = false;
strict_order = false;
strict_increase = false;
for (size_t i = 1; i < params.size(); ++i)
{
String option = params.at(i).safeGet<String>();
if (option == "strict")
strict = true;
if (option == "strict_deduplication")
strict_deduplication = true;
else if (option == "strict_order")
strict_order = true;
else if (option == "strict_increase")
strict_increase = true;
else if (option == "strict")
throw Exception{"strict is replaced with strict_deduplication in Aggregate function " + getName(), ErrorCodes::BAD_ARGUMENTS};
else
throw Exception{"Aggregate function " + getName() + " doesn't support a parameter: " + option, ErrorCodes::BAD_ARGUMENTS};
}

View File

@ -0,0 +1,28 @@
#include <Backups/BackupEntryConcat.h>
#include <IO/ConcatReadBuffer.h>
namespace DB
{
BackupEntryConcat::BackupEntryConcat(
BackupEntryPtr first_source_,
BackupEntryPtr second_source_,
const std::optional<UInt128> & checksum_)
: first_source(std::move(first_source_))
, second_source(std::move(second_source_))
, checksum(checksum_)
{
}
UInt64 BackupEntryConcat::getSize() const
{
if (!size)
size = first_source->getSize() + second_source->getSize();
return *size;
}
std::unique_ptr<ReadBuffer> BackupEntryConcat::getReadBuffer() const
{
return std::make_unique<ConcatReadBuffer>(first_source->getReadBuffer(), second_source->getReadBuffer());
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <Backups/IBackupEntry.h>
namespace DB
{
/// Concatenates data of two backup entries.
class BackupEntryConcat : public IBackupEntry
{
public:
/// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data.
BackupEntryConcat(
BackupEntryPtr first_source_,
BackupEntryPtr second_source_,
const std::optional<UInt128> & checksum_ = {});
UInt64 getSize() const override;
std::optional<UInt128> getChecksum() const override { return checksum; }
std::unique_ptr<ReadBuffer> getReadBuffer() const override;
private:
BackupEntryPtr first_source;
BackupEntryPtr second_source;
mutable std::optional<UInt64> size;
std::optional<UInt128> checksum;
};
}

View File

@ -0,0 +1,35 @@
#include <Backups/BackupEntryFromAppendOnlyFile.h>
#include <IO/LimitReadBuffer.h>
namespace DB
{
BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile(
const String & file_path_,
const std::optional<UInt64> & file_size_,
const std::optional<UInt128> & checksum_,
const std::shared_ptr<Poco::TemporaryFile> & temporary_file_)
: BackupEntryFromImmutableFile(file_path_, file_size_, checksum_, temporary_file_)
, limit(BackupEntryFromImmutableFile::getSize())
{
}
BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile(
const DiskPtr & disk_,
const String & file_path_,
const std::optional<UInt64> & file_size_,
const std::optional<UInt128> & checksum_,
const std::shared_ptr<TemporaryFileOnDisk> & temporary_file_)
: BackupEntryFromImmutableFile(disk_, file_path_, file_size_, checksum_, temporary_file_)
, limit(BackupEntryFromImmutableFile::getSize())
{
}
std::unique_ptr<ReadBuffer> BackupEntryFromAppendOnlyFile::getReadBuffer() const
{
auto buf = BackupEntryFromImmutableFile::getReadBuffer();
return std::make_unique<LimitReadBuffer>(std::move(buf), limit, true);
}
}

View File

@ -0,0 +1,35 @@
#pragma once
#include <Backups/BackupEntryFromImmutableFile.h>
namespace DB
{
/// Represents a file prepared to be included in a backup, assuming that until this backup entry is destroyed
/// the file can be appended with new data, but the bytes which are already in the file won't be changed.
class BackupEntryFromAppendOnlyFile : public BackupEntryFromImmutableFile
{
public:
/// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data.
BackupEntryFromAppendOnlyFile(
const String & file_path_,
const std::optional<UInt64> & file_size_ = {},
const std::optional<UInt128> & checksum_ = {},
const std::shared_ptr<Poco::TemporaryFile> & temporary_file_ = {});
BackupEntryFromAppendOnlyFile(
const DiskPtr & disk_,
const String & file_path_,
const std::optional<UInt64> & file_size_ = {},
const std::optional<UInt128> & checksum_ = {},
const std::shared_ptr<TemporaryFileOnDisk> & temporary_file_ = {});
UInt64 getSize() const override { return limit; }
std::unique_ptr<ReadBuffer> getReadBuffer() const override;
private:
const UInt64 limit;
};
}

View File

@ -0,0 +1,47 @@
#include <Backups/BackupEntryFromImmutableFile.h>
#include <Disks/IDisk.h>
#include <IO/createReadBufferFromFileBase.h>
#include <Poco/File.h>
namespace DB
{
BackupEntryFromImmutableFile::BackupEntryFromImmutableFile(
const String & file_path_,
const std::optional<UInt64> & file_size_,
const std::optional<UInt128> & checksum_,
const std::shared_ptr<Poco::TemporaryFile> & temporary_file_)
: file_path(file_path_), file_size(file_size_), checksum(checksum_), temporary_file(temporary_file_)
{
}
BackupEntryFromImmutableFile::BackupEntryFromImmutableFile(
const DiskPtr & disk_,
const String & file_path_,
const std::optional<UInt64> & file_size_,
const std::optional<UInt128> & checksum_,
const std::shared_ptr<TemporaryFileOnDisk> & temporary_file_)
: disk(disk_), file_path(file_path_), file_size(file_size_), checksum(checksum_), temporary_file_on_disk(temporary_file_)
{
}
BackupEntryFromImmutableFile::~BackupEntryFromImmutableFile() = default;
UInt64 BackupEntryFromImmutableFile::getSize() const
{
std::lock_guard lock{get_file_size_mutex};
if (!file_size)
file_size = disk ? disk->getFileSize(file_path) : Poco::File(file_path).getSize();
return *file_size;
}
std::unique_ptr<ReadBuffer> BackupEntryFromImmutableFile::getReadBuffer() const
{
if (disk)
return disk->readFile(file_path);
else
return createReadBufferFromFileBase(file_path, 0, 0, 0, nullptr);
}
}

View File

@ -0,0 +1,51 @@
#pragma once
#include <Backups/IBackupEntry.h>
#include <mutex>
namespace Poco { class TemporaryFile; }
namespace DB
{
class TemporaryFileOnDisk;
class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
/// Represents a file prepared to be included in a backup, assuming that until this backup entry is destroyed the file won't be changed.
class BackupEntryFromImmutableFile : public IBackupEntry
{
public:
/// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data.
BackupEntryFromImmutableFile(
const String & file_path_,
const std::optional<UInt64> & file_size_ = {},
const std::optional<UInt128> & checksum_ = {},
const std::shared_ptr<Poco::TemporaryFile> & temporary_file_ = {});
BackupEntryFromImmutableFile(
const DiskPtr & disk_,
const String & file_path_,
const std::optional<UInt64> & file_size_ = {},
const std::optional<UInt128> & checksum_ = {},
const std::shared_ptr<TemporaryFileOnDisk> & temporary_file_ = {});
~BackupEntryFromImmutableFile() override;
UInt64 getSize() const override;
std::optional<UInt128> getChecksum() const override { return checksum; }
std::unique_ptr<ReadBuffer> getReadBuffer() const override;
String getFilePath() const { return file_path; }
DiskPtr getDisk() const { return disk; }
private:
const DiskPtr disk;
const String file_path;
mutable std::optional<UInt64> file_size;
mutable std::mutex get_file_size_mutex;
const std::optional<UInt128> checksum;
const std::shared_ptr<Poco::TemporaryFile> temporary_file;
const std::shared_ptr<TemporaryFileOnDisk> temporary_file_on_disk;
};
}

View File

@ -0,0 +1,23 @@
#include <Backups/BackupEntryFromMemory.h>
#include <IO/ReadBufferFromString.h>
namespace DB
{
BackupEntryFromMemory::BackupEntryFromMemory(const void * data_, size_t size_, const std::optional<UInt128> & checksum_)
: BackupEntryFromMemory(String{reinterpret_cast<const char *>(data_), size_}, checksum_)
{
}
BackupEntryFromMemory::BackupEntryFromMemory(String data_, const std::optional<UInt128> & checksum_)
: data(std::move(data_)), checksum(checksum_)
{
}
std::unique_ptr<ReadBuffer> BackupEntryFromMemory::getReadBuffer() const
{
return std::make_unique<ReadBufferFromString>(data);
}
}

View File

@ -0,0 +1,27 @@
#pragma once
#include <Backups/IBackupEntry.h>
#include <IO/ReadBufferFromString.h>
namespace DB
{
/// Represents small preloaded data to be included in a backup.
class BackupEntryFromMemory : public IBackupEntry
{
public:
/// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data.
BackupEntryFromMemory(const void * data_, size_t size_, const std::optional<UInt128> & checksum_ = {});
BackupEntryFromMemory(String data_, const std::optional<UInt128> & checksum_ = {});
UInt64 getSize() const override { return data.size(); }
std::optional<UInt128> getChecksum() const override { return checksum; }
std::unique_ptr<ReadBuffer> getReadBuffer() const override;
private:
const String data;
const std::optional<UInt128> checksum;
};
}

View File

@ -0,0 +1,39 @@
#include <Backups/BackupEntryFromSmallFile.h>
#include <Disks/IDisk.h>
#include <IO/createReadBufferFromFileBase.h>
#include <IO/ReadHelpers.h>
namespace DB
{
namespace
{
String readFile(const String & file_path)
{
auto buf = createReadBufferFromFileBase(file_path, 0, 0, 0, nullptr);
String s;
readStringUntilEOF(s, *buf);
return s;
}
String readFile(const DiskPtr & disk, const String & file_path)
{
auto buf = disk->readFile(file_path);
String s;
readStringUntilEOF(s, *buf);
return s;
}
}
BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_, const std::optional<UInt128> & checksum_)
: BackupEntryFromMemory(readFile(file_path_), checksum_), file_path(file_path_)
{
}
BackupEntryFromSmallFile::BackupEntryFromSmallFile(
const DiskPtr & disk_, const String & file_path_, const std::optional<UInt128> & checksum_)
: BackupEntryFromMemory(readFile(disk_, file_path_), checksum_), disk(disk_), file_path(file_path_)
{
}
}

View File

@ -0,0 +1,34 @@
#pragma once
#include <Backups/BackupEntryFromMemory.h>
namespace DB
{
class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
/// Represents a file prepared to be included in a backup,
/// assuming that the file is small and can be easily loaded into memory.
class BackupEntryFromSmallFile : public BackupEntryFromMemory
{
public:
/// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data.
BackupEntryFromSmallFile(
const String & file_path_,
const std::optional<UInt128> & checksum_ = {});
BackupEntryFromSmallFile(
const DiskPtr & disk_,
const String & file_path_,
const std::optional<UInt128> & checksum_ = {});
String getFilePath() const { return file_path; }
DiskPtr getDisk() const { return disk; }
private:
const DiskPtr disk;
const String file_path;
};
}

View File

@ -0,0 +1,65 @@
#include <Backups/BackupFactory.h>
#include <Backups/BackupInDirectory.h>
#include <Interpreters/Context.h>
#include <Disks/IVolume.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BACKUP_NOT_FOUND;
extern const int BACKUP_ALREADY_EXISTS;
extern const int NOT_ENOUGH_SPACE;
extern const int LOGICAL_ERROR;
}
BackupFactory & BackupFactory::instance()
{
static BackupFactory the_instance;
return the_instance;
}
void BackupFactory::setBackupsVolume(VolumePtr backups_volume_)
{
backups_volume = backups_volume_;
}
BackupMutablePtr BackupFactory::createBackup(const String & backup_name, UInt64 estimated_backup_size, const BackupPtr & base_backup) const
{
if (!backups_volume)
throw Exception(ErrorCodes::LOGICAL_ERROR, "No backups volume");
for (const auto & disk : backups_volume->getDisks())
{
if (disk->exists(backup_name))
throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} already exists", quoteString(backup_name));
}
auto reservation = backups_volume->reserve(estimated_backup_size);
if (!reservation)
throw Exception(
ErrorCodes::NOT_ENOUGH_SPACE,
"Couldn't reserve {} bytes of free space for new backup {}",
estimated_backup_size,
quoteString(backup_name));
return std::make_shared<BackupInDirectory>(IBackup::OpenMode::WRITE, reservation->getDisk(), backup_name, base_backup);
}
BackupPtr BackupFactory::openBackup(const String & backup_name, const BackupPtr & base_backup) const
{
if (!backups_volume)
throw Exception(ErrorCodes::LOGICAL_ERROR, "No backups volume");
for (const auto & disk : backups_volume->getDisks())
{
if (disk->exists(backup_name))
return std::make_shared<BackupInDirectory>(IBackup::OpenMode::READ, disk, backup_name, base_backup);
}
throw Exception(ErrorCodes::BACKUP_NOT_FOUND, "Backup {} not found", quoteString(backup_name));
}
}

View File

@ -0,0 +1,38 @@
#pragma once
#include <Core/Types.h>
#include <boost/noncopyable.hpp>
#include <memory>
namespace DB
{
class IBackup;
using BackupPtr = std::shared_ptr<const IBackup>;
using BackupMutablePtr = std::shared_ptr<IBackup>;
class Context;
using ContextMutablePtr = std::shared_ptr<Context>;
class IVolume;
using VolumePtr = std::shared_ptr<IVolume>;
/// Factory for implementations of the IBackup interface.
class BackupFactory : boost::noncopyable
{
public:
static BackupFactory & instance();
/// Must be called to initialize the backup factory.
void setBackupsVolume(VolumePtr backups_volume_);
/// Creates a new backup and open it for writing.
BackupMutablePtr createBackup(const String & backup_name, UInt64 estimated_backup_size, const BackupPtr & base_backup = {}) const;
/// Opens an existing backup for reading.
BackupPtr openBackup(const String & backup_name, const BackupPtr & base_backup = {}) const;
private:
VolumePtr backups_volume;
};
}

View File

@ -0,0 +1,454 @@
#include <Backups/BackupInDirectory.h>
#include <Backups/BackupFactory.h>
#include <Backups/BackupEntryConcat.h>
#include <Backups/BackupEntryFromImmutableFile.h>
#include <Backups/BackupEntryFromMemory.h>
#include <Backups/IBackupEntry.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/typeid_cast.h>
#include <Common/quoteString.h>
#include <Disks/DiskSelector.h>
#include <Disks/IDisk.h>
#include <IO/HashingReadBuffer.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/ReadHelpers.h>
#include <IO/SeekableReadBuffer.h>
#include <IO/WriteBufferFromFileBase.h>
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
#include <boost/range/adaptor/map.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int BACKUP_NOT_FOUND;
extern const int BACKUP_ALREADY_EXISTS;
extern const int BACKUP_VERSION_NOT_SUPPORTED;
extern const int BACKUP_DAMAGED;
extern const int NO_BASE_BACKUP;
extern const int WRONG_BASE_BACKUP;
extern const int BACKUP_ENTRY_ALREADY_EXISTS;
extern const int BACKUP_ENTRY_NOT_FOUND;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
namespace
{
const UInt64 BACKUP_VERSION = 1;
}
BackupInDirectory::BackupInDirectory(OpenMode open_mode_, const DiskPtr & disk_, const String & path_, const std::shared_ptr<const IBackup> & base_backup_)
: open_mode(open_mode_), disk(disk_), path(path_), path_with_sep(path_), base_backup(base_backup_)
{
if (!path_with_sep.ends_with('/'))
path_with_sep += '/';
trimRight(path, '/');
open();
}
BackupInDirectory::~BackupInDirectory()
{
close();
}
void BackupInDirectory::open()
{
if (open_mode == OpenMode::WRITE)
{
if (disk->exists(path))
throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} already exists", quoteString(path));
disk->createDirectories(path);
directory_was_created = true;
writePathToBaseBackup();
}
if (open_mode == OpenMode::READ)
{
if (!disk->isDirectory(path))
throw Exception(ErrorCodes::BACKUP_NOT_FOUND, "Backup {} not found", quoteString(path));
readContents();
readPathToBaseBackup();
}
}
void BackupInDirectory::close()
{
if (open_mode == OpenMode::WRITE)
{
if (!finalized && directory_was_created)
{
/// Creating of the backup wasn't finished correctly,
/// so the backup cannot be used and it's better to remove its files.
disk->removeRecursive(path);
}
}
}
void BackupInDirectory::writePathToBaseBackup()
{
String file_path = path_with_sep + ".base_backup";
if (!base_backup)
{
disk->removeFileIfExists(file_path);
return;
}
auto out = disk->writeFile(file_path);
writeString(base_backup->getPath(), *out);
}
void BackupInDirectory::readPathToBaseBackup()
{
if (base_backup)
return;
String file_path = path_with_sep + ".base_backup";
if (!disk->exists(file_path))
return;
auto in = disk->readFile(file_path);
String base_backup_path;
readStringUntilEOF(base_backup_path, *in);
if (base_backup_path.empty())
return;
base_backup = BackupFactory::instance().openBackup(base_backup_path);
}
void BackupInDirectory::writeContents()
{
auto out = disk->writeFile(path_with_sep + ".contents");
writeVarUInt(BACKUP_VERSION, *out);
writeVarUInt(infos.size(), *out);
for (const auto & [path_in_backup, info] : infos)
{
writeBinary(path_in_backup, *out);
writeVarUInt(info.size, *out);
if (info.size)
{
writeBinary(info.checksum, *out);
writeVarUInt(info.base_size, *out);
if (info.base_size && (info.base_size != info.size))
writeBinary(info.base_checksum, *out);
}
}
}
void BackupInDirectory::readContents()
{
auto in = disk->readFile(path_with_sep + ".contents");
UInt64 version;
readVarUInt(version, *in);
if (version != BACKUP_VERSION)
throw Exception(ErrorCodes::BACKUP_VERSION_NOT_SUPPORTED, "Backup {}: Version {} is not supported", quoteString(path), version);
size_t num_infos;
readVarUInt(num_infos, *in);
infos.clear();
for (size_t i = 0; i != num_infos; ++i)
{
String path_in_backup;
readBinary(path_in_backup, *in);
EntryInfo info;
readVarUInt(info.size, *in);
if (info.size)
{
readBinary(info.checksum, *in);
readVarUInt(info.base_size, *in);
if (info.base_size && (info.base_size != info.size))
readBinary(info.base_checksum, *in);
else if (info.base_size)
info.base_checksum = info.checksum;
}
infos.emplace(path_in_backup, info);
}
}
IBackup::OpenMode BackupInDirectory::getOpenMode() const
{
return open_mode;
}
String BackupInDirectory::getPath() const
{
return path;
}
Strings BackupInDirectory::list(const String & prefix, const String & terminator) const
{
if (!prefix.ends_with('/') && !prefix.empty())
throw Exception("prefix should end with '/'", ErrorCodes::BAD_ARGUMENTS);
std::lock_guard lock{mutex};
Strings elements;
for (auto it = infos.lower_bound(prefix); it != infos.end(); ++it)
{
const String & name = it->first;
if (!name.starts_with(prefix))
break;
size_t start_pos = prefix.length();
size_t end_pos = String::npos;
if (!terminator.empty())
end_pos = name.find(terminator, start_pos);
std::string_view new_element = std::string_view{name}.substr(start_pos, end_pos - start_pos);
if (!elements.empty() && (elements.back() == new_element))
continue;
elements.push_back(String{new_element});
}
return elements;
}
bool BackupInDirectory::exists(const String & name) const
{
std::lock_guard lock{mutex};
return infos.count(name) != 0;
}
size_t BackupInDirectory::getSize(const String & name) const
{
std::lock_guard lock{mutex};
auto it = infos.find(name);
if (it == infos.end())
throw Exception(
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", quoteString(path), quoteString(name));
return it->second.size;
}
UInt128 BackupInDirectory::getChecksum(const String & name) const
{
std::lock_guard lock{mutex};
auto it = infos.find(name);
if (it == infos.end())
throw Exception(
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", quoteString(path), quoteString(name));
return it->second.checksum;
}
BackupEntryPtr BackupInDirectory::read(const String & name) const
{
std::lock_guard lock{mutex};
auto it = infos.find(name);
if (it == infos.end())
throw Exception(
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", quoteString(path), quoteString(name));
const auto & info = it->second;
if (!info.size)
{
/// Entry's data is empty.
return std::make_unique<BackupEntryFromMemory>(nullptr, 0, UInt128{0, 0});
}
if (!info.base_size)
{
/// Data goes completely from this backup, the base backup isn't used.
return std::make_unique<BackupEntryFromImmutableFile>(disk, path_with_sep + name, info.size, info.checksum);
}
if (info.size < info.base_size)
{
throw Exception(
ErrorCodes::BACKUP_DAMAGED,
"Backup {}: Entry {} has its data size less than in the base backup {}: {} < {}",
quoteString(path), quoteString(name), quoteString(base_backup->getPath()), info.size, info.base_size);
}
if (!base_backup)
{
throw Exception(
ErrorCodes::NO_BASE_BACKUP,
"Backup {}: Entry {} is marked to be read from a base backup, but there is no base backup specified",
quoteString(path), quoteString(name));
}
if (!base_backup->exists(name))
{
throw Exception(
ErrorCodes::WRONG_BASE_BACKUP,
"Backup {}: Entry {} is marked to be read from a base backup, but doesn't exist there",
quoteString(path), quoteString(name));
}
auto base_entry = base_backup->read(name);
auto base_size = base_entry->getSize();
if (base_size != info.base_size)
{
throw Exception(
ErrorCodes::WRONG_BASE_BACKUP,
"Backup {}: Entry {} has unexpected size in the base backup {}: {} (expected size: {})",
quoteString(path), quoteString(name), quoteString(base_backup->getPath()), base_size, info.base_size);
}
auto base_checksum = base_entry->getChecksum();
if (base_checksum && (*base_checksum != info.base_checksum))
{
throw Exception(
ErrorCodes::WRONG_BASE_BACKUP,
"Backup {}: Entry {} has unexpected checksum in the base backup {}",
quoteString(path), quoteString(name), quoteString(base_backup->getPath()));
}
if (info.size == info.base_size)
{
/// Data goes completely from the base backup (nothing goes from this backup).
return base_entry;
}
/// The beginning of the data goes from the base backup,
/// and the ending goes from this backup.
return std::make_unique<BackupEntryConcat>(
std::move(base_entry),
std::make_unique<BackupEntryFromImmutableFile>(disk, path_with_sep + name, info.size - info.base_size),
info.checksum);
}
void BackupInDirectory::write(const String & name, BackupEntryPtr entry)
{
std::lock_guard lock{mutex};
if (open_mode != OpenMode::WRITE)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal operation: Cannot write to a backup opened for reading");
if (infos.contains(name))
throw Exception(
ErrorCodes::BACKUP_ENTRY_ALREADY_EXISTS, "Backup {}: Entry {} already exists", quoteString(path), quoteString(name));
UInt64 size = entry->getSize();
std::optional<UInt128> checksum = entry->getChecksum();
/// Check if the entry's data is empty.
if (!size)
{
infos.emplace(name, EntryInfo{});
return;
}
/// Check if a entry with such name exists in the base backup.
bool base_exists = (base_backup && base_backup->exists(name));
UInt64 base_size = 0;
UInt128 base_checksum{0, 0};
if (base_exists)
{
base_size = base_backup->getSize(name);
base_checksum = base_backup->getChecksum(name);
}
std::unique_ptr<ReadBuffer> read_buffer; /// We'll set that later.
UInt64 read_pos = 0; /// Current position in read_buffer.
/// Determine whether it's possible to receive this entry's data from the base backup completely or partly.
bool use_base = false;
if (base_exists && base_size)
{
if (size == base_size)
{
/// The size is the same, we need to compare checksums to find out
/// if the entry's data has not been changed since the base backup.
if (!checksum)
{
read_buffer = entry->getReadBuffer();
HashingReadBuffer hashing_read_buffer{*read_buffer};
hashing_read_buffer.ignore(size);
read_pos = size;
checksum = hashing_read_buffer.getHash();
}
if (checksum == base_checksum)
use_base = true; /// The data has not been changed.
}
else if (size > base_size)
{
/// The size has been increased, we need to calculate a partial checksum to find out
/// if the entry's data has been only appended since the base backup.
read_buffer = entry->getReadBuffer();
HashingReadBuffer hashing_read_buffer{*read_buffer};
hashing_read_buffer.ignore(base_size);
UInt128 partial_checksum = hashing_read_buffer.getHash();
read_pos = base_size;
if (!checksum)
{
hashing_read_buffer.ignore(size - base_size);
checksum = hashing_read_buffer.getHash();
read_pos = size;
}
if (partial_checksum == base_checksum)
use_base = true; /// The data has been appended.
}
}
if (use_base && (size == base_size))
{
/// The entry's data has not been changed since the base backup.
EntryInfo info;
info.size = base_size;
info.checksum = base_checksum;
info.base_size = base_size;
info.base_checksum = base_checksum;
infos.emplace(name, info);
return;
}
{
/// Either the entry wasn't exist in the base backup
/// or the entry has data appended to the end of the data from the base backup.
/// In both those cases we have to copy data to this backup.
/// Find out where the start position to copy data is.
auto copy_pos = use_base ? base_size : 0;
/// Move the current read position to the start position to copy data.
/// If `read_buffer` is seekable it's easier, otherwise we can use ignore().
if ((read_pos > copy_pos) && !typeid_cast<SeekableReadBuffer *>(read_buffer.get()))
{
read_buffer.reset();
read_pos = 0;
}
if (!read_buffer)
read_buffer = entry->getReadBuffer();
if (read_pos != copy_pos)
{
if (auto * seekable_buffer = typeid_cast<SeekableReadBuffer *>(read_buffer.get()))
seekable_buffer->seek(copy_pos, SEEK_SET);
else if (copy_pos)
read_buffer->ignore(copy_pos - read_pos);
}
/// If we haven't received or calculated a checksum yet, calculate it now.
ReadBuffer * maybe_hashing_read_buffer = read_buffer.get();
std::optional<HashingReadBuffer> hashing_read_buffer;
if (!checksum)
maybe_hashing_read_buffer = &hashing_read_buffer.emplace(*read_buffer);
/// Copy the entry's data after `copy_pos`.
String out_file_path = path_with_sep + name;
disk->createDirectories(directoryPath(out_file_path));
auto out = disk->writeFile(out_file_path);
copyData(*maybe_hashing_read_buffer, *out, size - copy_pos);
if (hashing_read_buffer)
checksum = hashing_read_buffer->getHash();
/// Done!
EntryInfo info;
info.size = size;
info.checksum = *checksum;
if (use_base)
{
info.base_size = base_size;
info.base_checksum = base_checksum;
}
infos.emplace(name, info);
}
}
void BackupInDirectory::finalizeWriting()
{
if (open_mode != OpenMode::WRITE)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal operation: Cannot write to a backup opened for reading");
writeContents();
finalized = true;
}
}

View File

@ -0,0 +1,66 @@
#pragma once
#include <Backups/IBackup.h>
#include <map>
#include <mutex>
namespace DB
{
class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
/// Represents a backup stored on a disk.
/// A backup is stored as a directory, each entry is stored as a file in that directory.
/// Also three system files are stored:
/// 1) ".base" is an XML file with information about the base backup.
/// 2) ".contents" is a binary file containing a list of all entries along with their sizes
/// and checksums and information whether the base backup should be used for each entry
/// 3) ".write_lock" is a temporary empty file which is created before writing of a backup
/// and deleted after finishing that writing.
class BackupInDirectory : public IBackup
{
public:
BackupInDirectory(OpenMode open_mode_, const DiskPtr & disk_, const String & path_, const std::shared_ptr<const IBackup> & base_backup_ = {});
~BackupInDirectory() override;
OpenMode getOpenMode() const override;
String getPath() const override;
Strings list(const String & prefix, const String & terminator) const override;
bool exists(const String & name) const override;
size_t getSize(const String & name) const override;
UInt128 getChecksum(const String & name) const override;
BackupEntryPtr read(const String & name) const override;
void write(const String & name, BackupEntryPtr entry) override;
void finalizeWriting() override;
private:
void open();
void close();
void writePathToBaseBackup();
void readPathToBaseBackup();
void writeContents();
void readContents();
struct EntryInfo
{
UInt64 size = 0;
UInt128 checksum{0, 0};
/// for incremental backups
UInt64 base_size = 0;
UInt128 base_checksum{0, 0};
};
const OpenMode open_mode;
const DiskPtr disk;
String path;
String path_with_sep;
std::shared_ptr<const IBackup> base_backup;
std::map<String, EntryInfo> infos;
bool directory_was_created = false;
bool finalized = false;
mutable std::mutex mutex;
};
}

View File

@ -0,0 +1,89 @@
#include <Backups/BackupRenamingConfig.h>
#include <Parsers/ASTBackupQuery.h>
namespace DB
{
using Kind = ASTBackupQuery::Kind;
using ElementType = ASTBackupQuery::ElementType;
void BackupRenamingConfig::setNewTableName(const DatabaseAndTableName & old_table_name, const DatabaseAndTableName & new_table_name)
{
old_to_new_table_names[old_table_name] = new_table_name;
}
void BackupRenamingConfig::setNewDatabaseName(const String & old_database_name, const String & new_database_name)
{
old_to_new_database_names[old_database_name] = new_database_name;
}
void BackupRenamingConfig::setNewTemporaryTableName(const String & old_temporary_table_name, const String & new_temporary_table_name)
{
old_to_new_temporary_table_names[old_temporary_table_name] = new_temporary_table_name;
}
void BackupRenamingConfig::setFromBackupQuery(const ASTBackupQuery & backup_query)
{
setFromBackupQueryElements(backup_query.elements);
}
void BackupRenamingConfig::setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements)
{
for (const auto & element : backup_query_elements)
{
switch (element.type)
{
case ElementType::TABLE: [[fallthrough]];
case ElementType::DICTIONARY:
{
const auto & new_name = element.new_name.second.empty() ? element.name : element.new_name;
setNewTableName(element.name, new_name);
break;
}
case ASTBackupQuery::DATABASE:
{
const auto & new_name = element.new_name.first.empty() ? element.name.first : element.new_name.first;
setNewDatabaseName(element.name.first, new_name);
break;
}
case ASTBackupQuery::TEMPORARY_TABLE:
{
const auto & new_name = element.new_name.second.empty() ? element.name.second : element.new_name.second;
setNewTemporaryTableName(element.name.second, new_name);
break;
}
case ASTBackupQuery::ALL_DATABASES: break;
case ASTBackupQuery::ALL_TEMPORARY_TABLES: break;
case ASTBackupQuery::EVERYTHING: break;
}
}
}
DatabaseAndTableName BackupRenamingConfig::getNewTableName(const DatabaseAndTableName & old_table_name) const
{
auto it = old_to_new_table_names.find(old_table_name);
if (it != old_to_new_table_names.end())
return it->second;
return {getNewDatabaseName(old_table_name.first), old_table_name.second};
}
const String & BackupRenamingConfig::getNewDatabaseName(const String & old_database_name) const
{
auto it = old_to_new_database_names.find(old_database_name);
if (it != old_to_new_database_names.end())
return it->second;
return old_database_name;
}
const String & BackupRenamingConfig::getNewTemporaryTableName(const String & old_temporary_table_name) const
{
auto it = old_to_new_temporary_table_names.find(old_temporary_table_name);
if (it != old_to_new_temporary_table_names.end())
return it->second;
return old_temporary_table_name;
}
}

View File

@ -0,0 +1,39 @@
#pragma once
#include <Parsers/ASTBackupQuery.h>
#include <Core/Types.h>
#include <map>
#include <unordered_map>
namespace DB
{
using DatabaseAndTableName = std::pair<String, String>;
/// Keeps information about renamings of databases or tables being processed
/// while we're making a backup or while we're restoring from a backup.
class BackupRenamingConfig
{
public:
BackupRenamingConfig() = default;
void setNewTableName(const DatabaseAndTableName & old_table_name, const DatabaseAndTableName & new_table_name);
void setNewDatabaseName(const String & old_database_name, const String & new_database_name);
void setNewTemporaryTableName(const String & old_temporary_table_name, const String & new_temporary_table_name);
void setFromBackupQuery(const ASTBackupQuery & backup_query);
void setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements);
/// Changes names according to the renaming.
DatabaseAndTableName getNewTableName(const DatabaseAndTableName & old_table_name) const;
const String & getNewDatabaseName(const String & old_database_name) const;
const String & getNewTemporaryTableName(const String & old_temporary_table_name) const;
private:
std::map<DatabaseAndTableName, DatabaseAndTableName> old_to_new_table_names;
std::unordered_map<String, String> old_to_new_database_names;
std::unordered_map<String, String> old_to_new_temporary_table_names;
};
using BackupRenamingConfigPtr = std::shared_ptr<const BackupRenamingConfig>;
}

View File

@ -0,0 +1,6 @@
#include <Backups/BackupSettings.h>
namespace DB
{
IMPLEMENT_SETTINGS_TRAITS(BackupSettingsTraits, LIST_OF_BACKUP_SETTINGS)
}

View File

@ -0,0 +1,16 @@
#pragma once
#include <Core/BaseSettings.h>
namespace DB
{
#define LIST_OF_BACKUP_SETTINGS(M) \
M(String, base_backup, "", "Name of the base backup. Only differences made after the base backup will be included in a newly created backup, so this option allows to make an incremental backup.", 0) \
DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(BackupSettingsTraits, LIST_OF_BACKUP_SETTINGS)
struct BackupSettings : public BaseSettings<BackupSettingsTraits> {};
}

830
src/Backups/BackupUtils.cpp Normal file
View File

@ -0,0 +1,830 @@
#include <Backups/BackupUtils.h>
#include <Backups/BackupEntryFromMemory.h>
#include <Backups/BackupRenamingConfig.h>
#include <Backups/IBackup.h>
#include <Backups/hasCompatibleDataToRestoreTable.h>
#include <Backups/renameInCreateQuery.h>
#include <Common/escapeForFileName.h>
#include <Databases/IDatabase.h>
#include <IO/ReadHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/formatAST.h>
#include <Storages/IStorage.h>
#include <common/insertAtEnd.h>
#include <boost/range/adaptor/reversed.hpp>
#include <filesystem>
namespace DB
{
namespace ErrorCodes
{
extern const int BACKUP_ELEMENT_DUPLICATE;
extern const int BACKUP_IS_EMPTY;
extern const int LOGICAL_ERROR;
extern const int TABLE_ALREADY_EXISTS;
extern const int CANNOT_RESTORE_TABLE;
}
namespace
{
using Kind = ASTBackupQuery::Kind;
using Element = ASTBackupQuery::Element;
using Elements = ASTBackupQuery::Elements;
using ElementType = ASTBackupQuery::ElementType;
/// Replace elements of types DICTIONARY or EVERYTHING with elements of other types.
void replaceElementTypesWithBaseElementTypes(Elements & elements)
{
for (size_t i = 0; i != elements.size(); ++i)
{
auto & element = elements[i];
switch (element.type)
{
case ElementType::DICTIONARY:
{
element.type = ElementType::TABLE;
break;
}
case ElementType::EVERYTHING:
{
element.type = ElementType::ALL_DATABASES;
auto & new_element = elements.emplace_back();
new_element.type = ElementType::ALL_TEMPORARY_TABLES;
break;
}
default:
break;
}
}
}
/// Replaces an empty database with the current database.
void replaceEmptyDatabaseWithCurrentDatabase(Elements & elements, const String & current_database)
{
for (auto & element : elements)
{
if (element.type == ElementType::TABLE)
{
if (element.name.first.empty() && !element.name.second.empty())
element.name.first = current_database;
if (element.new_name.first.empty() && !element.new_name.second.empty())
element.new_name.first = current_database;
}
}
}
/// Replaces elements of types TEMPORARY_TABLE or ALL_TEMPORARY_TABLES with elements of type TABLE or DATABASE.
void replaceTemporaryTablesWithTemporaryDatabase(Elements & elements)
{
for (size_t i = 0; i != elements.size(); ++i)
{
auto & element = elements[i];
switch (element.type)
{
case ElementType::TEMPORARY_TABLE:
{
element.type = ElementType::TABLE;
element.name.first = DatabaseCatalog::TEMPORARY_DATABASE;
if (element.new_name.first.empty() && !element.new_name.second.empty())
element.new_name.first = DatabaseCatalog::TEMPORARY_DATABASE;
break;
}
case ElementType::ALL_TEMPORARY_TABLES:
{
element.type = ElementType::DATABASE;
element.name.first = DatabaseCatalog::TEMPORARY_DATABASE;
break;
}
default:
break;
}
}
}
/// Set new names if they are not specified.
void setNewNamesIfNotSet(Elements & elements)
{
for (auto & element : elements)
{
switch (element.type)
{
case ElementType::TABLE:
{
if (element.new_name.second.empty())
element.new_name = element.name;
break;
}
case ElementType::DATABASE:
{
if (element.new_name.first.empty())
element.new_name = element.name;
break;
}
default:
break;
}
}
}
/// Removes duplications in the elements of a backup query by removing some excessive elements and by updating except_lists.
/// This function helps deduplicate elements in queries like "BACKUP ALL DATABASES, DATABASE xxx USING NAME yyy"
/// (we need a deduplication for that query because `ALL DATABASES` includes `xxx` however we don't want
/// to backup/restore the same database twice while executing the same query).
/// Also this function slightly reorders elements: it puts databases before tables and dictionaries they contain.
void deduplicateAndReorderElements(Elements & elements)
{
std::set<size_t> skip_indices; /// Indices of elements which should be removed in the end of this function.
size_t index_all_databases = static_cast<size_t>(-1); /// Index of the first element of type ALL_DATABASES or -1 if not found.
struct DatabaseInfo
{
size_t index = static_cast<size_t>(-1);
std::unordered_map<std::string_view, size_t> tables;
};
std::unordered_map<std::string_view, DatabaseInfo> databases; /// Found databases and tables.
for (size_t i = 0; i != elements.size(); ++i)
{
auto & element = elements[i];
switch (element.type)
{
case ElementType::TABLE:
{
auto & tables = databases.emplace(element.name.first, DatabaseInfo{}).first->second.tables;
auto it = tables.find(element.name.second);
if (it == tables.end())
{
tables.emplace(element.name.second, i);
}
else
{
size_t prev_index = it->second;
if ((elements[i].new_name == elements[prev_index].new_name)
&& (elements[i].partitions.empty() == elements[prev_index].partitions.empty()))
{
insertAtEnd(elements[prev_index].partitions, elements[i].partitions);
skip_indices.emplace(i);
}
else
{
throw Exception(
"Table " + backQuote(element.name.first) + "." + backQuote(element.name.second) + " was specified twice",
ErrorCodes::BACKUP_ELEMENT_DUPLICATE);
}
}
break;
}
case ElementType::DATABASE:
{
auto it = databases.find(element.name.first);
if (it == databases.end())
{
DatabaseInfo new_db_info;
new_db_info.index = i;
databases.emplace(element.name.first, new_db_info);
}
else if (it->second.index == static_cast<size_t>(-1))
{
it->second.index = i;
}
else
{
size_t prev_index = it->second.index;
if ((elements[i].new_name == elements[prev_index].new_name)
&& (elements[i].except_list == elements[prev_index].except_list))
{
skip_indices.emplace(i);
}
else
{
throw Exception("Database " + backQuote(element.name.first) + " was specified twice", ErrorCodes::BACKUP_ELEMENT_DUPLICATE);
}
}
break;
}
case ElementType::ALL_DATABASES:
{
if (index_all_databases == static_cast<size_t>(-1))
{
index_all_databases = i;
}
else
{
size_t prev_index = index_all_databases;
if (elements[i].except_list == elements[prev_index].except_list)
skip_indices.emplace(i);
else
throw Exception("The tag ALL DATABASES was specified twice", ErrorCodes::BACKUP_ELEMENT_DUPLICATE);
}
break;
}
default:
/// replaceElementTypesWithBaseElementTypes() and replaceTemporaryTablesWithTemporaryDatabase() should have removed all other element types.
throw Exception("Unexpected element type: " + std::to_string(static_cast<int>(element.type)), ErrorCodes::LOGICAL_ERROR);
}
}
if (index_all_databases != static_cast<size_t>(-1))
{
for (auto & [database_name, database] : databases)
{
elements[index_all_databases].except_list.emplace(database_name);
if (database.index == static_cast<size_t>(-1))
{
auto & new_element = elements.emplace_back();
new_element.type = ElementType::DATABASE;
new_element.name.first = database_name;
new_element.new_name = new_element.name;
database.index = elements.size() - 1;
}
}
}
for (auto & [database_name, database] : databases)
{
if (database.index == static_cast<size_t>(-1))
continue;
for (const auto & [table_name, table_index] : database.tables)
elements[database.index].except_list.emplace(table_name);
}
/// Reorder the elements: databases should be before tables and dictionaries they contain.
for (auto & [database_name, database] : databases)
{
if (database.index == static_cast<size_t>(-1))
continue;
size_t min_index = std::numeric_limits<size_t>::max();
auto min_index_it = database.tables.end();
for (auto it = database.tables.begin(); it != database.tables.end(); ++it)
{
if (min_index > it->second)
{
min_index = it->second;
min_index_it = it;
}
}
if (database.index > min_index)
{
std::swap(elements[database.index], elements[min_index]);
std::swap(database.index, min_index_it->second);
}
}
for (auto skip_index : skip_indices | boost::adaptors::reversed)
elements.erase(elements.begin() + skip_index);
}
Elements adjustElements(const Elements & elements, const String & current_database)
{
auto res = elements;
replaceElementTypesWithBaseElementTypes(res);
replaceEmptyDatabaseWithCurrentDatabase(res, current_database);
replaceTemporaryTablesWithTemporaryDatabase(res);
setNewNamesIfNotSet(res);
deduplicateAndReorderElements(res);
return res;
}
String getDataPathInBackup(const DatabaseAndTableName & table_name)
{
if (table_name.first.empty() || table_name.second.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty");
assert(!table_name.first.empty() && !table_name.second.empty());
return String{"data/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + "/";
}
String getDataPathInBackup(const IAST & create_query)
{
const auto & create = create_query.as<const ASTCreateQuery &>();
if (create.table.empty())
return {};
if (create.temporary)
return getDataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.table});
return getDataPathInBackup({create.database, create.table});
}
String getMetadataPathInBackup(const DatabaseAndTableName & table_name)
{
if (table_name.first.empty() || table_name.second.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty");
return String{"metadata/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + ".sql";
}
String getMetadataPathInBackup(const String & database_name)
{
if (database_name.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name must not be empty");
return String{"metadata/"} + escapeForFileName(database_name) + ".sql";
}
String getMetadataPathInBackup(const IAST & create_query)
{
const auto & create = create_query.as<const ASTCreateQuery &>();
if (create.table.empty())
return getMetadataPathInBackup(create.database);
if (create.temporary)
return getMetadataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.table});
return getMetadataPathInBackup({create.database, create.table});
}
void backupCreateQuery(const IAST & create_query, BackupEntries & backup_entries)
{
auto metadata_entry = std::make_unique<BackupEntryFromMemory>(serializeAST(create_query));
String metadata_path = getMetadataPathInBackup(create_query);
backup_entries.emplace_back(metadata_path, std::move(metadata_entry));
}
void backupTable(
const DatabaseAndTable & database_and_table,
const String & table_name,
const ASTs & partitions,
const ContextPtr & context,
const BackupRenamingConfigPtr & renaming_config,
BackupEntries & backup_entries)
{
const auto & database = database_and_table.first;
const auto & storage = database_and_table.second;
context->checkAccess(AccessType::SELECT, database->getDatabaseName(), table_name);
auto create_query = database->getCreateTableQuery(table_name, context);
ASTPtr new_create_query = renameInCreateQuery(create_query, renaming_config, context);
backupCreateQuery(*new_create_query, backup_entries);
auto data_backup = storage->backup(partitions, context);
if (!data_backup.empty())
{
String data_path = getDataPathInBackup(*new_create_query);
for (auto & [path_in_backup, backup_entry] : data_backup)
backup_entries.emplace_back(data_path + path_in_backup, std::move(backup_entry));
}
}
void backupDatabase(
const DatabasePtr & database,
const std::set<String> & except_list,
const ContextPtr & context,
const BackupRenamingConfigPtr & renaming_config,
BackupEntries & backup_entries)
{
context->checkAccess(AccessType::SHOW_TABLES, database->getDatabaseName());
auto create_query = database->getCreateDatabaseQuery();
ASTPtr new_create_query = renameInCreateQuery(create_query, renaming_config, context);
backupCreateQuery(*new_create_query, backup_entries);
for (auto it = database->getTablesIteratorForBackup(context); it->isValid(); it->next())
{
if (except_list.contains(it->name()))
continue;
backupTable({database, it->table()}, it->name(), {}, context, renaming_config, backup_entries);
}
}
void backupAllDatabases(
const std::set<String> & except_list,
const ContextPtr & context,
const BackupRenamingConfigPtr & renaming_config,
BackupEntries & backup_entries)
{
for (const auto & [database_name, database] : DatabaseCatalog::instance().getDatabases())
{
if (except_list.contains(database_name))
continue;
if (database_name == DatabaseCatalog::SYSTEM_DATABASE || database_name == DatabaseCatalog::TEMPORARY_DATABASE)
continue;
backupDatabase(database, {}, context, renaming_config, backup_entries);
}
}
void makeDatabaseIfNotExists(const String & database_name, ContextMutablePtr context)
{
if (DatabaseCatalog::instance().isDatabaseExist(database_name))
return;
/// We create and execute `create` query for the database name.
auto create_query = std::make_shared<ASTCreateQuery>();
create_query->database = database_name;
create_query->if_not_exists = true;
InterpreterCreateQuery create_interpreter{create_query, context};
create_interpreter.execute();
}
ASTPtr readCreateQueryFromBackup(const DatabaseAndTableName & table_name, const BackupPtr & backup)
{
String create_query_path = getMetadataPathInBackup(table_name);
auto read_buffer = backup->read(create_query_path)->getReadBuffer();
String create_query_str;
readStringUntilEOF(create_query_str, *read_buffer);
read_buffer.reset();
ParserCreateQuery create_parser;
return parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
}
ASTPtr readCreateQueryFromBackup(const String & database_name, const BackupPtr & backup)
{
String create_query_path = getMetadataPathInBackup(database_name);
auto read_buffer = backup->read(create_query_path)->getReadBuffer();
String create_query_str;
readStringUntilEOF(create_query_str, *read_buffer);
read_buffer.reset();
ParserCreateQuery create_parser;
return parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
}
void restoreTable(
const DatabaseAndTableName & table_name,
const ASTs & partitions,
ContextMutablePtr context,
const BackupPtr & backup,
const BackupRenamingConfigPtr & renaming_config,
RestoreObjectsTasks & restore_tasks)
{
ASTPtr create_query = readCreateQueryFromBackup(table_name, backup);
auto new_create_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(renameInCreateQuery(create_query, renaming_config, context));
restore_tasks.emplace_back([table_name, new_create_query, partitions, context, backup]() -> RestoreDataTasks
{
DatabaseAndTableName new_table_name{new_create_query->database, new_create_query->table};
if (new_create_query->temporary)
new_table_name.first = DatabaseCatalog::TEMPORARY_DATABASE;
context->checkAccess(AccessType::INSERT, new_table_name.first, new_table_name.second);
StoragePtr storage;
for (size_t try_index = 0; try_index != 10; ++try_index)
{
if (DatabaseCatalog::instance().isTableExist({new_table_name.first, new_table_name.second}, context))
{
DatabasePtr existing_database;
StoragePtr existing_storage;
std::tie(existing_database, existing_storage) = DatabaseCatalog::instance().tryGetDatabaseAndTable({new_table_name.first, new_table_name.second}, context);
if (existing_storage)
{
if (auto existing_table_create_query = existing_database->tryGetCreateTableQuery(new_table_name.second, context))
{
if (hasCompatibleDataToRestoreTable(*new_create_query, existing_table_create_query->as<ASTCreateQuery &>()))
{
storage = existing_storage;
break;
}
else
{
String error_message = (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE)
? ("Temporary table " + backQuoteIfNeed(new_table_name.second) + " already exists")
: ("Table " + backQuoteIfNeed(new_table_name.first) + "." + backQuoteIfNeed(new_table_name.second)
+ " already exists");
throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE);
}
}
}
}
makeDatabaseIfNotExists(new_table_name.first, context);
try
{
InterpreterCreateQuery create_interpreter{new_create_query, context};
create_interpreter.execute();
}
catch (Exception & e)
{
if (e.code() != ErrorCodes::TABLE_ALREADY_EXISTS)
throw;
}
}
if (!storage)
{
String error_message = (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE)
? ("Could not create temporary table " + backQuoteIfNeed(new_table_name.second) + " for restoring")
: ("Could not create table " + backQuoteIfNeed(new_table_name.first) + "." + backQuoteIfNeed(new_table_name.second)
+ " for restoring");
throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE);
}
String data_path_in_backup = getDataPathInBackup(table_name);
RestoreDataTasks restore_data_tasks = storage->restoreFromBackup(backup, data_path_in_backup, partitions, context);
/// Keep `storage` alive while we're executing `restore_data_tasks`.
for (auto & restore_data_task : restore_data_tasks)
restore_data_task = [restore_data_task, storage]() { restore_data_task(); };
return restore_data_tasks;
});
}
void restoreDatabase(const String & database_name, const std::set<String> & except_list, ContextMutablePtr context, const BackupPtr & backup, const BackupRenamingConfigPtr & renaming_config, RestoreObjectsTasks & restore_tasks)
{
ASTPtr create_query = readCreateQueryFromBackup(database_name, backup);
auto new_create_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(renameInCreateQuery(create_query, renaming_config, context));
restore_tasks.emplace_back([database_name, new_create_query, except_list, context, backup, renaming_config]() -> RestoreDataTasks
{
const String & new_database_name = new_create_query->database;
context->checkAccess(AccessType::SHOW_TABLES, new_database_name);
if (!DatabaseCatalog::instance().isDatabaseExist(new_database_name))
{
/// We create and execute `create` query for the database name.
new_create_query->if_not_exists = true;
InterpreterCreateQuery create_interpreter{new_create_query, context};
create_interpreter.execute();
}
RestoreObjectsTasks restore_objects_tasks;
Strings table_names = backup->list("metadata/" + escapeForFileName(database_name) + "/", "/");
for (const String & table_name : table_names)
{
if (except_list.contains(table_name))
continue;
restoreTable({database_name, table_name}, {}, context, backup, renaming_config, restore_objects_tasks);
}
RestoreDataTasks restore_data_tasks;
for (auto & restore_object_task : restore_objects_tasks)
insertAtEnd(restore_data_tasks, std::move(restore_object_task)());
return restore_data_tasks;
});
}
void restoreAllDatabases(const std::set<String> & except_list, ContextMutablePtr context, const BackupPtr & backup, const BackupRenamingConfigPtr & renaming_config, RestoreObjectsTasks & restore_tasks)
{
restore_tasks.emplace_back([except_list, context, backup, renaming_config]() -> RestoreDataTasks
{
Strings database_names = backup->list("metadata/", "/");
RestoreObjectsTasks restore_objects_tasks;
for (const String & database_name : database_names)
{
if (except_list.contains(database_name))
continue;
restoreDatabase(database_name, {}, context, backup, renaming_config, restore_objects_tasks);
}
RestoreDataTasks restore_data_tasks;
for (auto & restore_object_task : restore_objects_tasks)
insertAtEnd(restore_data_tasks, std::move(restore_object_task)());
return restore_data_tasks;
});
}
}
BackupEntries makeBackupEntries(const Elements & elements, const ContextPtr & context)
{
BackupEntries backup_entries;
auto elements2 = adjustElements(elements, context->getCurrentDatabase());
auto renaming_config = std::make_shared<BackupRenamingConfig>();
renaming_config->setFromBackupQueryElements(elements2);
for (const auto & element : elements2)
{
switch (element.type)
{
case ElementType::TABLE:
{
const String & database_name = element.name.first;
const String & table_name = element.name.second;
auto [database, storage] = DatabaseCatalog::instance().getDatabaseAndTable({database_name, table_name}, context);
backupTable({database, storage}, table_name, element.partitions, context, renaming_config, backup_entries);
break;
}
case ElementType::DATABASE:
{
const String & database_name = element.name.first;
auto database = DatabaseCatalog::instance().getDatabase(database_name, context);
backupDatabase(database, element.except_list, context, renaming_config, backup_entries);
break;
}
case ElementType::ALL_DATABASES:
{
backupAllDatabases(element.except_list, context, renaming_config, backup_entries);
break;
}
default:
throw Exception("Unexpected element type", ErrorCodes::LOGICAL_ERROR); /// other element types have been removed in deduplicateElements()
}
}
/// A backup cannot be empty.
if (backup_entries.empty())
throw Exception("Backup must not be empty", ErrorCodes::BACKUP_IS_EMPTY);
/// Check that all backup entries are unique.
std::sort(
backup_entries.begin(),
backup_entries.end(),
[](const std::pair<String, std::unique_ptr<IBackupEntry>> & lhs, const std::pair<String, std::unique_ptr<IBackupEntry>> & rhs)
{
return lhs.first < rhs.first;
});
auto adjacent = std::adjacent_find(backup_entries.begin(), backup_entries.end());
if (adjacent != backup_entries.end())
throw Exception("Cannot write multiple entries with the same name " + quoteString(adjacent->first), ErrorCodes::BACKUP_ELEMENT_DUPLICATE);
return backup_entries;
}
UInt64 estimateBackupSize(const BackupEntries & backup_entries, const BackupPtr & base_backup)
{
UInt64 total_size = 0;
for (const auto & [name, entry] : backup_entries)
{
UInt64 data_size = entry->getSize();
if (base_backup)
{
if (base_backup->exists(name) && (data_size == base_backup->getSize(name)))
{
auto checksum = entry->getChecksum();
if (checksum && (*checksum == base_backup->getChecksum(name)))
continue;
}
}
total_size += data_size;
}
return total_size;
}
void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, size_t num_threads)
{
if (!num_threads)
num_threads = 1;
std::vector<ThreadFromGlobalPool> threads;
size_t num_active_threads = 0;
std::mutex mutex;
std::condition_variable cond;
std::exception_ptr exception;
for (auto & name_and_entry : backup_entries)
{
auto & name = name_and_entry.first;
auto & entry = name_and_entry.second;
{
std::unique_lock lock{mutex};
if (exception)
break;
cond.wait(lock, [&] { return num_active_threads < num_threads; });
if (exception)
break;
++num_active_threads;
}
threads.emplace_back([backup, &name, &entry, &mutex, &cond, &num_active_threads, &exception]()
{
try
{
backup->write(name, std::move(entry));
}
catch (...)
{
std::lock_guard lock{mutex};
if (!exception)
exception = std::current_exception();
}
{
std::lock_guard lock{mutex};
--num_active_threads;
cond.notify_all();
}
});
}
for (auto & thread : threads)
thread.join();
backup_entries.clear();
if (exception)
{
/// We don't call finalizeWriting() if an error occurs.
/// And IBackup's implementation should remove the backup in its destructor if finalizeWriting() hasn't called before.
std::rethrow_exception(exception);
}
backup->finalizeWriting();
}
RestoreObjectsTasks makeRestoreTasks(const Elements & elements, ContextMutablePtr context, const BackupPtr & backup)
{
RestoreObjectsTasks restore_tasks;
auto elements2 = adjustElements(elements, context->getCurrentDatabase());
auto renaming_config = std::make_shared<BackupRenamingConfig>();
renaming_config->setFromBackupQueryElements(elements2);
for (const auto & element : elements2)
{
switch (element.type)
{
case ElementType::TABLE:
{
const String & database_name = element.name.first;
const String & table_name = element.name.second;
restoreTable({database_name, table_name}, element.partitions, context, backup, renaming_config, restore_tasks);
break;
}
case ElementType::DATABASE:
{
const String & database_name = element.name.first;
auto database = DatabaseCatalog::instance().getDatabase(database_name, context);
restoreDatabase(database_name, element.except_list, context, backup, renaming_config, restore_tasks);
break;
}
case ElementType::ALL_DATABASES:
{
restoreAllDatabases(element.except_list, context, backup, renaming_config, restore_tasks);
break;
}
default:
throw Exception("Unexpected element type", ErrorCodes::LOGICAL_ERROR); /// other element types have been removed in deduplicateElements()
}
}
return restore_tasks;
}
void executeRestoreTasks(RestoreObjectsTasks && restore_tasks, size_t num_threads)
{
if (!num_threads)
num_threads = 1;
RestoreDataTasks restore_data_tasks;
for (auto & restore_object_task : restore_tasks)
insertAtEnd(restore_data_tasks, std::move(restore_object_task)());
restore_tasks.clear();
std::vector<ThreadFromGlobalPool> threads;
size_t num_active_threads = 0;
std::mutex mutex;
std::condition_variable cond;
std::exception_ptr exception;
for (auto & restore_data_task : restore_data_tasks)
{
{
std::unique_lock lock{mutex};
if (exception)
break;
cond.wait(lock, [&] { return num_active_threads < num_threads; });
if (exception)
break;
++num_active_threads;
}
threads.emplace_back([&restore_data_task, &mutex, &cond, &num_active_threads, &exception]() mutable
{
try
{
restore_data_task();
restore_data_task = {};
}
catch (...)
{
std::lock_guard lock{mutex};
if (!exception)
exception = std::current_exception();
}
{
std::lock_guard lock{mutex};
--num_active_threads;
cond.notify_all();
}
});
}
for (auto & thread : threads)
thread.join();
restore_data_tasks.clear();
if (exception)
std::rethrow_exception(exception);
}
}

39
src/Backups/BackupUtils.h Normal file
View File

@ -0,0 +1,39 @@
#pragma once
#include <Parsers/ASTBackupQuery.h>
namespace DB
{
class IBackup;
using BackupPtr = std::shared_ptr<const IBackup>;
using BackupMutablePtr = std::shared_ptr<IBackup>;
class IBackupEntry;
using BackupEntryPtr = std::unique_ptr<IBackupEntry>;
using BackupEntries = std::vector<std::pair<String, BackupEntryPtr>>;
using RestoreDataTask = std::function<void()>;
using RestoreDataTasks = std::vector<RestoreDataTask>;
using RestoreObjectTask = std::function<RestoreDataTasks()>;
using RestoreObjectsTasks = std::vector<RestoreObjectTask>;
class Context;
using ContextPtr = std::shared_ptr<const Context>;
using ContextMutablePtr = std::shared_ptr<Context>;
/// Prepares backup entries.
BackupEntries makeBackupEntries(const ASTBackupQuery::Elements & elements, const ContextPtr & context);
/// Estimate total size of the backup which would be written from the specified entries.
UInt64 estimateBackupSize(const BackupEntries & backup_entries, const BackupPtr & base_backup);
/// Write backup entries to an opened backup.
void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, size_t num_threads);
/// Prepare restore tasks.
RestoreObjectsTasks makeRestoreTasks(const ASTBackupQuery::Elements & elements, ContextMutablePtr context, const BackupPtr & backup);
/// Execute restore tasks.
void executeRestoreTasks(RestoreObjectsTasks && restore_tasks, size_t num_threads);
}

View File

65
src/Backups/IBackup.h Normal file
View File

@ -0,0 +1,65 @@
#pragma once
#include <Core/Types.h>
#include <memory>
namespace DB
{
class IBackupEntry;
using BackupEntryPtr = std::unique_ptr<IBackupEntry>;
/// Represents a backup, i.e. a storage of BackupEntries which can be accessed by their names.
/// A backup can be either incremental or non-incremental. An incremental backup doesn't store
/// the data of the entries which are not changed compared to its base backup.
class IBackup
{
public:
virtual ~IBackup() = default;
enum class OpenMode
{
READ,
WRITE,
};
/// A backup can be open either in READ or WRITE mode.
virtual OpenMode getOpenMode() const = 0;
/// Returns the path to the backup.
virtual String getPath() const = 0;
/// Returns names of entries stored in the backup.
/// If `prefix` isn't empty the function will return only the names starting with
/// the prefix (but without the prefix itself).
/// If the `terminator` isn't empty the function will returns only parts of the names
/// before the terminator. For example, list("", "") returns names of all the entries
/// in the backup; and list("data/", "/") return kind of a list of folders and
/// files stored in the "data/" directory inside the backup.
virtual Strings list(const String & prefix = "", const String & terminator = "/") const = 0;
/// Checks if an entry with a specified name exists.
virtual bool exists(const String & name) const = 0;
/// Returns the size of the entry's data.
/// This function does the same as `read(name)->getSize()` but faster.
virtual size_t getSize(const String & name) const = 0;
/// Returns the checksum of the entry's data.
/// This function does the same as `read(name)->getCheckum()` but faster.
virtual UInt128 getChecksum(const String & name) const = 0;
/// Reads an entry from the backup.
virtual BackupEntryPtr read(const String & name) const = 0;
/// Puts a new entry to the backup.
virtual void write(const String & name, BackupEntryPtr entry) = 0;
/// Finalizes writing the backup, should be called after all entries have been successfully written.
virtual void finalizeWriting() = 0;
};
using BackupPtr = std::shared_ptr<const IBackup>;
using BackupMutablePtr = std::shared_ptr<IBackup>;
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <Core/Types.h>
#include <memory>
#include <optional>
#include <vector>
namespace DB
{
class ReadBuffer;
/// A backup entry represents some data which should be written to the backup or has been read from the backup.
class IBackupEntry
{
public:
virtual ~IBackupEntry() = default;
/// Returns the size of the data.
virtual UInt64 getSize() const = 0;
/// Returns the checksum of the data if it's precalculated.
/// Can return nullopt which means the checksum should be calculated from the read buffer.
virtual std::optional<UInt128> getChecksum() const { return {}; }
/// Returns a read buffer for reading the data.
virtual std::unique_ptr<ReadBuffer> getReadBuffer() const = 0;
};
using BackupEntryPtr = std::unique_ptr<IBackupEntry>;
using BackupEntries = std::vector<std::pair<String, BackupEntryPtr>>;
}

View File

@ -0,0 +1,22 @@
#include <Backups/hasCompatibleDataToRestoreTable.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/formatAST.h>
namespace DB
{
bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCreateQuery & query2)
{
/// TODO: Write more subtle condition here.
auto q1 = typeid_cast<std::shared_ptr<ASTCreateQuery>>(query1.clone());
auto q2 = typeid_cast<std::shared_ptr<ASTCreateQuery>>(query2.clone());
/// Remove UUIDs.
q1->uuid = UUIDHelpers::Nil;
q2->uuid = UUIDHelpers::Nil;
return serializeAST(*q1) == serializeAST(*q2);
}
}

View File

@ -0,0 +1,11 @@
#pragma once
namespace DB
{
class ASTCreateQuery;
/// Whether the data of the first table can be inserted to the second table.
bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCreateQuery & query2);
}

View File

@ -0,0 +1,276 @@
#include <Backups/renameInCreateQuery.h>
#include <Backups/BackupRenamingConfig.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/evaluateConstantExpression.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
class RenameInCreateQueryTransformMatcher
{
public:
struct Data
{
BackupRenamingConfigPtr renaming_config;
ContextPtr context;
};
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; }
static void visit(ASTPtr & ast, const Data & data)
{
if (auto * create = ast->as<ASTCreateQuery>())
visitCreateQuery(*create, data);
else if (auto * expr = ast->as<ASTTableExpression>())
visitTableExpression(*expr, data);
else if (auto * function = ast->as<ASTFunction>())
visitFunction(*function, data);
else if (auto * dictionary = ast->as<ASTDictionary>())
visitDictionary(*dictionary, data);
}
private:
/// Replaces names of tables and databases used in a CREATE query, which can be either CREATE TABLE or
/// CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query.
static void visitCreateQuery(ASTCreateQuery & create, const Data & data)
{
if (create.temporary)
{
if (create.table.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table name specified in the CREATE TEMPORARY TABLE query must not be empty");
create.table = data.renaming_config->getNewTemporaryTableName(create.table);
}
else if (create.table.empty())
{
if (create.database.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE DATABASE query must not be empty");
create.database = data.renaming_config->getNewDatabaseName(create.database);
}
else
{
if (create.database.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE TABLE query must not be empty");
std::tie(create.database, create.table) = data.renaming_config->getNewTableName({create.database, create.table});
}
create.uuid = UUIDHelpers::Nil;
if (!create.as_table.empty() && !create.as_database.empty())
std::tie(create.as_database, create.as_table) = data.renaming_config->getNewTableName({create.as_database, create.as_table});
if (!create.to_table_id.table_name.empty() && !create.to_table_id.database_name.empty())
{
auto to_table = data.renaming_config->getNewTableName({create.to_table_id.database_name, create.to_table_id.table_name});
create.to_table_id = StorageID{to_table.first, to_table.second};
}
}
/// Replaces names of a database and a table in a expression like `db`.`table`
static void visitTableExpression(ASTTableExpression & expr, const Data & data)
{
if (!expr.database_and_table_name)
return;
ASTIdentifier * id = expr.database_and_table_name->as<ASTIdentifier>();
if (!id)
return;
auto table_id = id->createTable();
if (!table_id)
return;
const String & db_name = table_id->getDatabaseName();
const String & table_name = table_id->shortName();
if (db_name.empty() || table_name.empty())
return;
String new_db_name, new_table_name;
std::tie(new_db_name, new_table_name) = data.renaming_config->getNewTableName({db_name, table_name});
if ((new_db_name == db_name) && (new_table_name == table_name))
return;
expr.database_and_table_name = std::make_shared<ASTIdentifier>(Strings{new_db_name, new_table_name});
expr.children.push_back(expr.database_and_table_name);
}
/// Replaces names of tables and databases used in arguments of a table function or a table engine.
static void visitFunction(ASTFunction & function, const Data & data)
{
if ((function.name == "merge") || (function.name == "Merge"))
{
visitFunctionMerge(function, data);
}
else if ((function.name == "remote") || (function.name == "remoteSecure") || (function.name == "cluster") ||
(function.name == "clusterAllReplicas") || (function.name == "Distributed"))
{
visitFunctionRemote(function, data);
}
}
/// Replaces a database's name passed via an argument of the function merge() or the table engine Merge.
static void visitFunctionMerge(ASTFunction & function, const Data & data)
{
if (!function.arguments)
return;
/// The first argument is a database's name and we can rename it.
/// The second argument is a regular expression and we can do nothing about it.
auto & args = function.arguments->as<ASTExpressionList &>().children;
size_t db_name_arg_index = 0;
if (args.size() <= db_name_arg_index)
return;
String db_name = evaluateConstantExpressionForDatabaseName(args[db_name_arg_index], data.context)->as<ASTLiteral &>().value.safeGet<String>();
if (db_name.empty())
return;
String new_db_name = data.renaming_config->getNewDatabaseName(db_name);
if (new_db_name == db_name)
return;
args[db_name_arg_index] = std::make_shared<ASTLiteral>(new_db_name);
}
/// Replaces names of a table and a database passed via arguments of the function remote() or cluster() or the table engine Distributed.
static void visitFunctionRemote(ASTFunction & function, const Data & data)
{
if (!function.arguments)
return;
/// The first argument is an address or cluster's name, so we skip it.
/// The second argument can be either 'db.name' or just 'db' followed by the third argument 'table'.
auto & args = function.arguments->as<ASTExpressionList &>().children;
const auto * second_arg_as_function = args[1]->as<ASTFunction>();
if (second_arg_as_function && TableFunctionFactory::instance().isTableFunctionName(second_arg_as_function->name))
return;
size_t db_name_index = 1;
if (args.size() <= db_name_index)
return;
String db_name = evaluateConstantExpressionForDatabaseName(args[db_name_index], data.context)->as<ASTLiteral &>().value.safeGet<String>();
String table_name;
size_t table_name_index = static_cast<size_t>(-1);
size_t dot = String::npos;
if (function.name != "Distributed")
dot = db_name.find('.');
if (dot != String::npos)
{
table_name = db_name.substr(dot + 1);
db_name.resize(dot);
}
else
{
table_name_index = 2;
if (args.size() <= table_name_index)
return;
table_name = evaluateConstantExpressionForDatabaseName(args[table_name_index], data.context)->as<ASTLiteral &>().value.safeGet<String>();
}
if (db_name.empty() || table_name.empty())
return;
String new_db_name, new_table_name;
std::tie(new_db_name, new_table_name) = data.renaming_config->getNewTableName({db_name, table_name});
if ((new_db_name == db_name) && (new_table_name == table_name))
return;
if (table_name_index != static_cast<size_t>(-1))
{
if (new_db_name != db_name)
args[db_name_index] = std::make_shared<ASTLiteral>(new_db_name);
if (new_table_name != table_name)
args[table_name_index] = std::make_shared<ASTLiteral>(new_table_name);
}
else
{
args[db_name_index] = std::make_shared<ASTLiteral>(new_db_name);
args.insert(args.begin() + db_name_index + 1, std::make_shared<ASTLiteral>(new_table_name));
}
}
/// Replaces names of a table and a database used in source parameters of a dictionary.
static void visitDictionary(ASTDictionary & dictionary, const Data & data)
{
if (!dictionary.source || dictionary.source->name != "clickhouse" || !dictionary.source->elements)
return;
auto & elements = dictionary.source->elements->as<ASTExpressionList &>().children;
String db_name, table_name;
size_t db_name_index = static_cast<size_t>(-1);
size_t table_name_index = static_cast<size_t>(-1);
for (size_t i = 0; i != elements.size(); ++i)
{
auto & pair = elements[i]->as<ASTPair &>();
if (pair.first == "db")
{
if (db_name_index != static_cast<size_t>(-1))
return;
db_name = pair.second->as<ASTLiteral &>().value.safeGet<String>();
db_name_index = i;
}
else if (pair.first == "table")
{
if (table_name_index != static_cast<size_t>(-1))
return;
table_name = pair.second->as<ASTLiteral &>().value.safeGet<String>();
table_name_index = i;
}
}
if (db_name.empty() || table_name.empty())
return;
String new_db_name, new_table_name;
std::tie(new_db_name, new_table_name) = data.renaming_config->getNewTableName({db_name, table_name});
if ((new_db_name == db_name) && (new_table_name == table_name))
return;
if (new_db_name != db_name)
{
auto & pair = elements[db_name_index]->as<ASTPair &>();
pair.replace(pair.second, std::make_shared<ASTLiteral>(new_db_name));
}
if (new_table_name != table_name)
{
auto & pair = elements[table_name_index]->as<ASTPair &>();
pair.replace(pair.second, std::make_shared<ASTLiteral>(new_table_name));
}
}
};
using RenameInCreateQueryTransformVisitor = InDepthNodeVisitor<RenameInCreateQueryTransformMatcher, false>;
}
ASTPtr renameInCreateQuery(const ASTPtr & ast, const BackupRenamingConfigPtr & renaming_config, const ContextPtr & context)
{
auto new_ast = ast->clone();
try
{
RenameInCreateQueryTransformVisitor::Data data{renaming_config, context};
RenameInCreateQueryTransformVisitor{data}.visit(new_ast);
return new_ast;
}
catch (...)
{
tryLogCurrentException("Backup", "Error while renaming in AST");
return ast;
}
}
}

View File

@ -0,0 +1,16 @@
#pragma once
#include <memory>
namespace DB
{
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
class Context;
using ContextPtr = std::shared_ptr<const Context>;
class BackupRenamingConfig;
using BackupRenamingConfigPtr = std::shared_ptr<const BackupRenamingConfig>;
/// Changes names in AST according to the renaming settings.
ASTPtr renameInCreateQuery(const ASTPtr & ast, const BackupRenamingConfigPtr & renaming_config, const ContextPtr & context);
}

View File

@ -45,6 +45,7 @@ if (COMPILER_GCC)
endif ()
add_subdirectory (Access)
add_subdirectory (Backups)
add_subdirectory (Columns)
add_subdirectory (Common)
add_subdirectory (Core)
@ -180,6 +181,7 @@ macro(add_object_library name common_path)
endmacro()
add_object_library(clickhouse_access Access)
add_object_library(clickhouse_backups Backups)
add_object_library(clickhouse_core Core)
add_object_library(clickhouse_core_mysql Core/MySQL)
add_object_library(clickhouse_compression Compression)

View File

@ -304,7 +304,7 @@ size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
if (x.getType() == Field::Types::Null)
return getNullValueIndex();
if (isNumeric())
if (valuesHaveFixedSize())
return uniqueInsertData(&x.reinterpret<char>(), size_of_value_if_fixed);
auto & val = x.get<String>();

View File

@ -566,6 +566,17 @@
M(595, BZIP2_STREAM_ENCODER_FAILED) \
M(596, INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH) \
M(597, NO_SUCH_ERROR_CODE) \
M(598, BACKUP_ALREADY_EXISTS) \
M(599, BACKUP_NOT_FOUND) \
M(600, BACKUP_VERSION_NOT_SUPPORTED) \
M(601, BACKUP_DAMAGED) \
M(602, NO_BASE_BACKUP) \
M(603, WRONG_BASE_BACKUP) \
M(604, BACKUP_ENTRY_ALREADY_EXISTS) \
M(605, BACKUP_ENTRY_NOT_FOUND) \
M(606, BACKUP_IS_EMPTY) \
M(607, BACKUP_ELEMENT_DUPLICATE) \
M(608, CANNOT_RESTORE_TABLE) \
\
M(998, POSTGRESQL_CONNECTION_FAILURE) \
M(999, KEEPER_EXCEPTION) \

View File

@ -276,14 +276,30 @@ inline void trimLeft(std::string_view & str, char c = ' ')
str.remove_prefix(1);
}
inline void trimLeft(std::string & str, char c = ' ')
{
str.erase(0, str.find_first_not_of(c));
}
inline void trimRight(std::string_view & str, char c = ' ')
{
while (str.ends_with(c))
str.remove_suffix(1);
}
inline void trimRight(std::string & str, char c = ' ')
{
str.erase(str.find_last_not_of(c) + 1);
}
inline void trim(std::string_view & str, char c = ' ')
{
trimLeft(str, c);
trimRight(str, c);
}
inline void trim(std::string & str, char c = ' ')
{
trimRight(str, c);
trimLeft(str, c);
}

View File

@ -80,8 +80,3 @@ target_link_libraries (average PRIVATE clickhouse_common_io)
add_executable (shell_command_inout shell_command_inout.cpp)
target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io)
if (ENABLE_FUZZING)
add_executable(YAML_fuzzer YAML_fuzzer.cpp ${SRCS})
target_link_libraries(YAML_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE})
endif ()

View File

@ -1,39 +0,0 @@
#include <iostream>
#include <fstream>
#include <string>
#include <cstdio>
#include <time.h>
#include <filesystem>
extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size)
{
/// How to test:
/// build ClickHouse with YAML_fuzzer.cpp
/// ./YAML_fuzzer YAML_CORPUS
/// where YAML_CORPUS is a directory with different YAML configs for libfuzzer
char file_name[L_tmpnam];
if (!std::tmpnam(file_name))
{
std::cerr << "Cannot create temp file!\n";
return 1;
}
std::string input = std::string(reinterpret_cast<const char*>(data), size);
DB::YAMLParser parser;
{
std::ofstream temp_file(file_name);
temp_file << input;
}
try
{
DB::YAMLParser::parse(std::string(file_name));
}
catch (...)
{
std::cerr << "YAML_fuzzer failed: " << getCurrentExceptionMessage() << std::endl;
return 1;
}
return 0;
}

View File

@ -1,3 +1,18 @@
if(ENABLE_EXAMPLES)
if (ENABLE_FUZZING)
include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
add_headers_and_sources(fuzz_compression .)
# Remove this file, because it has dependencies on DataTypes
list(REMOVE_ITEM ${fuzz_compression_sources} CompressionFactoryAdditions.cpp)
add_library(fuzz_compression ${fuzz_compression_headers} ${fuzz_compression_sources})
target_link_libraries(fuzz_compression PUBLIC clickhouse_parsers clickhouse_common_io common lz4)
endif()
if (ENABLE_EXAMPLES)
add_subdirectory(examples)
endif()
if (ENABLE_FUZZING)
add_subdirectory(fuzzers)
endif()

View File

@ -22,13 +22,10 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_CODEC;
extern const int BAD_ARGUMENTS;
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS;
}
static constexpr auto DEFAULT_CODEC_NAME = "Default";
CompressionCodecPtr CompressionCodecFactory::getDefaultCodec() const
{
return default_codec;
@ -49,184 +46,6 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std
}
}
void CompressionCodecFactory::validateCodec(
const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs) const
{
if (family_name.empty())
throw Exception("Compression codec name cannot be empty", ErrorCodes::BAD_ARGUMENTS);
if (level)
{
auto literal = std::make_shared<ASTLiteral>(static_cast<UInt64>(*level));
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)),
{}, sanity_check, allow_experimental_codecs);
}
else
{
auto identifier = std::make_shared<ASTIdentifier>(Poco::toUpper(family_name));
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier),
{}, sanity_check, allow_experimental_codecs);
}
}
ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const
{
if (const auto * func = ast->as<ASTFunction>())
{
ASTPtr codecs_descriptions = std::make_shared<ASTExpressionList>();
bool is_compression = false;
bool has_none = false;
std::optional<size_t> generic_compression_codec_pos;
std::set<size_t> post_processing_codecs;
bool can_substitute_codec_arguments = true;
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
{
const auto & inner_codec_ast = func->arguments->children[i];
String codec_family_name;
ASTPtr codec_arguments;
if (const auto * family_name = inner_codec_ast->as<ASTIdentifier>())
{
codec_family_name = family_name->name();
codec_arguments = {};
}
else if (const auto * ast_func = inner_codec_ast->as<ASTFunction>())
{
codec_family_name = ast_func->name;
codec_arguments = ast_func->arguments;
}
else
throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
/// Default codec replaced with current default codec which may depend on different
/// settings (and properties of data) in runtime.
CompressionCodecPtr result_codec;
if (codec_family_name == DEFAULT_CODEC_NAME)
{
if (codec_arguments != nullptr)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"{} codec cannot have any arguments, it's just an alias for codec specified in config.xml", DEFAULT_CODEC_NAME);
result_codec = default_codec;
codecs_descriptions->children.emplace_back(std::make_shared<ASTIdentifier>(DEFAULT_CODEC_NAME));
}
else
{
if (column_type)
{
CompressionCodecPtr prev_codec;
IDataType::StreamCallbackWithType callback = [&](
const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type)
{
if (ISerialization::isSpecialCompressionAllowed(substream_path))
{
result_codec = getImpl(codec_family_name, codec_arguments, &substream_type);
/// Case for column Tuple, which compressed with codec which depends on data type, like Delta.
/// We cannot substitute parameters for such codecs.
if (prev_codec && prev_codec->getHash() != result_codec->getHash())
can_substitute_codec_arguments = false;
prev_codec = result_codec;
}
};
ISerialization::SubstreamPath stream_path;
column_type->enumerateStreams(column_type->getDefaultSerialization(), callback, stream_path);
if (!result_codec)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName());
}
else
{
result_codec = getImpl(codec_family_name, codec_arguments, nullptr);
}
if (!allow_experimental_codecs && result_codec->isExperimental())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Codec {} is experimental and not meant to be used in production."
" You can enable it with the 'allow_experimental_codecs' setting.",
codec_family_name);
codecs_descriptions->children.emplace_back(result_codec->getCodecDesc());
}
is_compression |= result_codec->isCompression();
has_none |= result_codec->isNone();
if (!generic_compression_codec_pos && result_codec->isGenericCompression())
generic_compression_codec_pos = i;
if (result_codec->isPostProcessing())
post_processing_codecs.insert(i);
}
String codec_description = queryToString(codecs_descriptions);
if (sanity_check)
{
if (codecs_descriptions->children.size() > 1 && has_none)
throw Exception(
"It does not make sense to have codec NONE along with other compression codecs: " + codec_description
+ ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
ErrorCodes::BAD_ARGUMENTS);
/// Allow to explicitly specify single NONE codec if user don't want any compression.
/// But applying other transformations solely without compression (e.g. Delta) does not make sense.
/// It's okay to apply post-processing codecs solely without anything else.
if (!is_compression && !has_none && post_processing_codecs.size() != codecs_descriptions->children.size())
throw Exception(
"Compression codec " + codec_description
+ " does not compress anything."
" You may want to add generic compression algorithm after other transformations, like: "
+ codec_description
+ ", LZ4."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
ErrorCodes::BAD_ARGUMENTS);
/// It does not make sense to apply any non-post-processing codecs
/// after post-processing one.
if (!post_processing_codecs.empty() &&
*post_processing_codecs.begin() != codecs_descriptions->children.size() - post_processing_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any non-post-processing codecs after"
" post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs'"
" to skip this check).", ErrorCodes::BAD_ARGUMENTS);
/// It does not make sense to apply any transformations after generic compression algorithm
/// So, generic compression can be only one and only at the end.
if (generic_compression_codec_pos &&
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - post_processing_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any transformations after generic compression algorithm."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS);
}
/// For columns with nested types like Tuple(UInt32, UInt64) we
/// obviously cannot substitute parameters for codecs which depend on
/// data type, because for the first column Delta(4) is suitable and
/// Delta(8) for the second. So we should leave codec description as is
/// and deduce them in get method for each subtype separately. For all
/// other types it's better to substitute parameters, for better
/// readability and backward compatibility.
if (can_substitute_codec_arguments)
{
std::shared_ptr<ASTFunction> result = std::make_shared<ASTFunction>();
result->name = "CODEC";
result->arguments = codecs_descriptions;
return result;
}
else
{
return ast;
}
}
throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC);
}
CompressionCodecPtr CompressionCodecFactory::get(
const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default, bool only_generic) const

View File

@ -14,6 +14,8 @@
namespace DB
{
static constexpr auto DEFAULT_CODEC_NAME = "Default";
class ICompressionCodec;
using CompressionCodecPtr = std::shared_ptr<ICompressionCodec>;

View File

@ -0,0 +1,214 @@
/**
* This file contains a part of CompressionCodecFactory methods definitions and
* is needed only because they have dependencies on DataTypes.
* They are not useful for fuzzers, so we leave them in other translation unit.
*/
#include <Compression/CompressionFactory.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeNested.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int UNKNOWN_CODEC;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
void CompressionCodecFactory::validateCodec(
const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs) const
{
if (family_name.empty())
throw Exception("Compression codec name cannot be empty", ErrorCodes::BAD_ARGUMENTS);
if (level)
{
auto literal = std::make_shared<ASTLiteral>(static_cast<UInt64>(*level));
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)),
{}, sanity_check, allow_experimental_codecs);
}
else
{
auto identifier = std::make_shared<ASTIdentifier>(Poco::toUpper(family_name));
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier),
{}, sanity_check, allow_experimental_codecs);
}
}
ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const
{
if (const auto * func = ast->as<ASTFunction>())
{
ASTPtr codecs_descriptions = std::make_shared<ASTExpressionList>();
bool is_compression = false;
bool has_none = false;
std::optional<size_t> generic_compression_codec_pos;
std::set<size_t> post_processing_codecs;
bool can_substitute_codec_arguments = true;
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
{
const auto & inner_codec_ast = func->arguments->children[i];
String codec_family_name;
ASTPtr codec_arguments;
if (const auto * family_name = inner_codec_ast->as<ASTIdentifier>())
{
codec_family_name = family_name->name();
codec_arguments = {};
}
else if (const auto * ast_func = inner_codec_ast->as<ASTFunction>())
{
codec_family_name = ast_func->name;
codec_arguments = ast_func->arguments;
}
else
throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
/// Default codec replaced with current default codec which may depend on different
/// settings (and properties of data) in runtime.
CompressionCodecPtr result_codec;
if (codec_family_name == DEFAULT_CODEC_NAME)
{
if (codec_arguments != nullptr)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"{} codec cannot have any arguments, it's just an alias for codec specified in config.xml", DEFAULT_CODEC_NAME);
result_codec = default_codec;
codecs_descriptions->children.emplace_back(std::make_shared<ASTIdentifier>(DEFAULT_CODEC_NAME));
}
else
{
if (column_type)
{
CompressionCodecPtr prev_codec;
IDataType::StreamCallbackWithType callback = [&](
const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type)
{
if (ISerialization::isSpecialCompressionAllowed(substream_path))
{
result_codec = getImpl(codec_family_name, codec_arguments, &substream_type);
/// Case for column Tuple, which compressed with codec which depends on data type, like Delta.
/// We cannot substitute parameters for such codecs.
if (prev_codec && prev_codec->getHash() != result_codec->getHash())
can_substitute_codec_arguments = false;
prev_codec = result_codec;
}
};
ISerialization::SubstreamPath stream_path;
column_type->enumerateStreams(column_type->getDefaultSerialization(), callback, stream_path);
if (!result_codec)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName());
}
else
{
result_codec = getImpl(codec_family_name, codec_arguments, nullptr);
}
if (!allow_experimental_codecs && result_codec->isExperimental())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Codec {} is experimental and not meant to be used in production."
" You can enable it with the 'allow_experimental_codecs' setting.",
codec_family_name);
codecs_descriptions->children.emplace_back(result_codec->getCodecDesc());
}
is_compression |= result_codec->isCompression();
has_none |= result_codec->isNone();
if (!generic_compression_codec_pos && result_codec->isGenericCompression())
generic_compression_codec_pos = i;
if (result_codec->isPostProcessing())
post_processing_codecs.insert(i);
}
String codec_description = queryToString(codecs_descriptions);
if (sanity_check)
{
if (codecs_descriptions->children.size() > 1 && has_none)
throw Exception(
"It does not make sense to have codec NONE along with other compression codecs: " + codec_description
+ ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
ErrorCodes::BAD_ARGUMENTS);
/// Allow to explicitly specify single NONE codec if user don't want any compression.
/// But applying other transformations solely without compression (e.g. Delta) does not make sense.
/// It's okay to apply post-processing codecs solely without anything else.
if (!is_compression && !has_none && post_processing_codecs.size() != codecs_descriptions->children.size())
throw Exception(
"Compression codec " + codec_description
+ " does not compress anything."
" You may want to add generic compression algorithm after other transformations, like: "
+ codec_description
+ ", LZ4."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
ErrorCodes::BAD_ARGUMENTS);
/// It does not make sense to apply any non-post-processing codecs
/// after post-processing one.
if (!post_processing_codecs.empty() &&
*post_processing_codecs.begin() != codecs_descriptions->children.size() - post_processing_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any non-post-processing codecs after"
" post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs'"
" to skip this check).", ErrorCodes::BAD_ARGUMENTS);
/// It does not make sense to apply any transformations after generic compression algorithm
/// So, generic compression can be only one and only at the end.
if (generic_compression_codec_pos &&
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - post_processing_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any transformations after generic compression algorithm."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS);
}
/// For columns with nested types like Tuple(UInt32, UInt64) we
/// obviously cannot substitute parameters for codecs which depend on
/// data type, because for the first column Delta(4) is suitable and
/// Delta(8) for the second. So we should leave codec description as is
/// and deduce them in get method for each subtype separately. For all
/// other types it's better to substitute parameters, for better
/// readability and backward compatibility.
if (can_substitute_codec_arguments)
{
std::shared_ptr<ASTFunction> result = std::make_shared<ASTFunction>();
result->name = "CODEC";
result->arguments = codecs_descriptions;
return result;
}
else
{
return ast;
}
}
throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC);
}
}

View File

@ -3,8 +3,3 @@ target_link_libraries (compressed_buffer PRIVATE dbms)
add_executable (cached_compressed_read_buffer cached_compressed_read_buffer.cpp)
target_link_libraries (cached_compressed_read_buffer PRIVATE dbms)
if (ENABLE_FUZZING)
add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp)
target_link_libraries (compressed_buffer_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE})
endif ()

View File

@ -0,0 +1,2 @@
add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp)
target_link_libraries (compressed_buffer_fuzzer PRIVATE fuzz_compression clickhouse_common_io ${LIB_FUZZING_ENGINE})

View File

@ -1,3 +1,7 @@
if (ENABLE_EXAMPLES)
add_subdirectory(examples)
endif ()
if (ENABLE_FUZZING)
add_subdirectory(fuzzers)
endif()

View File

@ -55,7 +55,7 @@ class IColumn;
M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \
M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \
M(Seconds, drain_timeout, DBMS_DEFAULT_DRAIN_TIMEOUT_SEC, "", 0) \
M(Seconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \
M(Seconds, tcp_keep_alive_timeout, 290 /* less than DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC */, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \
M(Milliseconds, hedged_connection_timeout_ms, DBMS_DEFAULT_HEDGED_CONNECTION_TIMEOUT_MS, "Connection timeout for establishing connection with replica for Hedged requests", 0) \
M(Milliseconds, receive_data_timeout_ms, DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_MS, "Connection timeout for receiving first packet of data or packet with positive progress from replica", 0) \
M(Bool, use_hedged_requests, true, "Use hedged requests for distributed queries", 0) \
@ -115,6 +115,7 @@ class IColumn;
M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \
M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \
M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \
M(Bool, enable_positional_arguments, false, "Enable positional arguments in ORDER BY, GROUP BY and LIMIT BY", 0) \
\
M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \
M(UInt64, parallel_replicas_count, 0, "", 0) \
@ -253,6 +254,7 @@ class IColumn;
M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \
M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \
M(Bool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \
M(Bool, empty_result_for_aggregation_by_constant_keys_on_empty_set, true, "Return empty result when aggregating by constant keys on empty set.", 0) \
M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \
M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \
M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \
@ -353,6 +355,8 @@ class IColumn;
M(UInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.", 0)\
M(UInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.", 0) \
\
M(UInt64, max_backup_threads, 0, "The maximum number of threads to execute a BACKUP or RESTORE request. By default, it is determined automatically.", 0) \
\
M(Bool, log_profile_events, true, "Log query performance statistics into the query_log, query_thread_log and query_views_log.", 0) \
M(Bool, log_query_settings, true, "Log query settings into the query_log.", 0) \
M(Bool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \

View File

@ -8,11 +8,6 @@ target_link_libraries (field PRIVATE dbms)
add_executable (string_ref_hash string_ref_hash.cpp)
target_link_libraries (string_ref_hash PRIVATE clickhouse_common_io)
if (ENABLE_FUZZING)
add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp)
target_link_libraries (names_and_types_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE})
endif ()
add_executable (mysql_protocol mysql_protocol.cpp)
target_link_libraries (mysql_protocol PRIVATE dbms)
if(USE_SSL)

View File

@ -0,0 +1,2 @@
add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp)
target_link_libraries (names_and_types_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE})

View File

@ -26,23 +26,6 @@ namespace ErrorCodes
IDataType::~IDataType() = default;
String IDataType::getName() const
{
if (custom_name)
{
return custom_name->getName();
}
else
{
return doGetName();
}
}
String IDataType::doGetName() const
{
return getFamilyName();
}
void IDataType::updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint)
{
/// Update the average value size hint if amount of read rows isn't too small

View File

@ -62,7 +62,13 @@ public:
/// static constexpr bool is_parametric = false;
/// Name of data type (examples: UInt64, Array(String)).
String getName() const;
String getName() const
{
if (custom_name)
return custom_name->getName();
else
return doGetName();
}
/// Name of data type family (example: FixedString, Array).
virtual const char * getFamilyName() const = 0;
@ -105,7 +111,7 @@ public:
void enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback) const { enumerateStreams(serialization, callback, {}); }
protected:
virtual String doGetName() const;
virtual String doGetName() const { return getFamilyName(); }
virtual SerializationPtr doGetDefaultSerialization() const = 0;
DataTypePtr getTypeForSubstream(const ISerialization::SubstreamPath & substream_path) const;

View File

@ -403,7 +403,7 @@ void DatabaseAtomic::assertCanBeDetached(bool cleanup)
}
DatabaseTablesIteratorPtr
DatabaseAtomic::getTablesIterator(ContextPtr local_context, const IDatabase::FilterByNameFunction & filter_by_table_name)
DatabaseAtomic::getTablesIterator(ContextPtr local_context, const IDatabase::FilterByNameFunction & filter_by_table_name) const
{
auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(local_context, filter_by_table_name);
return std::make_unique<AtomicDatabaseTablesSnapshotIterator>(std::move(typeid_cast<DatabaseTablesSnapshotIterator &>(*base_iter)));

View File

@ -45,7 +45,7 @@ public:
void drop(ContextPtr /*context*/) override;
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override;
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override;
void loadStoredObjects(ContextMutablePtr context, bool has_force_restore_data_flag, bool force_attach) override;

View File

@ -52,7 +52,7 @@ DatabaseDictionary::DatabaseDictionary(const String & name_, ContextPtr context_
{
}
Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_name)
Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_name) const
{
Tables tables;
auto load_results = getContext()->getExternalDictionariesLoader().getLoadResults(filter_by_name);
@ -77,7 +77,7 @@ StoragePtr DatabaseDictionary::tryGetTable(const String & table_name, ContextPtr
return createStorageDictionary(getDatabaseName(), load_result, getContext());
}
DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name)
DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) const
{
return std::make_unique<DatabaseTablesSnapshotIterator>(listTables(filter_by_table_name), getDatabaseName());
}

View File

@ -34,7 +34,7 @@ public:
StoragePtr tryGetTable(const String & table_name, ContextPtr context) const override;
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override;
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override;
bool empty() const override;
@ -50,7 +50,7 @@ protected:
private:
Poco::Logger * log;
Tables listTables(const FilterByNameFunction & filter_by_name);
Tables listTables(const FilterByNameFunction & filter_by_name) const;
};
}

View File

@ -143,7 +143,7 @@ StoragePtr DatabaseLazy::tryGetTable(const String & table_name) const
return loadTable(table_name);
}
DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name)
DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) const
{
std::lock_guard lock(mutex);
Strings filtered_tables;
@ -304,7 +304,7 @@ void DatabaseLazy::clearExpiredTables() const
}
DatabaseLazyIterator::DatabaseLazyIterator(DatabaseLazy & database_, Strings && table_names_)
DatabaseLazyIterator::DatabaseLazyIterator(const DatabaseLazy & database_, Strings && table_names_)
: IDatabaseTablesIterator(database_.database_name)
, database(database_)
, table_names(std::move(table_names_))

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