Merge branch 'master' into sqltest

This commit is contained in:
Alexey Milovidov 2023-08-02 20:08:39 +02:00
commit a98e869912
104 changed files with 4195 additions and 441 deletions

View File

@ -17,7 +17,8 @@
#ifndef METROHASH_PLATFORM_H
#define METROHASH_PLATFORM_H
#include <stdint.h>
#include <bit>
#include <cstdint>
#include <cstring>
// rotate right idiom recognized by most compilers
@ -33,6 +34,11 @@ inline static uint64_t read_u64(const void * const ptr)
// so we use memcpy() which is the most portable. clang & gcc usually translates `memcpy()` into a single `load` instruction
// when hardware supports it, so using memcpy() is efficient too.
memcpy(&result, ptr, sizeof(result));
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
result = std::byteswap(result);
#endif
return result;
}
@ -40,6 +46,11 @@ inline static uint64_t read_u32(const void * const ptr)
{
uint32_t result;
memcpy(&result, ptr, sizeof(result));
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
result = std::byteswap(result);
#endif
return result;
}
@ -47,6 +58,11 @@ inline static uint64_t read_u16(const void * const ptr)
{
uint16_t result;
memcpy(&result, ptr, sizeof(result));
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
result = std::byteswap(result);
#endif
return result;
}

View File

@ -6,7 +6,7 @@ Usage:
Build deb package with `clang-14` in `debug` mode:
```
$ mkdir deb/test_output
$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-14 --build-type=debug
$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-14 --debug-build
$ ls -l deb/test_output
-rw-r--r-- 1 root root 3730 clickhouse-client_22.2.2+debug_all.deb
-rw-r--r-- 1 root root 84221888 clickhouse-common-static_22.2.2+debug_amd64.deb

View File

@ -112,12 +112,12 @@ def run_docker_image_with_env(
subprocess.check_call(cmd, shell=True)
def is_release_build(build_type: str, package_type: str, sanitizer: str) -> bool:
return build_type == "" and package_type == "deb" and sanitizer == ""
def is_release_build(debug_build: bool, package_type: str, sanitizer: str) -> bool:
return not debug_build and package_type == "deb" and sanitizer == ""
def parse_env_variables(
build_type: str,
debug_build: bool,
compiler: str,
sanitizer: str,
package_type: str,
@ -240,7 +240,7 @@ def parse_env_variables(
build_target = (
f"{build_target} clickhouse-odbc-bridge clickhouse-library-bridge"
)
if is_release_build(build_type, package_type, sanitizer):
if is_release_build(debug_build, package_type, sanitizer):
cmake_flags.append("-DSPLIT_DEBUG_SYMBOLS=ON")
result.append("WITH_PERFORMANCE=1")
if is_cross_arm:
@ -255,8 +255,8 @@ def parse_env_variables(
if sanitizer:
result.append(f"SANITIZER={sanitizer}")
if build_type:
result.append(f"BUILD_TYPE={build_type.capitalize()}")
if debug_build:
result.append("BUILD_TYPE=Debug")
else:
result.append("BUILD_TYPE=None")
@ -361,7 +361,7 @@ def parse_args() -> argparse.Namespace:
help="ClickHouse git repository",
)
parser.add_argument("--output-dir", type=dir_name, required=True)
parser.add_argument("--build-type", choices=("debug", ""), default="")
parser.add_argument("--debug-build", action="store_true")
parser.add_argument(
"--compiler",
@ -467,7 +467,7 @@ def main():
build_image(image_with_version, dockerfile)
env_prepared = parse_env_variables(
args.build_type,
args.debug_build,
args.compiler,
args.sanitizer,
args.package_type,

View File

@ -95,6 +95,7 @@ RUN python3 -m pip install --no-cache-dir \
pytest-timeout \
pytest-xdist \
pytz \
pyyaml==5.3.1 \
redis \
requests-kerberos \
tzlocal==2.1 \

View File

@ -14,6 +14,20 @@ Supported platforms:
- PowerPC 64 LE (experimental)
- RISC-V 64 (experimental)
## Building in docker
We use the docker image `clickhouse/binary-builder` for our CI builds. It contains everything necessary to build the binary and packages. There is a script `docker/packager/packager` to ease the image usage:
```bash
# define a directory for the output artifacts
output_dir="build_results"
# a simplest build
./docker/packager/packager --package-type=binary --output-dir "$output_dir"
# build debian packages
./docker/packager/packager --package-type=deb --output-dir "$output_dir"
# by default, debian packages use thin LTO, so we can override it to speed up the build
CMAKE_FLAGS='-DENABLE_THINLTO=' ./docker/packager/packager --package-type=deb --output-dir "./$(git rev-parse --show-cdup)/build_results"
```
## Building on Ubuntu
The following tutorial is based on Ubuntu Linux.

View File

@ -35,7 +35,7 @@ The [system.clusters](../../operations/system-tables/clusters.md) system table c
When creating a new replica of the database, this replica creates tables by itself. If the replica has been unavailable for a long time and has lagged behind the replication log — it checks its local metadata with the current metadata in ZooKeeper, moves the extra tables with data to a separate non-replicated database (so as not to accidentally delete anything superfluous), creates the missing tables, updates the table names if they have been renamed. The data is replicated at the `ReplicatedMergeTree` level, i.e. if the table is not replicated, the data will not be replicated (the database is responsible only for metadata).
[`ALTER TABLE ATTACH|FETCH|DROP|DROP DETACHED|DETACH PARTITION|PART`](../../sql-reference/statements/alter/partition.md) queries are allowed but not replicated. The database engine will only add/fetch/remove the partition/part to the current replica. However, if the table itself uses a Replicated table engine, then the data will be replicated after using `ATTACH`.
[`ALTER TABLE FREEZE|ATTACH|FETCH|DROP|DROP DETACHED|DETACH PARTITION|PART`](../../sql-reference/statements/alter/partition.md) queries are allowed but not replicated. The database engine will only add/fetch/remove the partition/part to the current replica. However, if the table itself uses a Replicated table engine, then the data will be replicated after using `ATTACH`.
## Usage Example {#usage-example}

View File

@ -60,6 +60,7 @@ Engines in the family:
- [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md)
- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md)
- [PostgreSQL](../../engines/table-engines/integrations/postgresql.md)
- [S3Queue](../../engines/table-engines/integrations/s3queue.md)
### Special Engines {#special-engines}

View File

@ -0,0 +1,224 @@
---
slug: /en/engines/table-engines/integrations/s3queue
sidebar_position: 7
sidebar_label: S3Queue
---
# S3Queue Table Engine
This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ecosystem and allows streaming import. This engine is similar to the [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) engines, but provides S3-specific features.
## Create Table {#creating-a-table}
``` sql
CREATE TABLE s3_queue_engine_table (name String, value UInt32)
ENGINE = S3Queue(path [, NOSIGN | aws_access_key_id, aws_secret_access_key,] format, [compression])
[SETTINGS]
[mode = 'unordered',]
[after_processing = 'keep',]
[keeper_path = '',]
[s3queue_loading_retries = 0,]
[s3queue_polling_min_timeout_ms = 1000,]
[s3queue_polling_max_timeout_ms = 10000,]
[s3queue_polling_backoff_ms = 0,]
[s3queue_tracked_files_limit = 1000,]
[s3queue_tracked_file_ttl_sec = 0,]
[s3queue_polling_size = 50,]
```
**Engine parameters**
- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path).
- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed.
- `format` — The [format](../../../interfaces/formats.md#formats) of the file.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3).
- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension.
**Example**
```sql
CREATE TABLE s3queue_engine_table (name String, value UInt32)
ENGINE=S3Queue('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/*', 'CSV', 'gzip')
SETTINGS
mode = 'ordred';
```
Using named collections:
``` xml
<clickhouse>
<named_collections>
<s3queue_conf>
<url>'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/*</url>
<access_key_id>test<access_key_id>
<secret_access_key>test</secret_access_key>
</s3queue_conf>
</named_collections>
</clickhouse>
```
```sql
CREATE TABLE s3queue_engine_table (name String, value UInt32)
ENGINE=S3Queue(s3queue_conf, format = 'CSV', compression_method = 'gzip')
SETTINGS
mode = 'ordred';
```
## Settings {#s3queue-settings}
### mode {#mode}
Possible values:
- unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKeeper.
- ordered — With ordered mode, only the max name of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper.
Default value: `unordered`.
### after_processing {#after_processing}
Delete or keep file after successful processing.
Possible values:
- keep.
- delete.
Default value: `keep`.
### keeper_path {#keeper_path}
The path in ZooKeeper can be specified as a table engine setting or default path can be formed from the global configuration-provided path and table UUID.
Possible values:
- String.
Default value: `/`.
### s3queue_loading_retries {#s3queue_loading_retries}
Retry file loading up to specified number of times. By default, there are no retries.
Possible values:
- Positive integer.
Default value: `0`.
### s3queue_polling_min_timeout_ms {#s3queue_polling_min_timeout_ms}
Minimal timeout before next polling (in milliseconds).
Possible values:
- Positive integer.
Default value: `1000`.
### s3queue_polling_max_timeout_ms {#s3queue_polling_max_timeout_ms}
Maximum timeout before next polling (in milliseconds).
Possible values:
- Positive integer.
Default value: `10000`.
### s3queue_polling_backoff_ms {#s3queue_polling_backoff_ms}
Polling backoff (in milliseconds).
Possible values:
- Positive integer.
Default value: `0`.
### s3queue_tracked_files_limit {#s3queue_tracked_files_limit}
Allows to limit the number of Zookeeper nodes if the 'unordered' mode is used, does nothing for 'ordered' mode.
If limit reached the oldest processed files will be deleted from ZooKeeper node and processed again.
Possible values:
- Positive integer.
Default value: `1000`.
### s3queue_tracked_file_ttl_sec {#s3queue_tracked_file_ttl_sec}
Maximum number of seconds to store processed files in ZooKeeper node (store forever by default) for 'unordered' mode, does nothing for 'ordered' mode.
After the specified number of seconds, the file will be re-imported.
Possible values:
- Positive integer.
Default value: `0`.
### s3queue_polling_size {#s3queue_polling_size}
Maximum files to fetch from S3 with SELECT or in background task.
Engine takes files for processing from S3 in batches.
We limit the batch size to increase concurrency if multiple table engines with the same `keeper_path` consume files from the same path.
Possible values:
- Positive integer.
Default value: `50`.
## S3-related Settings {#s3-settings}
Engine supports all s3 related settings. For more information about S3 settings see [here](../../../engines/table-engines/integrations/s3.md).
## Description {#description}
`SELECT` is not particularly useful for streaming import (except for debugging), because each file can be imported only once. It is more practical to create real-time threads using [materialized views](../../../sql-reference/statements/create/view.md). To do this:
1. Use the engine to create a table for consuming from specified path in S3 and consider it a data stream.
2. Create a table with the desired structure.
3. Create a materialized view that converts data from the engine and puts it into a previously created table.
When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the background.
Example:
``` sql
CREATE TABLE s3queue_engine_table (name String, value UInt32)
ENGINE=S3Queue('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/*', 'CSV', 'gzip')
SETTINGS
mode = 'unordred',
keeper_path = '/clickhouse/s3queue/';
CREATE TABLE stats (name String, value UInt32)
ENGINE = MergeTree() ORDER BY name;
CREATE MATERIALIZED VIEW consumer TO stats
AS SELECT name, value FROM s3queue_engine_table;
SELECT * FROM stats ORDER BY name;
```
## Virtual columns {#virtual-columns}
- `_path` — Path to the file.
- `_file` — Name of the file.
For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns).
## Wildcards In Path {#wildcards-in-path}
`path` argument can specify multiple files using bash-like wildcards. For being processed file should exist and match to the whole path pattern. Listing of files is determined during `SELECT` (not at `CREATE` moment).
- `*` — Substitutes any number of any characters except `/` including empty string.
- `?` — Substitutes any single character.
- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`.
- `{N..M}` — Substitutes any number in range from N to M including both borders. N and M can have leading zeroes e.g. `000..078`.
Constructions with `{}` are similar to the [remote](../../../sql-reference/table-functions/remote.md) table function.
:::note
If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`.
:::

View File

@ -0,0 +1,26 @@
---
slug: /en/operations/optimizing-performance/profile-guided-optimization
sidebar_position: 54
sidebar_label: Profile Guided Optimization (PGO)
---
import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md';
# Profile Guided Optimization
Profile-Guided Optimization (PGO) is a compiler optimization technique where a program is optimized based on the runtime profile.
According to the tests, PGO helps with achieving better performance for ClickHouse. According to the tests, we see improvements up to 15% in QPS on the ClickBench test suite. The more detailed results are available [here](https://pastebin.com/xbue3HMU). The performance benefits depend on your typical workload - you can get better or worse results.
More information about PGO in ClickHouse you can read in the corresponding GitHub [issue](https://github.com/ClickHouse/ClickHouse/issues/44567).
## How to build ClickHouse with PGO?
There are two major kinds of PGO: [Instrumentation](https://clang.llvm.org/docs/UsersManual.html#using-sampling-profilers) and [Sampling](https://clang.llvm.org/docs/UsersManual.html#using-sampling-profilers) (also known as AutoFDO). In this guide is described the Instrumentation PGO with ClickHouse.
1. Build ClickHouse in Instrumented mode. In Clang it can be done via passing `-fprofile-instr-generate` option to `CXXFLAGS`.
2. Run instrumented ClickHouse on a sample workload. Here you need to use your usual workload. One of the approaches could be using [ClickBench](https://github.com/ClickHouse/ClickBench) as a sample workload. ClickHouse in the instrumentation mode could work slowly so be ready for that and do not run instrumented ClickHouse in performance-critical environments.
3. Recompile ClickHouse once again with `-fprofile-instr-use` compiler flags and profiles that are collected from the previous step.
A more detailed guide on how to apply PGO is in the Clang [documentation](https://clang.llvm.org/docs/UsersManual.html#profile-guided-optimization).
If you are going to collect a sample workload directly from a production environment, we recommend trying to use Sampling PGO.

View File

@ -4578,3 +4578,28 @@ Type: Int64
Default: 0
## precise_float_parsing {#precise_float_parsing}
Switches [Float32/Float64](../../sql-reference/data-types/float.md) parsing algorithms:
* If the value is `1`, then precise method is used. It is slower than fast method, but it always returns a number that is the closest machine representable number to the input.
* Otherwise, fast method is used (default). It usually returns the same value as precise, but in rare cases result may differ by one or two least significant digits.
Possible values: `0`, `1`.
Default value: `0`.
Example:
```sql
SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_parsing = 0;
┌─toFloat64('1.7091')─┬─toFloat64('1.5008753E7')─┐
│ 1.7090999999999998 │ 15008753.000000002 │
└─────────────────────┴──────────────────────────┘
SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_parsing = 1;
┌─toFloat64('1.7091')─┬─toFloat64('1.5008753E7')─┐
│ 1.7091 │ 15008753 │
└─────────────────────┴──────────────────────────┘
```

View File

@ -48,7 +48,7 @@ Columns:
- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Total number of rows read from all tables and table functions participated in query. It includes usual subqueries, subqueries for `IN` and `JOIN`. For distributed queries `read_rows` includes the total number of rows read at all replicas. Each replica sends its `read_rows` value, and the server-initiator of the query summarizes all received and local values. The cache volumes do not affect this value.
- `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Total number of bytes read from all tables and table functions participated in query. It includes usual subqueries, subqueries for `IN` and `JOIN`. For distributed queries `read_bytes` includes the total number of rows read at all replicas. Each replica sends its `read_bytes` value, and the server-initiator of the query summarizes all received and local values. The cache volumes do not affect this value.
- `written_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` queries, the number of written rows. For other queries, the column value is 0.
- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` queries, the number of written bytes. For other queries, the column value is 0.
- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` queries, the number of written bytes (uncompressed). For other queries, the column value is 0.
- `result_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of rows in a result of the `SELECT` query, or a number of rows in the `INSERT` query.
- `result_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — RAM volume in bytes used to store a query result.
- `memory_usage` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Memory consumption by the query.

View File

@ -51,3 +51,7 @@ keeper foo bar
- `rmr <path>` -- Recursively deletes path. Confirmation required
- `flwc <command>` -- Executes four-letter-word command
- `help` -- Prints this message
- `get_stat [path]` -- Returns the node's stat (default `.`)
- `find_super_nodes <threshold> [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`)
- `delete_stable_backups` -- Deletes ClickHouse nodes used for backups that are now inactive
- `find_big_family [path] [n]` -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10)

View File

@ -4213,3 +4213,29 @@ SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS sessi
- Запрос: `SELECT * FROM file('sample.csv')`
Если чтение и обработка `sample.csv` прошли успешно, файл будет переименован в `processed_sample_1683473210851438.csv`.
## precise_float_parsing {#precise_float_parsing}
Позволяет выбрать алгоритм, используемый при парсинге [Float32/Float64](../../sql-reference/data-types/float.md):
* Если установлено значение `1`, то используется точный метод. Он более медленный, но всегда возвращает число, наиболее близкое к входному значению.
* В противном случае используется быстрый метод (поведение по умолчанию). Обычно результат его работы совпадает с результатом, полученным точным методом, однако в редких случаях он может отличаться на 1 или 2 наименее значимых цифры.
Возможные значения: `0`, `1`.
Значение по умолчанию: `0`.
Пример:
```sql
SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_parsing = 0;
┌─toFloat64('1.7091')─┬─toFloat64('1.5008753E7')─┐
│ 1.7090999999999998 │ 15008753.000000002 │
└─────────────────────┴──────────────────────────┘
SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_parsing = 1;
┌─toFloat64('1.7091')─┬─toFloat64('1.5008753E7')─┐
│ 1.7091 │ 15008753 │
└─────────────────────┴──────────────────────────┘
```

View File

@ -55,6 +55,9 @@ contents:
- src: clickhouse
dst: /usr/bin/clickhouse-keeper
type: symlink
- src: clickhouse
dst: /usr/bin/clickhouse-keeper-client
type: symlink
- src: root/usr/bin/clickhouse-report
dst: /usr/bin/clickhouse-report
- src: root/usr/bin/clickhouse-server

View File

@ -1,5 +1,6 @@
#include "Commands.h"
#include <queue>
#include "KeeperClient.h"
@ -24,8 +25,18 @@ void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con
else
path = client->cwd;
for (const auto & child : client->zookeeper->getChildren(path))
std::cout << child << " ";
auto children = client->zookeeper->getChildren(path);
std::sort(children.begin(), children.end());
bool need_space = false;
for (const auto & child : children)
{
if (std::exchange(need_space, true))
std::cout << " ";
std::cout << child;
}
std::cout << "\n";
}
@ -130,6 +141,173 @@ void GetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co
std::cout << client->zookeeper->get(client->getAbsolutePath(query->args[0].safeGet<String>())) << "\n";
}
bool GetStatCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String arg;
if (!parseKeeperPath(pos, expected, arg))
return true;
node->args.push_back(std::move(arg));
return true;
}
void GetStatCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
Coordination::Stat stat;
String path;
if (!query->args.empty())
path = client->getAbsolutePath(query->args[0].safeGet<String>());
else
path = client->cwd;
client->zookeeper->get(path, &stat);
std::cout << "cZxid = " << stat.czxid << "\n";
std::cout << "mZxid = " << stat.mzxid << "\n";
std::cout << "pZxid = " << stat.pzxid << "\n";
std::cout << "ctime = " << stat.ctime << "\n";
std::cout << "mtime = " << stat.mtime << "\n";
std::cout << "version = " << stat.version << "\n";
std::cout << "cversion = " << stat.cversion << "\n";
std::cout << "aversion = " << stat.aversion << "\n";
std::cout << "ephemeralOwner = " << stat.ephemeralOwner << "\n";
std::cout << "dataLength = " << stat.dataLength << "\n";
std::cout << "numChildren = " << stat.numChildren << "\n";
}
bool FindSuperNodes::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
ASTPtr threshold;
if (!ParserUnsignedInteger{}.parse(pos, threshold, expected))
return false;
node->args.push_back(threshold->as<ASTLiteral &>().value);
String path;
if (!parseKeeperPath(pos, expected, path))
path = ".";
node->args.push_back(std::move(path));
return true;
}
void FindSuperNodes::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
auto threshold = query->args[0].safeGet<UInt64>();
auto path = client->getAbsolutePath(query->args[1].safeGet<String>());
Coordination::Stat stat;
client->zookeeper->get(path, &stat);
if (stat.numChildren >= static_cast<Int32>(threshold))
{
std::cout << static_cast<String>(path) << "\t" << stat.numChildren << "\n";
return;
}
auto children = client->zookeeper->getChildren(path);
std::sort(children.begin(), children.end());
for (const auto & child : children)
{
auto next_query = *query;
next_query.args[1] = DB::Field(path / child);
execute(&next_query, client);
}
}
bool DeleteStableBackups::parse(IParser::Pos & /* pos */, std::shared_ptr<ASTKeeperQuery> & /* node */, Expected & /* expected */) const
{
return true;
}
void DeleteStableBackups::execute(const ASTKeeperQuery * /* query */, KeeperClient * client) const
{
client->askConfirmation(
"You are going to delete all inactive backups in /clickhouse/backups.",
[client]
{
fs::path backup_root = "/clickhouse/backups";
auto backups = client->zookeeper->getChildren(backup_root);
std::sort(backups.begin(), backups.end());
for (const auto & child : backups)
{
auto backup_path = backup_root / child;
std::cout << "Found backup " << backup_path << ", checking if it's active\n";
String stage_path = backup_path / "stage";
auto stages = client->zookeeper->getChildren(stage_path);
bool is_active = false;
for (const auto & stage : stages)
{
if (startsWith(stage, "alive"))
{
is_active = true;
break;
}
}
if (is_active)
{
std::cout << "Backup " << backup_path << " is active, not going to delete\n";
continue;
}
std::cout << "Backup " << backup_path << " is not active, deleting it\n";
client->zookeeper->removeRecursive(backup_path);
}
});
}
bool FindBigFamily::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String path;
if (!parseKeeperPath(pos, expected, path))
path = ".";
node->args.push_back(std::move(path));
ASTPtr count;
if (ParserUnsignedInteger{}.parse(pos, count, expected))
node->args.push_back(count->as<ASTLiteral &>().value);
else
node->args.push_back(UInt64(10));
return true;
}
void FindBigFamily::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
auto path = client->getAbsolutePath(query->args[0].safeGet<String>());
auto n = query->args[1].safeGet<UInt64>();
std::vector<std::tuple<Int32, String>> result;
std::queue<fs::path> queue;
queue.push(path);
while (!queue.empty())
{
auto next_path = queue.front();
queue.pop();
auto children = client->zookeeper->getChildren(next_path);
std::transform(children.cbegin(), children.cend(), children.begin(), [&](const String & child) { return next_path / child; });
auto response = client->zookeeper->get(children);
for (size_t i = 0; i < response.size(); ++i)
{
result.emplace_back(response[i].stat.numChildren, children[i]);
queue.push(children[i]);
}
}
std::sort(result.begin(), result.end(), std::greater());
for (UInt64 i = 0; i < std::min(result.size(), static_cast<size_t>(n)); ++i)
std::cout << std::get<1>(result[i]) << "\t" << std::get<0>(result[i]) << "\n";
}
bool RMCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String arg;
@ -170,7 +348,7 @@ bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptr<ASTKeeperQuery
void HelpCommand::execute(const ASTKeeperQuery * /* query */, KeeperClient * /* client */) const
{
for (const auto & pair : KeeperClient::commands)
std::cout << pair.second->getHelpMessage() << "\n";
std::cout << pair.second->generateHelpString() << "\n";
}
bool FourLetterWordCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const

View File

@ -21,6 +21,12 @@ public:
virtual String getName() const = 0;
virtual ~IKeeperClientCommand() = default;
String generateHelpString() const
{
return fmt::vformat(getHelpMessage(), fmt::make_format_args(getName()));
}
};
using Command = std::shared_ptr<IKeeperClientCommand>;
@ -34,7 +40,7 @@ class LSCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "ls [path] -- Lists the nodes for the given path (default: cwd)"; }
String getHelpMessage() const override { return "{} [path] -- Lists the nodes for the given path (default: cwd)"; }
};
class CDCommand : public IKeeperClientCommand
@ -45,7 +51,7 @@ class CDCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "cd [path] -- Change the working path (default `.`)"; }
String getHelpMessage() const override { return "{} [path] -- Change the working path (default `.`)"; }
};
class SetCommand : public IKeeperClientCommand
@ -58,7 +64,7 @@ class SetCommand : public IKeeperClientCommand
String getHelpMessage() const override
{
return "set <path> <value> [version] -- Updates the node's value. Only update if version matches (default: -1)";
return "{} <path> <value> [version] -- Updates the node's value. Only update if version matches (default: -1)";
}
};
@ -70,7 +76,7 @@ class CreateCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "create <path> <value> -- Creates new node"; }
String getHelpMessage() const override { return "{} <path> <value> -- Creates new node"; }
};
class GetCommand : public IKeeperClientCommand
@ -81,9 +87,63 @@ class GetCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "get <path> -- Returns the node's value"; }
String getHelpMessage() const override { return "{} <path> -- Returns the node's value"; }
};
class GetStatCommand : public IKeeperClientCommand
{
String getName() const override { return "get_stat"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "{} [path] -- Returns the node's stat (default `.`)"; }
};
class FindSuperNodes : public IKeeperClientCommand
{
String getName() const override { return "find_super_nodes"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override
{
return "{} <threshold> [path] -- Finds nodes with number of children larger than some threshold for the given path (default `.`)";
}
};
class DeleteStableBackups : public IKeeperClientCommand
{
String getName() const override { return "delete_stable_backups"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override
{
return "{} -- Deletes ClickHouse nodes used for backups that are now inactive";
}
};
class FindBigFamily : public IKeeperClientCommand
{
String getName() const override { return "find_big_family"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override
{
return "{} [path] [n] -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10)";
}
};
class RMCommand : public IKeeperClientCommand
{
String getName() const override { return "rm"; }
@ -92,7 +152,7 @@ class RMCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "remove <path> -- Remove the node"; }
String getHelpMessage() const override { return "{} <path> -- Remove the node"; }
};
class RMRCommand : public IKeeperClientCommand
@ -103,7 +163,7 @@ class RMRCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "rmr <path> -- Recursively deletes path. Confirmation required"; }
String getHelpMessage() const override { return "{} <path> -- Recursively deletes path. Confirmation required"; }
};
class HelpCommand : public IKeeperClientCommand
@ -114,7 +174,7 @@ class HelpCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "help -- Prints this message"; }
String getHelpMessage() const override { return "{} -- Prints this message"; }
};
class FourLetterWordCommand : public IKeeperClientCommand
@ -125,7 +185,7 @@ class FourLetterWordCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "flwc <command> -- Executes four-letter-word command"; }
String getHelpMessage() const override { return "{} <command> -- Executes four-letter-word command"; }
};
}

View File

@ -177,6 +177,10 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */)
std::make_shared<SetCommand>(),
std::make_shared<CreateCommand>(),
std::make_shared<GetCommand>(),
std::make_shared<GetStatCommand>(),
std::make_shared<FindSuperNodes>(),
std::make_shared<DeleteStableBackups>(),
std::make_shared<FindBigFamily>(),
std::make_shared<RMCommand>(),
std::make_shared<RMRCommand>(),
std::make_shared<HelpCommand>(),

View File

@ -58,6 +58,7 @@ bool KeeperParser::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
String command_name(pos->begin, pos->end);
std::transform(command_name.begin(), command_name.end(), command_name.begin(), [](unsigned char c) { return std::tolower(c); });
Command command;
auto iter = KeeperClient::commands.find(command_name);

View File

@ -248,6 +248,7 @@ add_object_library(clickhouse_storages_distributed Storages/Distributed)
add_object_library(clickhouse_storages_mergetree Storages/MergeTree)
add_object_library(clickhouse_storages_liveview Storages/LiveView)
add_object_library(clickhouse_storages_windowview Storages/WindowView)
add_object_library(clickhouse_storages_s3queue Storages/S3Queue)
add_object_library(clickhouse_client Client)
add_object_library(clickhouse_bridge BridgeHelper)
add_object_library(clickhouse_server Server)

View File

@ -208,10 +208,10 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
* we allow exception about memory limit exceeded to be thrown only on next allocation.
* So, we allow over-allocations.
*/
Int64 will_be = size + amount.fetch_add(size, std::memory_order_relaxed);
Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed);
auto metric_loaded = metric.load(std::memory_order_relaxed);
if (metric_loaded != CurrentMetrics::end())
if (metric_loaded != CurrentMetrics::end() && size)
CurrentMetrics::add(metric_loaded, size);
Int64 current_hard_limit = hard_limit.load(std::memory_order_relaxed);

View File

@ -45,6 +45,25 @@ size_t shortest_literal_length(const Literals & literals)
return shortest;
}
const char * skipNameCapturingGroup(const char * pos, size_t offset, const char * end)
{
const char special = *(pos + offset) == '<' ? '>' : '\'';
offset ++;
while (pos + offset < end)
{
const char cur = *(pos + offset);
if (cur == special)
{
return pos + offset;
}
if (('0' <= cur && cur <= '9') || ('a' <= cur && cur <= 'z') || ('A' <= cur && cur <= 'Z'))
offset ++;
else
return pos;
}
return pos;
}
const char * analyzeImpl(
std::string_view regexp,
const char * pos,
@ -247,10 +266,15 @@ const char * analyzeImpl(
break;
}
}
/// (?:regex) means non-capturing parentheses group
if (pos + 2 < end && pos[1] == '?' && pos[2] == ':')
{
pos += 2;
}
if (pos + 3 < end && pos[1] == '?' && (pos[2] == '<' || pos[2] == '\'' || (pos[2] == 'P' && pos[3] == '<')))
{
pos = skipNameCapturingGroup(pos, pos[2] == 'P' ? 3: 2, end);
}
Literal group_required_substr;
bool group_is_trival = true;
Literals group_alters;

View File

@ -47,4 +47,8 @@ TEST(OptimizeRE, analyze)
test_f("abc|(:?xx|yy|zz|x?)def", "", {"abc", "def"});
test_f("abc|(:?xx|yy|zz|x?){1,2}def", "", {"abc", "def"});
test_f(R"(\\A(?:(?:[-0-9_a-z]+(?:\\.[-0-9_a-z]+)*)/k8s1)\\z)", "/k8s1");
test_f("[a-zA-Z]+(?P<num>\\d+)", "");
test_f("[a-zA-Z]+(?<num>\\d+)", "");
test_f("[a-zA-Z]+(?'num'\\d+)", "");
test_f("[a-zA-Z]+(?x<num>\\d+)", "x<num>");
}

View File

@ -46,15 +46,6 @@
#define DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION 54454
/// Version of ClickHouse TCP protocol.
///
/// Should be incremented manually on protocol changes.
///
/// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION,
/// later is just a number for server version (one number instead of commit SHA)
/// for simplicity (sometimes it may be more convenient in some use cases).
#define DBMS_TCP_PROTOCOL_VERSION 54464
#define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449
#define DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS_IN_INSERT 54456
@ -77,3 +68,14 @@
#define DBMS_MIN_PROTOCOL_VERSION_WITH_TOTAL_BYTES_IN_PROGRESS 54463
#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54464
#define DBMS_MIN_REVISION_WITH_SPARSE_SERIALIZATION 54465
/// Version of ClickHouse TCP protocol.
///
/// Should be incremented manually on protocol changes.
///
/// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION,
/// later is just a number for server version (one number instead of commit SHA)
/// for simplicity (sometimes it may be more convenient in some use cases).
#define DBMS_TCP_PROTOCOL_VERSION 54465

View File

@ -104,6 +104,7 @@ class IColumn;
M(UInt64, s3_retry_attempts, 10, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \
M(UInt64, s3_request_timeout_ms, 3000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \
M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \
M(String, s3queue_default_zookeeper_path, "/s3queue/", "Default zookeeper path prefix for S3Queue engine", 0) \
M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \
M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \
M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \
@ -1031,7 +1032,8 @@ class IColumn;
M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \
\
M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \
M(Bool, input_format_csv_allow_variable_number_of_columns, false, "Ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values", 0) \
M(Bool, input_format_csv_allow_variable_number_of_columns, false, "Ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values", 0) \
M(Bool, precise_float_parsing, false, "Prefer more precise (but slower) float parsing algorithm", 0) \
// End of FORMAT_FACTORY_SETTINGS
// Please add settings non-related to formats into the COMMON_SETTINGS above.

View File

@ -175,4 +175,11 @@ IMPLEMENT_SETTING_ENUM(ORCCompression, ErrorCodes::BAD_ARGUMENTS,
{"zlib", FormatSettings::ORCCompression::ZLIB},
{"lz4", FormatSettings::ORCCompression::LZ4}})
IMPLEMENT_SETTING_ENUM(S3QueueMode, ErrorCodes::BAD_ARGUMENTS,
{{"ordered", S3QueueMode::ORDERED},
{"unordered", S3QueueMode::UNORDERED}})
IMPLEMENT_SETTING_ENUM(S3QueueAction, ErrorCodes::BAD_ARGUMENTS,
{{"keep", S3QueueAction::KEEP},
{"delete", S3QueueAction::DELETE}})
}

View File

@ -221,4 +221,21 @@ enum class ParallelReplicasCustomKeyFilterType : uint8_t
DECLARE_SETTING_ENUM(ParallelReplicasCustomKeyFilterType)
DECLARE_SETTING_ENUM(LocalFSReadMethod)
enum class S3QueueMode
{
ORDERED,
UNORDERED,
};
DECLARE_SETTING_ENUM(S3QueueMode)
enum class S3QueueAction
{
KEEP,
DELETE,
};
DECLARE_SETTING_ENUM(S3QueueAction)
}

View File

@ -666,7 +666,7 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_
{
for (const auto & command : query_alter->command_list->children)
{
if (!isSupportedAlterType(command->as<ASTAlterCommand&>().type))
if (!isSupportedAlterTypeForOnClusterDDLQuery(command->as<ASTAlterCommand&>().type))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported type of ALTER query");
}
}
@ -1474,7 +1474,7 @@ bool DatabaseReplicated::shouldReplicateQuery(const ContextPtr & query_context,
/// Some ALTERs are not replicated on database level
if (const auto * alter = query_ptr->as<const ASTAlterQuery>())
{
if (alter->isAttachAlter() || alter->isFetchAlter() || alter->isDropPartitionAlter() || is_keeper_map_table(query_ptr))
if (alter->isAttachAlter() || alter->isFetchAlter() || alter->isDropPartitionAlter() || is_keeper_map_table(query_ptr) || alter->isFreezeAlter())
return false;
if (has_many_shards() || !is_replicated_table(query_ptr))

View File

@ -135,9 +135,19 @@ size_t NativeWriter::write(const Block & block)
if (client_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION)
{
auto info = column.type->getSerializationInfo(*column.column);
serialization = column.type->getSerialization(*info);
bool has_custom = false;
if (client_revision >= DBMS_MIN_REVISION_WITH_SPARSE_SERIALIZATION)
{
serialization = column.type->getSerialization(*info);
has_custom = info->hasCustomSerialization();
}
else
{
serialization = column.type->getDefaultSerialization();
column.column = recursiveRemoveSparse(column.column);
}
bool has_custom = info->hasCustomSerialization();
writeBinary(static_cast<UInt8>(has_custom), ostr);
if (has_custom)
info->serialializeKindBinary(ostr);

View File

@ -1040,13 +1040,21 @@ inline void convertFromTime<DataTypeDateTime>(DataTypeDateTime::FieldType & x, t
/** Conversion of strings to numbers, dates, datetimes: through parsing.
*/
template <typename DataType>
void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool precise_float_parsing)
{
readText(x, rb);
if constexpr (std::is_floating_point_v<typename DataType::FieldType>)
{
if (precise_float_parsing)
readFloatTextPrecise(x, rb);
else
readFloatTextFast(x, rb);
}
else
readText(x, rb);
}
template <>
inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool)
{
DayNum tmp(0);
readDateText(tmp, rb, *time_zone);
@ -1054,7 +1062,7 @@ inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb
}
template <>
inline void parseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
inline void parseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool)
{
ExtendedDayNum tmp(0);
readDateText(tmp, rb, *time_zone);
@ -1064,7 +1072,7 @@ inline void parseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer
// NOTE: no need of extra overload of DateTime64, since readDateTimeText64 has different signature and that case is explicitly handled in the calling code.
template <>
inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool)
{
time_t time = 0;
readDateTimeText(time, rb, *time_zone);
@ -1072,7 +1080,7 @@ inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuf
}
template <>
inline void parseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
inline void parseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool)
{
UUID tmp;
readUUIDText(tmp, rb);
@ -1080,7 +1088,7 @@ inline void parseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer & rb
}
template <>
inline void parseImpl<DataTypeIPv4>(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
inline void parseImpl<DataTypeIPv4>(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool)
{
IPv4 tmp;
readIPv4Text(tmp, rb);
@ -1088,7 +1096,7 @@ inline void parseImpl<DataTypeIPv4>(DataTypeIPv4::FieldType & x, ReadBuffer & rb
}
template <>
inline void parseImpl<DataTypeIPv6>(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
inline void parseImpl<DataTypeIPv6>(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool)
{
IPv6 tmp;
readIPv6Text(tmp, rb);
@ -1096,16 +1104,21 @@ inline void parseImpl<DataTypeIPv6>(DataTypeIPv6::FieldType & x, ReadBuffer & rb
}
template <typename DataType>
bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool precise_float_parsing)
{
if constexpr (std::is_floating_point_v<typename DataType::FieldType>)
return tryReadFloatText(x, rb);
{
if (precise_float_parsing)
return tryReadFloatTextPrecise(x, rb);
else
return tryReadFloatTextFast(x, rb);
}
else /*if constexpr (is_integer_v<typename DataType::FieldType>)*/
return tryReadIntText(x, rb);
}
template <>
inline bool tryParseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
inline bool tryParseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool)
{
DayNum tmp(0);
if (!tryReadDateText(tmp, rb, *time_zone))
@ -1115,7 +1128,7 @@ inline bool tryParseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer &
}
template <>
inline bool tryParseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
inline bool tryParseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool)
{
ExtendedDayNum tmp(0);
if (!tryReadDateText(tmp, rb, *time_zone))
@ -1125,7 +1138,7 @@ inline bool tryParseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuff
}
template <>
inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool)
{
time_t tmp = 0;
if (!tryReadDateTimeText(tmp, rb, *time_zone))
@ -1135,7 +1148,7 @@ inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, Read
}
template <>
inline bool tryParseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
inline bool tryParseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool)
{
UUID tmp;
if (!tryReadUUIDText(tmp, rb))
@ -1146,7 +1159,7 @@ inline bool tryParseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer &
}
template <>
inline bool tryParseImpl<DataTypeIPv4>(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
inline bool tryParseImpl<DataTypeIPv4>(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool)
{
IPv4 tmp;
if (!tryReadIPv4Text(tmp, rb))
@ -1157,7 +1170,7 @@ inline bool tryParseImpl<DataTypeIPv4>(DataTypeIPv4::FieldType & x, ReadBuffer &
}
template <>
inline bool tryParseImpl<DataTypeIPv6>(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
inline bool tryParseImpl<DataTypeIPv6>(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool)
{
IPv6 tmp;
if (!tryReadIPv6Text(tmp, rb))
@ -1336,6 +1349,16 @@ struct ConvertThroughParsing
size_t current_offset = 0;
bool precise_float_parsing = false;
if (DB::CurrentThread::isInitialized())
{
const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext();
if (query_context)
precise_float_parsing = query_context->getSettingsRef().precise_float_parsing;
}
for (size_t i = 0; i < size; ++i)
{
size_t next_offset = std::is_same_v<FromDataType, DataTypeString> ? (*offsets)[i] : (current_offset + fixed_string_size);
@ -1402,7 +1425,7 @@ struct ConvertThroughParsing
}
}
parseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
parseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone, precise_float_parsing);
} while (false);
}
}
@ -1472,7 +1495,7 @@ struct ConvertThroughParsing
}
}
parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone, precise_float_parsing);
} while (false);
}
}

View File

@ -153,15 +153,10 @@ struct IntHash64Impl
template<typename T, typename HashFunction>
T combineHashesFunc(T t1, T t2)
{
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
T tmp;
reverseMemcpy(&tmp, &t1, sizeof(T));
t1 = tmp;
reverseMemcpy(&tmp, &t2, sizeof(T));
t2 = tmp;
#endif
T hashes[] = {t1, t2};
return HashFunction::apply(reinterpret_cast<const char *>(hashes), 2 * sizeof(T));
transformEndianness<std::endian::little>(t1);
transformEndianness<std::endian::little>(t2);
const T hashes[] {t1, t2};
return HashFunction::apply(reinterpret_cast<const char *>(hashes), sizeof(hashes));
}
@ -184,21 +179,14 @@ struct HalfMD5Impl
MD5_Update(&ctx, reinterpret_cast<const unsigned char *>(begin), size);
MD5_Final(buf.char_data, &ctx);
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
return buf.uint64_data; /// No need to flip bytes on big endian machines
#else
return std::byteswap(buf.uint64_data); /// Compatibility with existing code. Cast need for old poco AND macos where UInt64 != uint64_t
#endif
/// Compatibility with existing code. Cast need for old poco AND macos where UInt64 != uint64_t
transformEndianness<std::endian::big>(buf.uint64_data);
return buf.uint64_data;
}
static UInt64 combineHashes(UInt64 h1, UInt64 h2)
{
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
h1 = std::byteswap(h1);
h2 = std::byteswap(h2);
#endif
UInt64 hashes[] = {h1, h2};
return apply(reinterpret_cast<const char *>(hashes), 16);
return combineHashesFunc<UInt64, HalfMD5Impl>(h1, h2);
}
/// If true, it will use intHash32 or intHash64 to hash POD types. This behaviour is intended for better performance of some functions.
@ -311,15 +299,8 @@ struct SipHash64Impl
static constexpr auto name = "sipHash64";
using ReturnType = UInt64;
static UInt64 apply(const char * begin, size_t size)
{
return sipHash64(begin, size);
}
static UInt64 combineHashes(UInt64 h1, UInt64 h2)
{
return combineHashesFunc<UInt64, SipHash64Impl>(h1, h2);
}
static UInt64 apply(const char * begin, size_t size) { return sipHash64(begin, size); }
static UInt64 combineHashes(UInt64 h1, UInt64 h2) { return combineHashesFunc<UInt64, SipHash64Impl>(h1, h2); }
static constexpr bool use_int_hash_for_pods = false;
};
@ -336,12 +317,10 @@ struct SipHash64KeyedImpl
static UInt64 combineHashesKeyed(const Key & key, UInt64 h1, UInt64 h2)
{
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
h1 = std::byteswap(h1);
h2 = std::byteswap(h2);
#endif
UInt64 hashes[] = {h1, h2};
return applyKeyed(key, reinterpret_cast<const char *>(hashes), 2 * sizeof(UInt64));
transformEndianness<std::endian::little>(h1);
transformEndianness<std::endian::little>(h2);
const UInt64 hashes[]{h1, h2};
return applyKeyed(key, reinterpret_cast<const char *>(hashes), sizeof(hashes));
}
static constexpr bool use_int_hash_for_pods = false;
@ -353,15 +332,8 @@ struct SipHash128Impl
using ReturnType = UInt128;
static UInt128 combineHashes(UInt128 h1, UInt128 h2)
{
return combineHashesFunc<UInt128, SipHash128Impl>(h1, h2);
}
static UInt128 apply(const char * data, const size_t size)
{
return sipHash128(data, size);
}
static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc<UInt128, SipHash128Impl>(h1, h2); }
static UInt128 apply(const char * data, const size_t size) { return sipHash128(data, size); }
static constexpr bool use_int_hash_for_pods = false;
};
@ -378,15 +350,10 @@ struct SipHash128KeyedImpl
static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2)
{
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
UInt128 tmp;
reverseMemcpy(&tmp, &h1, sizeof(UInt128));
h1 = tmp;
reverseMemcpy(&tmp, &h2, sizeof(UInt128));
h2 = tmp;
#endif
UInt128 hashes[] = {h1, h2};
return applyKeyed(key, reinterpret_cast<const char *>(hashes), 2 * sizeof(UInt128));
transformEndianness<std::endian::little>(h1);
transformEndianness<std::endian::little>(h2);
const UInt128 hashes[]{h1, h2};
return applyKeyed(key, reinterpret_cast<const char *>(hashes), sizeof(hashes));
}
static constexpr bool use_int_hash_for_pods = false;
@ -531,10 +498,7 @@ struct MurmurHash3Impl64
return h[0] ^ h[1];
}
static UInt64 combineHashes(UInt64 h1, UInt64 h2)
{
return IntHash64Impl::apply(h1) ^ h2;
}
static UInt64 combineHashes(UInt64 h1, UInt64 h2) { return IntHash64Impl::apply(h1) ^ h2; }
static constexpr bool use_int_hash_for_pods = false;
};
@ -552,10 +516,7 @@ struct MurmurHash3Impl128
return *reinterpret_cast<UInt128 *>(bytes);
}
static UInt128 combineHashes(UInt128 h1, UInt128 h2)
{
return combineHashesFunc<UInt128, MurmurHash3Impl128>(h1, h2);
}
static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc<UInt128, MurmurHash3Impl128>(h1, h2); }
static constexpr bool use_int_hash_for_pods = false;
};
@ -1040,11 +1001,10 @@ private:
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(column))
{
const typename ColVecType::Container & vec_from = col_from->getData();
size_t size = vec_from.size();
const size_t size = vec_from.size();
for (size_t i = 0; i < size; ++i)
{
ToType hash;
if constexpr (Impl::use_int_hash_for_pods)
{
if constexpr (std::is_same_v<ToType, UInt64>)
@ -1058,13 +1018,8 @@ private:
hash = JavaHashImpl::apply(vec_from[i]);
else
{
FromType value = vec_from[i];
if constexpr (std::endian::native == std::endian::big)
{
FromType value_reversed;
reverseMemcpy(&value_reversed, &value, sizeof(value));
value = value_reversed;
}
auto value = vec_from[i];
transformEndianness<std::endian::little>(value);
hash = apply(key, reinterpret_cast<const char *>(&value), sizeof(value));
}
}
@ -1078,8 +1033,8 @@ private:
else if (auto col_from_const = checkAndGetColumnConst<ColVecType>(column))
{
auto value = col_from_const->template getValue<FromType>();
ToType hash;
ToType hash;
if constexpr (Impl::use_int_hash_for_pods)
{
if constexpr (std::is_same_v<ToType, UInt64>)
@ -1093,17 +1048,12 @@ private:
hash = JavaHashImpl::apply(value);
else
{
if constexpr (std::endian::native == std::endian::big)
{
FromType value_reversed;
reverseMemcpy(&value_reversed, &value, sizeof(value));
value = value_reversed;
}
transformEndianness<std::endian::little>(value);
hash = apply(key, reinterpret_cast<const char *>(&value), sizeof(value));
}
}
size_t size = vec_to.size();
const size_t size = vec_to.size();
if constexpr (first)
vec_to.assign(size, hash);
else
@ -1120,6 +1070,16 @@ private:
{
using ColVecType = ColumnVectorOrDecimal<FromType>;
static const auto to_little_endian = [](auto & value)
{
// IPv6 addresses are parsed into four 32-bit components in big-endian ordering on both platforms, so no change is necessary.
// Reference: `parseIPv6orIPv4` in src/Common/formatIPv6.h.
if constexpr (std::endian::native == std::endian::big && std::is_same_v<std::remove_reference_t<decltype(value)>, IPv6>)
return;
transformEndianness<std::endian::little>(value);
};
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(column))
{
const typename ColVecType::Container & vec_from = col_from->getData();
@ -1131,9 +1091,10 @@ private:
hash = apply(key, reinterpret_cast<const char *>(&vec_from[i]), sizeof(vec_from[i]));
else
{
char tmp_buffer[sizeof(vec_from[i])];
reverseMemcpy(tmp_buffer, &vec_from[i], sizeof(vec_from[i]));
hash = apply(key, reinterpret_cast<const char *>(tmp_buffer), sizeof(vec_from[i]));
auto value = vec_from[i];
to_little_endian(value);
hash = apply(key, reinterpret_cast<const char *>(&value), sizeof(value));
}
if constexpr (first)
vec_to[i] = hash;
@ -1144,17 +1105,10 @@ private:
else if (auto col_from_const = checkAndGetColumnConst<ColVecType>(column))
{
auto value = col_from_const->template getValue<FromType>();
to_little_endian(value);
ToType hash;
if constexpr (std::endian::native == std::endian::little)
hash = apply(key, reinterpret_cast<const char *>(&value), sizeof(value));
else
{
char tmp_buffer[sizeof(value)];
reverseMemcpy(tmp_buffer, &value, sizeof(value));
hash = apply(key, reinterpret_cast<const char *>(tmp_buffer), sizeof(value));
}
size_t size = vec_to.size();
const auto hash = apply(key, reinterpret_cast<const char *>(&value), sizeof(value));
const size_t size = vec_to.size();
if constexpr (first)
vec_to.assign(size, hash);
else
@ -1423,6 +1377,9 @@ public:
if constexpr (std::is_same_v<ToType, UInt128>) /// backward-compatible
{
if (std::endian::native == std::endian::big)
std::ranges::for_each(col_to->getData(), transformEndianness<std::endian::little, ToType>);
auto col_to_fixed_string = ColumnFixedString::create(sizeof(UInt128));
const auto & data = col_to->getData();
auto & chars = col_to_fixed_string->getChars();
@ -1676,21 +1633,8 @@ struct ImplWyHash64
static constexpr auto name = "wyHash64";
using ReturnType = UInt64;
static UInt64 apply(const char * s, const size_t len)
{
return wyhash(s, len, 0, _wyp);
}
static UInt64 combineHashes(UInt64 h1, UInt64 h2)
{
union
{
UInt64 u64[2];
char chars[16];
};
u64[0] = h1;
u64[1] = h2;
return apply(chars, 16);
}
static UInt64 apply(const char * s, const size_t len) { return wyhash(s, len, 0, _wyp); }
static UInt64 combineHashes(UInt64 h1, UInt64 h2) { return combineHashesFunc<UInt64, ImplWyHash64>(h1, h2); }
static constexpr bool use_int_hash_for_pods = false;
};

View File

@ -529,6 +529,11 @@ void tryReadIntTextUnsafe(T & x, ReadBuffer & buf)
template <typename T> void readFloatText(T & x, ReadBuffer & in);
template <typename T> bool tryReadFloatText(T & x, ReadBuffer & in);
template <typename T> void readFloatTextPrecise(T & x, ReadBuffer & in);
template <typename T> bool tryReadFloatTextPrecise(T & x, ReadBuffer & in);
template <typename T> void readFloatTextFast(T & x, ReadBuffer & in);
template <typename T> bool tryReadFloatTextFast(T & x, ReadBuffer & in);
/// simple: all until '\n' or '\t'
void readString(String & s, ReadBuffer & buf);

View File

@ -984,6 +984,8 @@ void Aggregator::executeOnBlockSmall(
}
executeImpl(result, row_begin, row_end, key_columns, aggregate_instructions);
CurrentMemoryTracker::check();
}
void Aggregator::mergeOnBlockSmall(
@ -1023,6 +1025,8 @@ void Aggregator::mergeOnBlockSmall(
#undef M
else
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
CurrentMemoryTracker::check();
}
void Aggregator::executeImpl(
@ -1383,11 +1387,8 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl(
}
void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl(
AggregatedDataVariants & data_variants,
size_t row_begin,
size_t row_end,
AggregateFunctionInstruction * aggregate_instructions) const
void NO_INLINE Aggregator::executeOnIntervalWithoutKey(
AggregatedDataVariants & data_variants, size_t row_begin, size_t row_end, AggregateFunctionInstruction * aggregate_instructions) const
{
/// `data_variants` will destroy the states of aggregate functions in the destructor
data_variants.aggregator = this;
@ -1414,7 +1415,7 @@ void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl(
}
}
void NO_INLINE Aggregator::mergeOnIntervalWithoutKeyImpl(
void NO_INLINE Aggregator::mergeOnIntervalWithoutKey(
AggregatedDataVariants & data_variants,
size_t row_begin,
size_t row_end,
@ -2921,6 +2922,7 @@ void NO_INLINE Aggregator::mergeBlockWithoutKeyStreamsImpl(
AggregateColumnsConstData aggregate_columns = params.makeAggregateColumnsData(block);
mergeWithoutKeyStreamsImpl(result, 0, block.rows(), aggregate_columns);
}
void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
AggregatedDataVariants & result,
size_t row_begin,
@ -3139,6 +3141,8 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari
LOG_TRACE(log, "Merged partially aggregated single-level data.");
}
CurrentMemoryTracker::check();
}

View File

@ -1118,9 +1118,55 @@ public:
AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block
bool & no_more_keys) const;
/** This array serves two purposes.
*
* Function arguments are collected side by side, and they do not need to be collected from different places. Also the array is made zero-terminated.
* The inner loop (for the case without_key) is almost twice as compact; performance gain of about 30%.
*/
struct AggregateFunctionInstruction
{
const IAggregateFunction * that{};
size_t state_offset{};
const IColumn ** arguments{};
const IAggregateFunction * batch_that{};
const IColumn ** batch_arguments{};
const UInt64 * offsets{};
bool has_sparse_arguments = false;
};
/// Used for optimize_aggregation_in_order:
/// - No two-level aggregation
/// - No external aggregation
/// - No without_key support (it is implemented using executeOnIntervalWithoutKey())
void executeOnBlockSmall(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions) const;
void executeOnIntervalWithoutKey(
AggregatedDataVariants & data_variants,
size_t row_begin,
size_t row_end,
AggregateFunctionInstruction * aggregate_instructions) const;
/// Used for aggregate projection.
bool mergeOnBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys) const;
void mergeOnBlockSmall(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
const AggregateColumnsConstData & aggregate_columns_data,
const ColumnRawPtrs & key_columns) const;
void mergeOnIntervalWithoutKey(
AggregatedDataVariants & data_variants,
size_t row_begin,
size_t row_end,
const AggregateColumnsConstData & aggregate_columns_data) const;
/** Convert the aggregation data structure into a block.
* If overflow_row = true, then aggregates for rows that are not included in max_rows_to_group_by are put in the first block.
*
@ -1178,22 +1224,6 @@ private:
AggregateFunctionsPlainPtrs aggregate_functions;
/** This array serves two purposes.
*
* Function arguments are collected side by side, and they do not need to be collected from different places. Also the array is made zero-terminated.
* The inner loop (for the case without_key) is almost twice as compact; performance gain of about 30%.
*/
struct AggregateFunctionInstruction
{
const IAggregateFunction * that{};
size_t state_offset{};
const IColumn ** arguments{};
const IAggregateFunction * batch_that{};
const IColumn ** batch_arguments{};
const UInt64 * offsets{};
bool has_sparse_arguments = false;
};
using AggregateFunctionInstructions = std::vector<AggregateFunctionInstruction>;
using NestedColumnsHolder = std::vector<std::vector<const IColumn *>>;
@ -1239,26 +1269,6 @@ private:
*/
void destroyAllAggregateStates(AggregatedDataVariants & result) const;
/// Used for optimize_aggregation_in_order:
/// - No two-level aggregation
/// - No external aggregation
/// - No without_key support (it is implemented using executeOnIntervalWithoutKeyImpl())
void executeOnBlockSmall(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions) const;
void mergeOnBlockSmall(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
const AggregateColumnsConstData & aggregate_columns_data,
const ColumnRawPtrs & key_columns) const;
void mergeOnBlockImpl(Block block, AggregatedDataVariants & result, bool no_more_keys) const;
void executeImpl(
AggregatedDataVariants & result,
size_t row_begin,
@ -1300,17 +1310,6 @@ private:
AggregateFunctionInstruction * aggregate_instructions,
Arena * arena) const;
void executeOnIntervalWithoutKeyImpl(
AggregatedDataVariants & data_variants,
size_t row_begin,
size_t row_end,
AggregateFunctionInstruction * aggregate_instructions) const;
void mergeOnIntervalWithoutKeyImpl(
AggregatedDataVariants & data_variants,
size_t row_begin,
size_t row_end,
const AggregateColumnsConstData & aggregate_columns_data) const;
template <typename Method>
void writeToTemporaryFileImpl(
AggregatedDataVariants & data_variants,

View File

@ -258,6 +258,9 @@ void FileSegment::resetDownloader()
void FileSegment::resetDownloaderUnlocked(const FileSegmentGuard::Lock &)
{
if (downloader_id.empty())
return;
LOG_TEST(log, "Resetting downloader from {}", downloader_id);
downloader_id.clear();
}
@ -266,7 +269,6 @@ void FileSegment::assertIsDownloaderUnlocked(const std::string & operation, cons
{
auto caller = getCallerId();
auto current_downloader = getDownloaderUnlocked(lock);
LOG_TEST(log, "Downloader id: {}, caller id: {}, operation: {}", current_downloader, caller, operation);
if (caller != current_downloader)
{

View File

@ -48,6 +48,11 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
namespace ActionLocks
{
extern const StorageActionBlockType PartsMerge;
}
namespace
{
class StorageWithComment : public IAST
@ -560,6 +565,10 @@ void SystemLog<LogElement>::prepareTable()
rename->elements.emplace_back(std::move(elem));
ActionLock merges_lock;
if (DatabaseCatalog::instance().getDatabase(table_id.database_name)->getUUID() == UUIDHelpers::Nil)
merges_lock = table->getActionLock(ActionLocks::PartsMerge);
auto query_context = Context::createCopy(context);
/// As this operation is performed automatically we don't want it to fail because of user dependencies on log tables
query_context->setSetting("check_table_dependencies", Field{false});

View File

@ -49,7 +49,7 @@ static ZooKeeperRetriesInfo getRetriesInfo()
);
}
bool isSupportedAlterType(int type)
bool isSupportedAlterTypeForOnClusterDDLQuery(int type)
{
assert(type != ASTAlterCommand::NO_TYPE);
static const std::unordered_set<int> unsupported_alter_types{
@ -90,7 +90,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context,
{
for (const auto & command : query_alter->command_list->children)
{
if (!isSupportedAlterType(command->as<ASTAlterCommand&>().type))
if (!isSupportedAlterTypeForOnClusterDDLQuery(command->as<ASTAlterCommand&>().type))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported type of ALTER query");
}
}

View File

@ -21,7 +21,7 @@ class Cluster;
using ClusterPtr = std::shared_ptr<Cluster>;
/// Returns true if provided ALTER type can be executed ON CLUSTER
bool isSupportedAlterType(int type);
bool isSupportedAlterTypeForOnClusterDDLQuery(int type);
struct DDLQueryOnClusterParams
{

View File

@ -13,7 +13,7 @@ namespace ErrorCodes
String ASTAlterCommand::getID(char delim) const
{
return String("AlterCommand") + delim + typeToString(type);
return fmt::format("AlterCommand{}{}", delim, type);
}
ASTPtr ASTAlterCommand::clone() const
@ -80,53 +80,6 @@ ASTPtr ASTAlterCommand::clone() const
return res;
}
const char * ASTAlterCommand::typeToString(ASTAlterCommand::Type type)
{
switch (type)
{
case ADD_COLUMN: return "ADD_COLUMN";
case DROP_COLUMN: return "DROP_COLUMN";
case MODIFY_COLUMN: return "MODIFY_COLUMN";
case COMMENT_COLUMN: return "COMMENT_COLUMN";
case RENAME_COLUMN: return "RENAME_COLUMN";
case MATERIALIZE_COLUMN: return "MATERIALIZE_COLUMN";
case MODIFY_ORDER_BY: return "MODIFY_ORDER_BY";
case MODIFY_SAMPLE_BY: return "MODIFY_SAMPLE_BY";
case MODIFY_TTL: return "MODIFY_TTL";
case MATERIALIZE_TTL: return "MATERIALIZE_TTL";
case MODIFY_SETTING: return "MODIFY_SETTING";
case RESET_SETTING: return "RESET_SETTING";
case MODIFY_QUERY: return "MODIFY_QUERY";
case REMOVE_TTL: return "REMOVE_TTL";
case REMOVE_SAMPLE_BY: return "REMOVE_SAMPLE_BY";
case ADD_INDEX: return "ADD_INDEX";
case DROP_INDEX: return "DROP_INDEX";
case MATERIALIZE_INDEX: return "MATERIALIZE_INDEX";
case ADD_CONSTRAINT: return "ADD_CONSTRAINT";
case DROP_CONSTRAINT: return "DROP_CONSTRAINT";
case ADD_PROJECTION: return "ADD_PROJECTION";
case DROP_PROJECTION: return "DROP_PROJECTION";
case MATERIALIZE_PROJECTION: return "MATERIALIZE_PROJECTION";
case DROP_PARTITION: return "DROP_PARTITION";
case DROP_DETACHED_PARTITION: return "DROP_DETACHED_PARTITION";
case ATTACH_PARTITION: return "ATTACH_PARTITION";
case MOVE_PARTITION: return "MOVE_PARTITION";
case REPLACE_PARTITION: return "REPLACE_PARTITION";
case FETCH_PARTITION: return "FETCH_PARTITION";
case FREEZE_PARTITION: return "FREEZE_PARTITION";
case FREEZE_ALL: return "FREEZE_ALL";
case UNFREEZE_PARTITION: return "UNFREEZE_PARTITION";
case UNFREEZE_ALL: return "UNFREEZE_ALL";
case DELETE: return "DELETE";
case UPDATE: return "UPDATE";
case NO_TYPE: return "NO_TYPE";
case LIVE_VIEW_REFRESH: return "LIVE_VIEW_REFRESH";
case MODIFY_DATABASE_SETTING: return "MODIFY_DATABASE_SETTING";
case MODIFY_COMMENT: return "MODIFY_COMMENT";
}
UNREACHABLE();
}
void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (type == ASTAlterCommand::ADD_COLUMN)

View File

@ -208,8 +208,6 @@ public:
ASTPtr clone() const override;
static const char * typeToString(Type type);
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};

View File

@ -159,14 +159,14 @@ void AggregatingInOrderTransform::consume(Chunk chunk)
if (group_by_key)
params->aggregator.mergeOnBlockSmall(variants, key_begin, key_end, aggregate_columns_data, key_columns_raw);
else
params->aggregator.mergeOnIntervalWithoutKeyImpl(variants, key_begin, key_end, aggregate_columns_data);
params->aggregator.mergeOnIntervalWithoutKey(variants, key_begin, key_end, aggregate_columns_data);
}
else
{
if (group_by_key)
params->aggregator.executeOnBlockSmall(variants, key_begin, key_end, key_columns_raw, aggregate_function_instructions.data());
else
params->aggregator.executeOnIntervalWithoutKeyImpl(variants, key_begin, key_end, aggregate_function_instructions.data());
params->aggregator.executeOnIntervalWithoutKey(variants, key_begin, key_end, aggregate_function_instructions.data());
}
}

View File

@ -175,10 +175,13 @@ struct SocketInterruptablePollWrapper
}
while (rc < 0 && errno == POCO_EINTR);
if (rc >= 1 && poll_buf[0].revents & POLLIN)
socket_ready = true;
if (rc >= 2 && poll_buf[1].revents & POLLIN)
fd_ready = true;
if (rc >= 1)
{
if (poll_buf[0].revents & POLLIN)
socket_ready = true;
if (poll_buf[1].revents & POLLIN)
fd_ready = true;
}
#endif
}

View File

@ -138,6 +138,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
ReadSettings read_settings;
if (read_with_direct_io)
read_settings.direct_io_threshold = 1;
read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true;
MergeTreeReaderSettings reader_settings =
{

View File

@ -1841,6 +1841,7 @@ bool MutateTask::prepare()
context_for_reading->setSetting("max_threads", 1);
context_for_reading->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false);
context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0));
context_for_reading->setSetting("read_from_filesystem_cache_if_exists_otherwise_bypass_cache", 1);
MutationHelpers::splitAndModifyMutationCommands(
ctx->source_part, ctx->metadata_snapshot,

View File

@ -125,9 +125,6 @@ void localBackup(
size_t try_no = 0;
const size_t max_tries = 10;
CleanupOnFail cleanup(disk_transaction ? std::function<void()>([]{}) :
[disk, destination_path]() { disk->removeRecursive(destination_path); });
/** Files in the directory can be permanently added and deleted.
* If some file is deleted during an attempt to make a backup, then try again,
* because it's important to take into account any new files that might appear.
@ -136,10 +133,30 @@ void localBackup(
{
try
{
if (copy_instead_of_hardlinks && !disk_transaction)
disk->copyDirectoryContent(source_path, disk, destination_path);
else
if (disk_transaction)
{
localBackupImpl(disk, disk_transaction.get(), source_path, destination_path, make_source_readonly, 0, max_level, copy_instead_of_hardlinks, files_to_copy_intead_of_hardlinks);
}
else if (copy_instead_of_hardlinks)
{
CleanupOnFail cleanup([disk, destination_path]() { disk->removeRecursive(destination_path); });
disk->copyDirectoryContent(source_path, disk, destination_path);
cleanup.success();
}
else
{
std::function<void()> cleaner;
if (disk->supportZeroCopyReplication())
/// Note: this code will create garbage on s3. We should always remove `copy_instead_of_hardlinks` files.
/// The third argument should be a list of exceptions, but (looks like) it is ignored for keep_all_shared_data = true.
cleaner = [disk, destination_path]() { disk->removeSharedRecursive(destination_path, /*keep_all_shared_data*/ true, {}); };
else
cleaner = [disk, destination_path]() { disk->removeRecursive(destination_path); };
CleanupOnFail cleanup(std::move(cleaner));
localBackupImpl(disk, disk_transaction.get(), source_path, destination_path, make_source_readonly, 0, max_level, false, files_to_copy_intead_of_hardlinks);
cleanup.success();
}
}
catch (const DB::ErrnoException & e)
{
@ -166,8 +183,6 @@ void localBackup(
break;
}
cleanup.success();
}
}

View File

@ -0,0 +1,351 @@
#include "IO/VarInt.h"
#include "config.h"
#if USE_AWS_S3
# include <algorithm>
# include <IO/Operators.h>
# include <IO/ReadBufferFromString.h>
# include <IO/ReadHelpers.h>
# include <Storages/S3Queue/S3QueueFilesMetadata.h>
# include <Storages/S3Queue/StorageS3Queue.h>
# include <Storages/StorageS3Settings.h>
# include <Storages/StorageSnapshot.h>
# include <base/sleep.h>
# include <Common/ZooKeeper/ZooKeeper.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TIMEOUT_EXCEEDED;
}
namespace
{
UInt64 getCurrentTime()
{
return std::chrono::duration_cast<std::chrono::seconds>(std::chrono::system_clock::now().time_since_epoch()).count();
}
}
void S3QueueFilesMetadata::S3QueueCollection::read(ReadBuffer & in)
{
files = {};
if (in.eof())
return;
size_t files_num;
in >> files_num >> "\n";
while (files_num--)
{
TrackedCollectionItem item;
in >> item.file_path >> "\n";
in >> item.timestamp >> "\n";
in >> item.retries_count >> "\n";
in >> item.last_exception >> "\n";
files.push_back(item);
}
}
void S3QueueFilesMetadata::S3QueueCollection::write(WriteBuffer & out) const
{
out << files.size() << "\n";
for (const auto & processed_file : files)
{
out << processed_file.file_path << "\n";
out << processed_file.timestamp << "\n";
out << processed_file.retries_count << "\n";
out << processed_file.last_exception << "\n";
}
}
String S3QueueFilesMetadata::S3QueueCollection::toString() const
{
WriteBufferFromOwnString out;
write(out);
return out.str();
}
S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::S3QueueCollection::getFileNames()
{
S3FilesCollection keys = {};
for (const auto & pair : files)
keys.insert(pair.file_path);
return keys;
}
S3QueueFilesMetadata::S3QueueProcessedCollection::S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_)
: max_size(max_size_), max_age(max_age_)
{
}
void S3QueueFilesMetadata::S3QueueProcessedCollection::parse(const String & collection_str)
{
ReadBufferFromString buf(collection_str);
read(buf);
if (max_age > 0) // Remove old items
{
std::erase_if(
files,
[timestamp = getCurrentTime(), this](const TrackedCollectionItem & processed_file)
{ return (timestamp - processed_file.timestamp) > max_age; });
}
}
void S3QueueFilesMetadata::S3QueueProcessedCollection::add(const String & file_name)
{
TrackedCollectionItem processed_file = { .file_path=file_name, .timestamp = getCurrentTime() };
files.push_back(processed_file);
/// TODO: it is strange that in parse() we take into account only max_age, but here only max_size.
while (files.size() > max_size)
{
files.pop_front();
}
}
S3QueueFilesMetadata::S3QueueFailedCollection::S3QueueFailedCollection(const UInt64 & max_retries_count_)
: max_retries_count(max_retries_count_)
{
}
void S3QueueFilesMetadata::S3QueueFailedCollection::parse(const String & collection_str)
{
ReadBufferFromString buf(collection_str);
read(buf);
}
bool S3QueueFilesMetadata::S3QueueFailedCollection::add(const String & file_name, const String & exception_message)
{
auto failed_it = std::find_if(
files.begin(), files.end(),
[&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; });
if (failed_it == files.end())
{
files.emplace_back(file_name, 0, max_retries_count, exception_message);
}
else if (failed_it->retries_count == 0 || --failed_it->retries_count == 0)
{
return false;
}
return true;
}
S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::S3QueueFailedCollection::getFileNames()
{
S3FilesCollection failed_keys;
for (const auto & pair : files)
{
if (pair.retries_count == 0)
failed_keys.insert(pair.file_path);
}
return failed_keys;
}
void S3QueueFilesMetadata::S3QueueProcessingCollection::parse(const String & collection_str)
{
ReadBufferFromString rb(collection_str);
Strings result;
readQuoted(result, rb);
files = S3FilesCollection(result.begin(), result.end());
}
void S3QueueFilesMetadata::S3QueueProcessingCollection::add(const Strings & file_names)
{
files.insert(file_names.begin(), file_names.end());
}
void S3QueueFilesMetadata::S3QueueProcessingCollection::remove(const String & file_name)
{
files.erase(file_name);
}
String S3QueueFilesMetadata::S3QueueProcessingCollection::toString() const
{
return DB::toString(Strings(files.begin(), files.end()));
}
S3QueueFilesMetadata::S3QueueFilesMetadata(
const StorageS3Queue * storage_,
const S3QueueSettings & settings_)
: storage(storage_)
, mode(settings_.mode)
, max_set_size(settings_.s3queue_tracked_files_limit.value)
, max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value)
, max_loading_retries(settings_.s3queue_loading_retries.value)
, zookeeper_processing_path(fs::path(storage->getZooKeeperPath()) / "processing")
, zookeeper_processed_path(fs::path(storage->getZooKeeperPath()) / "processed")
, zookeeper_failed_path(fs::path(storage->getZooKeeperPath()) / "failed")
, zookeeper_lock_path(fs::path(storage->getZooKeeperPath()) / "lock")
, log(&Poco::Logger::get("S3QueueFilesMetadata"))
{
}
void S3QueueFilesMetadata::setFileProcessed(const String & file_path)
{
auto zookeeper = storage->getZooKeeper();
auto lock = acquireLock(zookeeper);
switch (mode)
{
case S3QueueMode::UNORDERED:
{
S3QueueProcessedCollection processed_files(max_set_size, max_set_age_sec);
processed_files.parse(zookeeper->get(zookeeper_processed_path));
processed_files.add(file_path);
zookeeper->set(zookeeper_processed_path, processed_files.toString());
break;
}
case S3QueueMode::ORDERED:
{
// Check that we set in ZooKeeper node only maximum processed file path.
// This check can be useful, when multiple table engines consume in ordered mode.
String max_file = getMaxProcessedFile();
if (max_file.compare(file_path) <= 0)
zookeeper->set(zookeeper_processed_path, file_path);
break;
}
}
removeProcessingFile(file_path);
}
bool S3QueueFilesMetadata::setFileFailed(const String & file_path, const String & exception_message)
{
auto zookeeper = storage->getZooKeeper();
auto lock = acquireLock(zookeeper);
S3QueueFailedCollection failed_collection(max_loading_retries);
failed_collection.parse(zookeeper->get(zookeeper_failed_path));
const bool can_be_retried = failed_collection.add(file_path, exception_message);
zookeeper->set(zookeeper_failed_path, failed_collection.toString());
removeProcessingFile(file_path);
return can_be_retried;
}
S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getFailedFiles()
{
auto zookeeper = storage->getZooKeeper();
String failed_files = zookeeper->get(zookeeper_failed_path);
S3QueueFailedCollection failed_collection(max_loading_retries);
failed_collection.parse(failed_files);
return failed_collection.getFileNames();
}
String S3QueueFilesMetadata::getMaxProcessedFile()
{
auto zookeeper = storage->getZooKeeper();
return zookeeper->get(zookeeper_processed_path);
}
S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getProcessingFiles()
{
auto zookeeper = storage->getZooKeeper();
String processing_files;
if (!zookeeper->tryGet(zookeeper_processing_path, processing_files))
return {};
S3QueueProcessingCollection processing_collection;
if (!processing_files.empty())
processing_collection.parse(processing_files);
return processing_collection.getFileNames();
}
void S3QueueFilesMetadata::setFilesProcessing(const Strings & file_paths)
{
auto zookeeper = storage->getZooKeeper();
String processing_files;
zookeeper->tryGet(zookeeper_processing_path, processing_files);
S3QueueProcessingCollection processing_collection;
if (!processing_files.empty())
processing_collection.parse(processing_files);
processing_collection.add(file_paths);
if (zookeeper->exists(zookeeper_processing_path))
zookeeper->set(zookeeper_processing_path, processing_collection.toString());
else
zookeeper->create(zookeeper_processing_path, processing_collection.toString(), zkutil::CreateMode::Ephemeral);
}
void S3QueueFilesMetadata::removeProcessingFile(const String & file_path)
{
auto zookeeper = storage->getZooKeeper();
String processing_files;
zookeeper->tryGet(zookeeper_processing_path, processing_files);
S3QueueProcessingCollection processing_collection;
processing_collection.parse(processing_files);
processing_collection.remove(file_path);
zookeeper->set(zookeeper_processing_path, processing_collection.toString());
}
S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getUnorderedProcessedFiles()
{
auto zookeeper = storage->getZooKeeper();
S3QueueProcessedCollection processed_collection(max_set_size, max_set_age_sec);
processed_collection.parse(zookeeper->get(zookeeper_processed_path));
return processed_collection.getFileNames();
}
S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getProcessedFailedAndProcessingFiles()
{
S3FilesCollection processed_and_failed_files = getFailedFiles();
switch (mode)
{
case S3QueueMode::UNORDERED:
{
processed_and_failed_files.merge(getUnorderedProcessedFiles());
break;
}
case S3QueueMode::ORDERED:
{
processed_and_failed_files.insert(getMaxProcessedFile());
break;
}
}
processed_and_failed_files.merge(getProcessingFiles());
return processed_and_failed_files;
}
std::shared_ptr<zkutil::EphemeralNodeHolder> S3QueueFilesMetadata::acquireLock(zkutil::ZooKeeperPtr zookeeper)
{
UInt32 retry_count = 200;
UInt32 sleep_ms = 100;
UInt32 retries = 0;
while (true)
{
Coordination::Error code = zookeeper->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral);
if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS)
{
retries++;
if (retries > retry_count)
{
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Can't acquire zookeeper lock");
}
sleepForMilliseconds(sleep_ms);
}
else if (code != Coordination::Error::ZOK)
{
throw Coordination::Exception(code, zookeeper_lock_path);
}
else
{
return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zookeeper);
}
}
}
}
#endif

View File

@ -0,0 +1,124 @@
#pragma once
#if USE_AWS_S3
# include <Core/UUID.h>
# include <Interpreters/Context.h>
# include <Storages/StorageS3Settings.h>
# include <Common/ZooKeeper/ZooKeeper.h>
namespace DB
{
class StorageS3Queue;
struct S3QueueSettings;
class S3QueueFilesMetadata
{
public:
struct TrackedCollectionItem
{
String file_path;
UInt64 timestamp = 0;
UInt64 retries_count = 0;
String last_exception;
};
using S3FilesCollection = std::unordered_set<String>;
using TrackedFiles = std::deque<TrackedCollectionItem>;
S3QueueFilesMetadata(const StorageS3Queue * storage_, const S3QueueSettings & settings_);
void setFilesProcessing(const Strings & file_paths);
void setFileProcessed(const String & file_path);
bool setFileFailed(const String & file_path, const String & exception_message);
S3FilesCollection getProcessedFailedAndProcessingFiles();
String getMaxProcessedFile();
std::shared_ptr<zkutil::EphemeralNodeHolder> acquireLock(zkutil::ZooKeeperPtr zookeeper);
struct S3QueueCollection
{
public:
virtual ~S3QueueCollection() = default;
virtual String toString() const;
S3FilesCollection getFileNames();
virtual void parse(const String & collection_str) = 0;
protected:
TrackedFiles files;
void read(ReadBuffer & in);
void write(WriteBuffer & out) const;
};
struct S3QueueProcessedCollection : public S3QueueCollection
{
public:
S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_);
void parse(const String & collection_str) override;
void add(const String & file_name);
private:
const UInt64 max_size;
const UInt64 max_age;
};
struct S3QueueFailedCollection : S3QueueCollection
{
public:
S3QueueFailedCollection(const UInt64 & max_retries_count_);
void parse(const String & collection_str) override;
bool add(const String & file_name, const String & exception_message);
S3FilesCollection getFileNames();
private:
UInt64 max_retries_count;
};
struct S3QueueProcessingCollection
{
public:
S3QueueProcessingCollection() = default;
void parse(const String & collection_str);
void add(const Strings & file_names);
void remove(const String & file_name);
String toString() const;
const S3FilesCollection & getFileNames() const { return files; }
private:
S3FilesCollection files;
};
private:
const StorageS3Queue * storage;
const S3QueueMode mode;
const UInt64 max_set_size;
const UInt64 max_set_age_sec;
const UInt64 max_loading_retries;
const String zookeeper_processing_path;
const String zookeeper_processed_path;
const String zookeeper_failed_path;
const String zookeeper_lock_path;
mutable std::mutex mutex;
Poco::Logger * log;
S3FilesCollection getFailedFiles();
S3FilesCollection getProcessingFiles();
S3FilesCollection getUnorderedProcessedFiles();
void removeProcessingFile(const String & file_path);
};
}
#endif

View File

@ -0,0 +1,41 @@
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSetQuery.h>
#include <Storages/S3Queue/S3QueueSettings.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_SETTING;
}
IMPLEMENT_SETTINGS_TRAITS(S3QueueSettingsTraits, LIST_OF_S3QUEUE_SETTINGS)
void S3QueueSettings::loadFromQuery(ASTStorage & storage_def)
{
if (storage_def.settings)
{
try
{
applyChanges(storage_def.settings->changes);
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::UNKNOWN_SETTING)
e.addMessage("for storage " + storage_def.engine->name);
throw;
}
}
else
{
auto settings_ast = std::make_shared<ASTSetQuery>();
settings_ast->is_standalone = false;
storage_def.set(storage_def.settings, settings_ast);
}
}
}

View File

@ -0,0 +1,46 @@
#pragma once
#include <Core/BaseSettings.h>
#include <Core/Settings.h>
#include <Core/SettingsEnums.h>
namespace DB
{
class ASTStorage;
#define S3QUEUE_RELATED_SETTINGS(M, ALIAS) \
M(S3QueueMode, \
mode, \
S3QueueMode::ORDERED, \
"With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer." \
"With ordered mode, only the max name of the successfully consumed file stored.", \
0) \
M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete or keep file in S3 after successful processing", 0) \
M(String, keeper_path, "", "Zookeeper node path", 0) \
M(UInt64, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \
M(UInt64, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \
M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \
M(UInt64, s3queue_polling_backoff_ms, 0, "Polling backoff", 0) \
M(UInt64, s3queue_tracked_files_limit, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \
M(UInt64, \
s3queue_tracked_file_ttl_sec, \
0, \
"Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", \
0) \
M(UInt64, s3queue_polling_size, 50, "Maximum files to fetch from S3 with SELECT", 0)
#define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \
S3QUEUE_RELATED_SETTINGS(M, ALIAS) \
FORMAT_FACTORY_SETTINGS(M, ALIAS)
DECLARE_SETTINGS_TRAITS(S3QueueSettingsTraits, LIST_OF_S3QUEUE_SETTINGS)
struct S3QueueSettings : public BaseSettings<S3QueueSettingsTraits>
{
void loadFromQuery(ASTStorage & storage_def);
};
}

View File

@ -0,0 +1,321 @@
#include <algorithm>
#include <Common/ProfileEvents.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include "IO/ParallelReadBuffer.h"
#include "Parsers/ASTCreateQuery.h"
#include "config.h"
#if USE_AWS_S3
# include <Common/isValidUTF8.h>
# include <Functions/FunctionsConversion.h>
# include <IO/S3/Requests.h>
# include <IO/S3Common.h>
# include <Interpreters/TreeRewriter.h>
# include <Parsers/ASTFunction.h>
# include <Parsers/ASTInsertQuery.h>
# include <Storages/NamedCollectionsHelpers.h>
# include <Storages/PartitionedSink.h>
# include <Storages/S3Queue/S3QueueSource.h>
# include <Storages/StorageS3.h>
# include <Storages/StorageS3Settings.h>
# include <Storages/VirtualColumnUtils.h>
# include <Storages/getVirtualsForStorage.h>
# include <Formats/FormatFactory.h>
# include <Processors/Formats/IInputFormat.h>
# include <Processors/Formats/IOutputFormat.h>
# include <Processors/Transforms/AddingDefaultsTransform.h>
# include <QueryPipeline/QueryPipelineBuilder.h>
# include <DataTypes/DataTypeString.h>
# include <Common/CurrentMetrics.h>
# include <Common/NamedCollections/NamedCollections.h>
# include <Common/parseGlobs.h>
# include <Processors/ISource.h>
# include <Processors/Sinks/SinkToStorage.h>
namespace CurrentMetrics
{
extern const Metric StorageS3Threads;
extern const Metric StorageS3ThreadsActive;
}
namespace ProfileEvents
{
extern const Event S3DeleteObjects;
extern const Event S3ListObjects;
}
namespace DB
{
namespace ErrorCodes
{
extern const int S3_ERROR;
}
StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator(
const S3::Client & client_,
const S3::URI & globbed_uri_,
ASTPtr query,
const Block & virtual_header,
ContextPtr context,
UInt64 & max_poll_size_,
const S3Settings::RequestSettings & request_settings_)
: max_poll_size(max_poll_size_)
, glob_iterator(std::make_unique<StorageS3QueueSource::DisclosedGlobIterator>(
client_, globbed_uri_, query, virtual_header, context, nullptr, request_settings_))
{
/// todo(kssenii): remove this loop, it should not be here
while (true)
{
KeyWithInfo val = glob_iterator->next();
if (val.key.empty())
break;
keys_buf.push_back(val);
}
}
Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles(
const S3QueueMode & engine_mode, std::unordered_set<String> & exclude_keys, const String & max_file)
{
for (const KeyWithInfo & val : keys_buf)
{
auto full_path = val.key;
if (exclude_keys.find(full_path) != exclude_keys.end())
{
LOG_TEST(log, "File {} will be skipped, because it was found in exclude files list "
"(either already processed or failed to be processed)", val.key);
continue;
}
if ((engine_mode == S3QueueMode::ORDERED) && (full_path.compare(max_file) <= 0))
continue;
if ((processing_keys.size() < max_poll_size) || (engine_mode == S3QueueMode::ORDERED))
{
processing_keys.push_back(val);
}
else
{
break;
}
}
if (engine_mode == S3QueueMode::ORDERED)
{
std::sort(
processing_keys.begin(),
processing_keys.end(),
[](const KeyWithInfo & lhs, const KeyWithInfo & rhs) { return lhs.key.compare(rhs.key) < 0; });
if (processing_keys.size() > max_poll_size)
{
processing_keys.erase(processing_keys.begin() + max_poll_size, processing_keys.end());
}
}
Strings keys;
for (const auto & key_info : processing_keys)
keys.push_back(key_info.key);
processing_keys.push_back(KeyWithInfo());
processing_iterator = processing_keys.begin();
return keys;
}
StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next()
{
std::lock_guard lock(mutex);
if (processing_iterator != processing_keys.end())
{
return *processing_iterator++;
}
return KeyWithInfo();
}
Block StorageS3QueueSource::getHeader(Block sample_block, const std::vector<NameAndTypePair> & requested_virtual_columns)
{
for (const auto & virtual_column : requested_virtual_columns)
sample_block.insert({virtual_column.type->createColumn(), virtual_column.type, virtual_column.name});
return sample_block;
}
StorageS3QueueSource::StorageS3QueueSource(
const std::vector<NameAndTypePair> & requested_virtual_columns_,
const String & format_,
String name_,
const Block & sample_block_,
ContextPtr context_,
std::optional<FormatSettings> format_settings_,
const ColumnsDescription & columns_,
UInt64 max_block_size_,
const S3Settings::RequestSettings & request_settings_,
String compression_hint_,
const std::shared_ptr<const S3::Client> & client_,
const String & bucket_,
const String & version_id_,
std::shared_ptr<IIterator> file_iterator_,
std::shared_ptr<S3QueueFilesMetadata> files_metadata_,
const S3QueueAction & action_,
const size_t download_thread_num_)
: ISource(getHeader(sample_block_, requested_virtual_columns_))
, WithContext(context_)
, name(std::move(name_))
, bucket(bucket_)
, version_id(version_id_)
, format(format_)
, columns_desc(columns_)
, request_settings(request_settings_)
, client(client_)
, files_metadata(files_metadata_)
, requested_virtual_columns(requested_virtual_columns_)
, file_iterator(file_iterator_)
, action(action_)
{
internal_source = std::make_shared<StorageS3Source>(
requested_virtual_columns_,
format_,
name_,
sample_block_,
context_,
format_settings_,
columns_,
max_block_size_,
request_settings_,
compression_hint_,
client_,
bucket_,
version_id_,
file_iterator,
download_thread_num_);
reader = std::move(internal_source->reader);
if (reader)
reader_future = std::move(internal_source->reader_future);
}
StorageS3QueueSource::~StorageS3QueueSource()
{
internal_source->create_reader_pool.wait();
}
String StorageS3QueueSource::getName() const
{
return name;
}
Chunk StorageS3QueueSource::generate()
{
auto file_progress = getContext()->getFileProgressCallback();
while (true)
{
if (isCancelled() || !reader)
{
if (reader)
reader->cancel();
break;
}
Chunk chunk;
bool success_in_pulling = false;
try
{
if (reader->pull(chunk))
{
UInt64 num_rows = chunk.getNumRows();
auto file_path = reader.getPath();
for (const auto & virtual_column : requested_virtual_columns)
{
if (virtual_column.name == "_path")
{
chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_path)->convertToFullColumnIfConst());
}
else if (virtual_column.name == "_file")
{
size_t last_slash_pos = file_path.find_last_of('/');
auto column = virtual_column.type->createColumnConst(num_rows, file_path.substr(last_slash_pos + 1));
chunk.addColumn(column->convertToFullColumnIfConst());
}
}
success_in_pulling = true;
}
}
catch (const Exception & e)
{
LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText());
files_metadata->setFileFailed(reader.getFile(), e.message());
success_in_pulling = false;
}
if (success_in_pulling)
{
applyActionAfterProcessing(reader.getFile());
files_metadata->setFileProcessed(reader.getFile());
return chunk;
}
assert(reader_future.valid());
reader = reader_future.get();
if (!reader)
break;
/// Even if task is finished the thread may be not freed in pool.
/// So wait until it will be freed before scheduling a new task.
internal_source->create_reader_pool.wait();
reader_future = internal_source->createReaderAsync();
}
return {};
}
void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path)
{
switch (action)
{
case S3QueueAction::DELETE:
deleteProcessedObject(file_path);
break;
case S3QueueAction::KEEP:
break;
}
}
void StorageS3QueueSource::deleteProcessedObject(const String & file_path)
{
LOG_INFO(log, "Delete processed file {} from bucket {}", file_path, bucket);
S3::DeleteObjectRequest request;
request.WithKey(file_path).WithBucket(bucket);
auto outcome = client->DeleteObject(request);
if (!outcome.IsSuccess())
{
const auto & err = outcome.GetError();
LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast<size_t>(err.GetErrorType()));
}
else
{
LOG_TRACE(log, "Object with path {} was removed from S3", file_path);
}
}
}
#endif

View File

@ -0,0 +1,124 @@
#pragma once
#include "config.h"
#if USE_AWS_S3
# include <Core/Types.h>
# include <Compression/CompressionInfo.h>
# include <Storages/IStorage.h>
# include <Storages/S3Queue/S3QueueFilesMetadata.h>
# include <Storages/StorageS3.h>
# include <Storages/StorageS3Settings.h>
# include <IO/CompressionMethod.h>
# include <IO/S3/getObjectInfo.h>
# include <Interpreters/Context.h>
# include <Interpreters/threadPoolCallbackRunner.h>
# include <Processors/Executors/PullingPipelineExecutor.h>
# include <Processors/ISource.h>
# include <Storages/Cache/SchemaCache.h>
# include <Storages/StorageConfiguration.h>
# include <Poco/URI.h>
# include <Common/ZooKeeper/ZooKeeper.h>
# include <Common/logger_useful.h>
namespace DB
{
class StorageS3QueueSource : public ISource, WithContext
{
public:
using IIterator = StorageS3Source::IIterator;
using DisclosedGlobIterator = StorageS3Source::DisclosedGlobIterator;
using KeysWithInfo = StorageS3Source::KeysWithInfo;
using KeyWithInfo = StorageS3Source::KeyWithInfo;
class QueueGlobIterator : public IIterator
{
public:
QueueGlobIterator(
const S3::Client & client_,
const S3::URI & globbed_uri_,
ASTPtr query,
const Block & virtual_header,
ContextPtr context,
UInt64 & max_poll_size_,
const S3Settings::RequestSettings & request_settings_ = {});
KeyWithInfo next() override;
Strings
filterProcessingFiles(const S3QueueMode & engine_mode, std::unordered_set<String> & exclude_keys, const String & max_file = "");
private:
UInt64 max_poll_size;
KeysWithInfo keys_buf;
KeysWithInfo processing_keys;
mutable std::mutex mutex;
std::unique_ptr<DisclosedGlobIterator> glob_iterator;
std::vector<KeyWithInfo>::iterator processing_iterator;
Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSourceIterator");
};
static Block getHeader(Block sample_block, const std::vector<NameAndTypePair> & requested_virtual_columns);
StorageS3QueueSource(
const std::vector<NameAndTypePair> & requested_virtual_columns_,
const String & format,
String name_,
const Block & sample_block,
ContextPtr context_,
std::optional<FormatSettings> format_settings_,
const ColumnsDescription & columns_,
UInt64 max_block_size_,
const S3Settings::RequestSettings & request_settings_,
String compression_hint_,
const std::shared_ptr<const S3::Client> & client_,
const String & bucket,
const String & version_id,
std::shared_ptr<IIterator> file_iterator_,
std::shared_ptr<S3QueueFilesMetadata> files_metadata_,
const S3QueueAction & action_,
size_t download_thread_num);
~StorageS3QueueSource() override;
String getName() const override;
Chunk generate() override;
private:
String name;
String bucket;
String version_id;
String format;
ColumnsDescription columns_desc;
S3Settings::RequestSettings request_settings;
std::shared_ptr<const S3::Client> client;
std::shared_ptr<S3QueueFilesMetadata> files_metadata;
using ReaderHolder = StorageS3Source::ReaderHolder;
ReaderHolder reader;
std::vector<NameAndTypePair> requested_virtual_columns;
std::shared_ptr<IIterator> file_iterator;
const S3QueueAction action;
Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSource");
std::future<ReaderHolder> reader_future;
mutable std::mutex mutex;
std::shared_ptr<StorageS3Source> internal_source;
void deleteProcessedObject(const String & file_path);
void applyActionAfterProcessing(const String & file_path);
};
}
#endif

View File

@ -0,0 +1,115 @@
#include <config.h>
#if USE_AWS_S3
# include <Poco/JSON/JSON.h>
# include <Poco/JSON/Object.h>
# include <Poco/JSON/Parser.h>
# include <Storages/S3Queue/S3QueueSettings.h>
# include <Storages/S3Queue/S3QueueTableMetadata.h>
# include <Storages/StorageS3.h>
namespace DB
{
namespace ErrorCodes
{
extern const int METADATA_MISMATCH;
}
S3QueueTableMetadata::S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings)
{
format_name = configuration.format;
after_processing = engine_settings.after_processing.toString();
mode = engine_settings.mode.toString();
s3queue_tracked_files_limit = engine_settings.s3queue_tracked_files_limit;
s3queue_tracked_file_ttl_sec = engine_settings.s3queue_tracked_file_ttl_sec;
}
String S3QueueTableMetadata::toString() const
{
Poco::JSON::Object json;
json.set("after_processing", after_processing);
json.set("mode", mode);
json.set("s3queue_tracked_files_limit", s3queue_tracked_files_limit);
json.set("s3queue_tracked_file_ttl_sec", s3queue_tracked_file_ttl_sec);
json.set("format_name", format_name);
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
oss.exceptions(std::ios::failbit);
Poco::JSON::Stringifier::stringify(json, oss);
return oss.str();
}
void S3QueueTableMetadata::read(const String & metadata_str)
{
Poco::JSON::Parser parser;
auto json = parser.parse(metadata_str).extract<Poco::JSON::Object::Ptr>();
after_processing = json->getValue<String>("after_processing");
mode = json->getValue<String>("mode");
s3queue_tracked_files_limit = json->getValue<UInt64>("s3queue_tracked_files_limit");
s3queue_tracked_file_ttl_sec = json->getValue<UInt64>("s3queue_tracked_file_ttl_sec");
format_name = json->getValue<String>("format_name");
}
S3QueueTableMetadata S3QueueTableMetadata::parse(const String & metadata_str)
{
S3QueueTableMetadata metadata;
metadata.read(metadata_str);
return metadata;
}
void S3QueueTableMetadata::checkImmutableFieldsEquals(const S3QueueTableMetadata & from_zk) const
{
if (after_processing != from_zk.after_processing)
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs "
"in action after processing. Stored in ZooKeeper: {}, local: {}",
DB::toString(from_zk.after_processing),
DB::toString(after_processing));
if (mode != from_zk.mode)
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in engine mode. "
"Stored in ZooKeeper: {}, local: {}",
DB::toString(from_zk.after_processing),
DB::toString(after_processing));
if (s3queue_tracked_files_limit != from_zk.s3queue_tracked_files_limit)
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in max set size. "
"Stored in ZooKeeper: {}, local: {}",
from_zk.s3queue_tracked_files_limit,
s3queue_tracked_files_limit);
if (s3queue_tracked_file_ttl_sec != from_zk.s3queue_tracked_file_ttl_sec)
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in max set age. "
"Stored in ZooKeeper: {}, local: {}",
from_zk.s3queue_tracked_file_ttl_sec,
s3queue_tracked_file_ttl_sec);
if (format_name != from_zk.format_name)
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in format name. "
"Stored in ZooKeeper: {}, local: {}",
from_zk.format_name,
format_name);
}
void S3QueueTableMetadata::checkEquals(const S3QueueTableMetadata & from_zk) const
{
checkImmutableFieldsEquals(from_zk);
}
}
#endif

View File

@ -0,0 +1,43 @@
#pragma once
#if USE_AWS_S3
# include <Storages/S3Queue/S3QueueSettings.h>
# include <Storages/StorageS3.h>
# include <base/types.h>
namespace DB
{
class WriteBuffer;
class ReadBuffer;
/** The basic parameters of S3Queue table engine for saving in ZooKeeper.
* Lets you verify that they match local ones.
*/
struct S3QueueTableMetadata
{
String format_name;
String after_processing;
String mode;
UInt64 s3queue_tracked_files_limit;
UInt64 s3queue_tracked_file_ttl_sec;
S3QueueTableMetadata() = default;
S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings);
void read(const String & metadata_str);
static S3QueueTableMetadata parse(const String & metadata_str);
String toString() const;
void checkEquals(const S3QueueTableMetadata & from_zk) const;
private:
void checkImmutableFieldsEquals(const S3QueueTableMetadata & from_zk) const;
};
}
#endif

View File

@ -0,0 +1,711 @@
#include "config.h"
#if USE_AWS_S3
# include <Databases/DatabaseReplicated.h>
# include <IO/WriteBuffer.h>
# include <IO/WriteHelpers.h>
# include <Interpreters/InterpreterInsertQuery.h>
# include <Processors/Executors/CompletedPipelineExecutor.h>
# include <Common/ProfileEvents.h>
# include <Common/ZooKeeper/ZooKeeper.h>
# include <Common/isValidUTF8.h>
# include "IO/ParallelReadBuffer.h"
# include <Functions/FunctionsConversion.h>
# include <IO/S3Common.h>
# include <Interpreters/TreeRewriter.h>
# include <Parsers/ASTFunction.h>
# include <Parsers/ASTInsertQuery.h>
# include <Storages/NamedCollectionsHelpers.h>
# include <Storages/PartitionedSink.h>
# include <Storages/S3Queue/S3QueueSource.h>
# include <Storages/S3Queue/S3QueueTableMetadata.h>
# include <Storages/S3Queue/StorageS3Queue.h>
# include <Storages/StorageFactory.h>
# include <Storages/StorageMaterializedView.h>
# include <Storages/StorageS3.h>
# include <Storages/StorageSnapshot.h>
# include <Storages/VirtualColumnUtils.h>
# include <Storages/getVirtualsForStorage.h>
# include <Common/NamedCollections/NamedCollections.h>
# include <Formats/FormatFactory.h>
# include <Processors/Formats/IInputFormat.h>
# include <Processors/Formats/IOutputFormat.h>
# include <Processors/Transforms/AddingDefaultsTransform.h>
# include <QueryPipeline/QueryPipelineBuilder.h>
# include <DataTypes/DataTypeString.h>
# include <Common/parseGlobs.h>
# include <filesystem>
# include <Processors/ISource.h>
# include <Processors/Sinks/SinkToStorage.h>
# include <QueryPipeline/Pipe.h>
namespace fs = std::filesystem;
namespace ProfileEvents
{
extern const Event S3DeleteObjects;
extern const Event S3ListObjects;
}
namespace DB
{
static const String PARTITION_ID_WILDCARD = "{_partition_id}";
static const auto MAX_THREAD_WORK_DURATION_MS = 60000;
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
extern const int S3_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int QUERY_NOT_ALLOWED;
extern const int NO_ZOOKEEPER;
extern const int REPLICA_ALREADY_EXISTS;
extern const int INCOMPATIBLE_COLUMNS;
}
StorageS3Queue::StorageS3Queue(
std::unique_ptr<S3QueueSettings> s3queue_settings_,
const StorageS3::Configuration & configuration_,
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_,
std::optional<FormatSettings> format_settings_,
ASTPtr partition_by_)
: IStorage(table_id_)
, WithContext(context_)
, s3queue_settings(std::move(s3queue_settings_))
, after_processing(s3queue_settings->after_processing)
, configuration{configuration_}
, reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms)
, format_settings(format_settings_)
, partition_by(partition_by_)
, log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")"))
{
if (configuration.url.key.ends_with('/'))
configuration.url.key += '*';
if (!withGlobs())
throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs");
String setting_zk_path = s3queue_settings->keeper_path;
if (setting_zk_path.empty())
{
auto database = DatabaseCatalog::instance().getDatabase(table_id_.database_name);
bool is_in_replicated_database = database->getEngineName() == "Replicated";
auto default_path = getContext()->getSettingsRef().s3queue_default_zookeeper_path.value;
String zk_path_prefix;
if (!default_path.empty())
{
zk_path_prefix = default_path;
}
else if (is_in_replicated_database)
{
LOG_INFO(log, "S3Queue engine zookeeper path is not specified. "
"Using replicated database zookeeper path");
zk_path_prefix = fs::path(assert_cast<const DatabaseReplicated *>(database.get())->getZooKeeperPath()) / "s3queue";
}
else
{
throw Exception(ErrorCodes::NO_ZOOKEEPER,
"S3Queue keeper_path engine setting not specified, "
"s3queue_default_zookeeper_path_prefix not specified");
}
zk_path = zkutil::extractZooKeeperPath(
fs::path(zk_path_prefix) / toString(table_id_.uuid), /* check_starts_with_slash */ true, log);
}
else
{
/// We do not add table uuid here on purpose.
zk_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path.value, /* check_starts_with_slash */ true, log);
}
LOG_INFO(log, "Using zookeeper path: {}", zk_path);
FormatFactory::instance().checkFormatName(configuration.format);
context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri);
StorageInMemoryMetadata storage_metadata;
configuration.update(context_);
if (columns_.empty())
{
auto columns = StorageS3::getTableStructureFromDataImpl(configuration, format_settings, context_);
storage_metadata.setColumns(columns);
}
else
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
auto metadata_snapshot = getInMemoryMetadataPtr();
const bool is_first_replica = createTableIfNotExists(metadata_snapshot);
if (!is_first_replica)
{
checkTableStructure(zk_path, metadata_snapshot);
}
files_metadata = std::make_shared<S3QueueFilesMetadata>(this, *s3queue_settings);
auto default_virtuals = NamesAndTypesList{
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
auto columns = storage_metadata.getSampleBlock().getNamesAndTypesList();
virtual_columns = getVirtualsForStorage(columns, default_virtuals);
for (const auto & column : virtual_columns)
virtual_block.insert({column.type->createColumn(), column.type, column.name});
auto poll_thread = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); });
task = std::make_shared<TaskContext>(std::move(poll_thread));
}
bool StorageS3Queue::supportsSubcolumns() const
{
return FormatFactory::instance().checkIfFormatSupportsSubcolumns(configuration.format);
}
bool StorageS3Queue::supportsSubsetOfColumns() const
{
return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format);
}
Pipe StorageS3Queue::read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr local_context,
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size,
size_t /* num_streams */)
{
if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select)
throw Exception(
ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`");
if (mv_attached)
throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageS3Queue with attached materialized views");
auto query_configuration = updateConfigurationAndGetCopy(local_context);
Pipes pipes;
std::unordered_set<String> column_names_set(column_names.begin(), column_names.end());
std::vector<NameAndTypePair> requested_virtual_columns;
for (const auto & virtual_column : getVirtuals())
{
if (column_names_set.contains(virtual_column.name))
requested_virtual_columns.push_back(virtual_column);
}
std::shared_ptr<StorageS3Source::IIterator> iterator_wrapper = createFileIterator(local_context, query_info.query);
ColumnsDescription columns_description;
Block block_for_format;
if (supportsSubsetOfColumns())
{
auto fetch_columns = column_names;
const auto & virtuals = getVirtuals();
std::erase_if(
fetch_columns,
[&](const String & col)
{
return std::any_of(
virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; });
});
if (fetch_columns.empty())
fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name);
columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns);
block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical());
}
else
{
columns_description = storage_snapshot->metadata->getColumns();
block_for_format = storage_snapshot->metadata->getSampleBlock();
}
const size_t max_download_threads = local_context->getSettingsRef().max_download_threads;
return Pipe(std::make_shared<StorageS3QueueSource>(
requested_virtual_columns,
configuration.format,
getName(),
block_for_format,
local_context,
format_settings,
columns_description,
max_block_size,
query_configuration.request_settings,
configuration.compression_method,
query_configuration.client,
query_configuration.url.bucket,
query_configuration.url.version_id,
iterator_wrapper,
files_metadata,
after_processing,
max_download_threads));
}
SinkToStoragePtr StorageS3Queue::write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, bool)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName());
}
void StorageS3Queue::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName());
}
NamesAndTypesList StorageS3Queue::getVirtuals() const
{
return virtual_columns;
}
bool StorageS3Queue::supportsPartitionBy() const
{
return true;
}
void StorageS3Queue::startup()
{
if (task)
task->holder->activateAndSchedule();
}
void StorageS3Queue::shutdown()
{
shutdown_called = true;
if (task)
{
task->stream_cancelled = true;
task->holder->deactivate();
}
}
size_t StorageS3Queue::getTableDependentCount() const
{
auto table_id = getStorageID();
// Check if at least one direct dependency is attached
return DatabaseCatalog::instance().getDependentViews(table_id).size();
}
bool StorageS3Queue::hasDependencies(const StorageID & table_id)
{
// Check if all dependencies are attached
auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id);
LOG_TEST(log, "Number of attached views {} for {}", view_ids.size(), table_id.getNameForLogs());
if (view_ids.empty())
return false;
// Check the dependencies are ready?
for (const auto & view_id : view_ids)
{
auto view = DatabaseCatalog::instance().tryGetTable(view_id, getContext());
if (!view)
return false;
// If it materialized view, check it's target table
auto * materialized_view = dynamic_cast<StorageMaterializedView *>(view.get());
if (materialized_view && !materialized_view->tryGetTargetTable())
return false;
}
return true;
}
void StorageS3Queue::threadFunc()
{
bool reschedule = true;
try
{
auto table_id = getStorageID();
auto dependencies_count = getTableDependentCount();
if (dependencies_count)
{
auto start_time = std::chrono::steady_clock::now();
mv_attached.store(true);
// Keep streaming as long as there are attached views and streaming is not cancelled
while (!task->stream_cancelled)
{
if (!hasDependencies(table_id))
{
/// For this case, we can not wait for watch thread to wake up
reschedule = true;
break;
}
LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count);
streamToViews();
auto ts = std::chrono::steady_clock::now();
auto duration = std::chrono::duration_cast<std::chrono::milliseconds>(ts - start_time);
if (duration.count() > MAX_THREAD_WORK_DURATION_MS)
{
LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule.");
reschedule = true;
break;
}
reschedule_processing_interval_ms = s3queue_settings->s3queue_polling_min_timeout_ms;
}
}
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
mv_attached.store(false);
if (reschedule && !shutdown_called)
{
LOG_TRACE(log, "Reschedule S3 Queue thread func.");
/// Reschedule with backoff.
if (reschedule_processing_interval_ms < s3queue_settings->s3queue_polling_max_timeout_ms)
reschedule_processing_interval_ms += s3queue_settings->s3queue_polling_backoff_ms;
task->holder->scheduleAfter(reschedule_processing_interval_ms);
}
}
void StorageS3Queue::streamToViews()
{
auto table_id = getStorageID();
auto table = DatabaseCatalog::instance().getTable(table_id, getContext());
if (!table)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Engine table {} doesn't exist.", table_id.getNameForLogs());
auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext());
// Create an INSERT query for streaming data
auto insert = std::make_shared<ASTInsertQuery>();
insert->table_id = table_id;
size_t block_size = 100;
auto s3queue_context = Context::createCopy(getContext());
s3queue_context->makeQueryContext();
auto query_configuration = updateConfigurationAndGetCopy(s3queue_context);
// Create a stream for each consumer and join them in a union stream
// Only insert into dependent views and expect that input blocks contain virtual columns
InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true);
auto block_io = interpreter.execute();
auto column_names = block_io.pipeline.getHeader().getNames();
// Create a stream for each consumer and join them in a union stream
std::vector<NameAndTypePair> requested_virtual_columns;
for (const auto & virtual_column : getVirtuals())
{
requested_virtual_columns.push_back(virtual_column);
}
std::shared_ptr<StorageS3Source::IIterator> iterator_wrapper = createFileIterator(s3queue_context, nullptr);
ColumnsDescription columns_description;
Block block_for_format;
if (supportsSubsetOfColumns())
{
auto fetch_columns = column_names;
const auto & virtuals = getVirtuals();
std::erase_if(
fetch_columns,
[&](const String & col)
{
return std::any_of(
virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; });
});
if (fetch_columns.empty())
fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name);
columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns);
block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical());
}
else
{
columns_description = storage_snapshot->metadata->getColumns();
block_for_format = storage_snapshot->metadata->getSampleBlock();
}
const size_t max_download_threads = s3queue_context->getSettingsRef().max_download_threads;
Pipes pipes;
auto pipe = Pipe(std::make_shared<StorageS3QueueSource>(
requested_virtual_columns,
configuration.format,
getName(),
block_for_format,
s3queue_context,
format_settings,
columns_description,
block_size,
query_configuration.request_settings,
configuration.compression_method,
query_configuration.client,
query_configuration.url.bucket,
query_configuration.url.version_id,
iterator_wrapper,
files_metadata,
after_processing,
max_download_threads));
std::atomic_size_t rows = 0;
{
block_io.pipeline.complete(std::move(pipe));
block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); });
CompletedPipelineExecutor executor(block_io.pipeline);
executor.execute();
}
}
StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context)
{
configuration.update(local_context);
return configuration;
}
zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const
{
std::lock_guard lock{zk_mutex};
if (!zk_client || zk_client->expired())
{
zk_client = getContext()->getZooKeeper();
zk_client->sync(zk_path);
}
return zk_client;
}
bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot)
{
auto zookeeper = getZooKeeper();
zookeeper->createAncestors(zk_path);
for (size_t i = 0; i < zk_create_table_retries; ++i)
{
Coordination::Requests ops;
bool is_first_replica = true;
if (zookeeper->exists(zk_path + "/metadata"))
{
if (!zookeeper->exists(zk_path + "/processing"))
ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processing", "", zkutil::CreateMode::Ephemeral));
LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zk_path);
is_first_replica = false;
}
else
{
String metadata_str = S3QueueTableMetadata(configuration, *s3queue_settings).toString();
ops.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processed", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/failed", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processing", "", zkutil::CreateMode::Ephemeral));
ops.emplace_back(zkutil::makeCreateRequest(
zk_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/metadata", metadata_str, zkutil::CreateMode::Persistent));
}
Coordination::Responses responses;
auto code = zookeeper->tryMulti(ops, responses);
if (code == Coordination::Error::ZNODEEXISTS)
{
LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zk_path);
continue;
}
else if (code != Coordination::Error::ZOK)
{
zkutil::KeeperMultiException::check(code, ops, responses);
}
return is_first_replica;
}
throw Exception(
ErrorCodes::REPLICA_ALREADY_EXISTS,
"Cannot create table, because it is created concurrently every time or because "
"of wrong zk_path or because of logical error");
}
/** Verify that list of columns and table settings match those specified in ZK (/metadata).
* If not, throw an exception.
*/
void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot)
{
auto zookeeper = getZooKeeper();
S3QueueTableMetadata old_metadata(configuration, *s3queue_settings);
Coordination::Stat metadata_stat;
String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata", &metadata_stat);
auto metadata_from_zk = S3QueueTableMetadata::parse(metadata_str);
old_metadata.checkEquals(metadata_from_zk);
Coordination::Stat columns_stat;
auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns", &columns_stat));
const ColumnsDescription & old_columns = metadata_snapshot->getColumns();
if (columns_from_zk != old_columns)
{
throw Exception(
ErrorCodes::INCOMPATIBLE_COLUMNS,
"Table columns structure in ZooKeeper is different from local table structure. Local columns:\n"
"{}\nZookeeper columns:\n{}",
old_columns.toString(),
columns_from_zk.toString());
}
}
std::shared_ptr<StorageS3QueueSource::IIterator>
StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query)
{
auto it = std::make_shared<StorageS3QueueSource::QueueGlobIterator>(
*configuration.client,
configuration.url,
query,
virtual_block,
local_context,
s3queue_settings->s3queue_polling_size.value,
configuration.request_settings);
auto zookeeper = getZooKeeper();
auto lock = files_metadata->acquireLock(zookeeper);
S3QueueFilesMetadata::S3FilesCollection files_to_skip = files_metadata->getProcessedFailedAndProcessingFiles();
Strings files_to_process;
if (s3queue_settings->mode == S3QueueMode::UNORDERED)
{
files_to_process = it->filterProcessingFiles(s3queue_settings->mode, files_to_skip);
}
else
{
String max_processed_file = files_metadata->getMaxProcessedFile();
files_to_process = it->filterProcessingFiles(s3queue_settings->mode, files_to_skip, max_processed_file);
}
LOG_TEST(log, "Found files to process: {}", fmt::join(files_to_process, ", "));
files_metadata->setFilesProcessing(files_to_process);
return it;
}
void StorageS3Queue::drop()
{
auto zookeeper = getZooKeeper();
if (zookeeper->exists(zk_path))
zookeeper->removeRecursive(zk_path);
}
void registerStorageS3QueueImpl(const String & name, StorageFactory & factory)
{
factory.registerStorage(
name,
[](const StorageFactory::Arguments & args)
{
auto & engine_args = args.engine_args;
if (engine_args.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments");
auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext());
// Use format settings from global server context + settings from
// the SETTINGS clause of the create query. Settings from current
// session and user are ignored.
std::optional<FormatSettings> format_settings;
auto s3queue_settings = std::make_unique<S3QueueSettings>();
if (args.storage_def->settings)
{
s3queue_settings->loadFromQuery(*args.storage_def);
FormatFactorySettings user_format_settings;
// Apply changed settings from global context, but ignore the
// unknown ones, because we only have the format settings here.
const auto & changes = args.getContext()->getSettingsRef().changes();
for (const auto & change : changes)
{
if (user_format_settings.has(change.name))
user_format_settings.set(change.name, change.value);
else
LOG_TRACE(&Poco::Logger::get("StorageS3"), "Remove: {}", change.name);
args.storage_def->settings->changes.removeSetting(change.name);
}
for (const auto & change : args.storage_def->settings->changes)
{
if (user_format_settings.has(change.name))
user_format_settings.applyChange(change);
}
format_settings = getFormatSettings(args.getContext(), user_format_settings);
}
else
{
format_settings = getFormatSettings(args.getContext());
}
ASTPtr partition_by;
if (args.storage_def->partition_by)
partition_by = args.storage_def->partition_by->clone();
return std::make_shared<StorageS3Queue>(
std::move(s3queue_settings),
std::move(configuration),
args.table_id,
args.columns,
args.constraints,
args.comment,
args.getContext(),
format_settings,
partition_by);
},
{
.supports_settings = true,
.supports_sort_order = true, // for partition by
.supports_schema_inference = true,
.source_access_type = AccessType::S3,
});
}
void registerStorageS3Queue(StorageFactory & factory)
{
return registerStorageS3QueueImpl("S3Queue", factory);
}
}
#endif

View File

@ -0,0 +1,146 @@
#pragma once
#include "config.h"
#if USE_AWS_S3
# include <Core/Types.h>
# include <Compression/CompressionInfo.h>
# include <Common/ZooKeeper/ZooKeeper.h>
# include <Core/BackgroundSchedulePool.h>
# include <Storages/IStorage.h>
# include <Storages/S3Queue/S3QueueFilesMetadata.h>
# include <Storages/S3Queue/S3QueueSettings.h>
# include <Storages/S3Queue/S3QueueSource.h>
# include <Storages/StorageS3Settings.h>
# include <IO/CompressionMethod.h>
# include <IO/S3/getObjectInfo.h>
# include <Interpreters/Context.h>
# include <Interpreters/threadPoolCallbackRunner.h>
# include <Processors/Executors/PullingPipelineExecutor.h>
# include <Processors/ISource.h>
# include <Storages/Cache/SchemaCache.h>
# include <Storages/StorageConfiguration.h>
# include <Storages/StorageS3.h>
# include <Poco/URI.h>
# include <Common/logger_useful.h>
namespace Aws::S3
{
class Client;
}
namespace DB
{
class StorageS3Queue : public IStorage, WithContext
{
public:
using Configuration = typename StorageS3::Configuration;
StorageS3Queue(
std::unique_ptr<S3QueueSettings> s3queue_settings_,
const Configuration & configuration_,
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_,
std::optional<FormatSettings> format_settings_,
ASTPtr partition_by_ = nullptr);
String getName() const override { return "S3Queue"; }
Pipe read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams) override;
SinkToStoragePtr write(
const ASTPtr & query,
const StorageMetadataPtr & metadata_snapshot,
ContextPtr context,
bool async_insert) override;
void truncate(
const ASTPtr & /*query*/,
const StorageMetadataPtr & /*metadata_snapshot*/,
ContextPtr /*local_context*/,
TableExclusiveLockHolder &) override;
NamesAndTypesList getVirtuals() const override;
bool supportsPartitionBy() const override;
const auto & getFormatName() const { return configuration.format; }
const String & getZooKeeperPath() const { return zk_path; }
zkutil::ZooKeeperPtr getZooKeeper() const;
private:
const std::unique_ptr<S3QueueSettings> s3queue_settings;
const S3QueueAction after_processing;
std::shared_ptr<S3QueueFilesMetadata> files_metadata;
Configuration configuration;
NamesAndTypesList virtual_columns;
Block virtual_block;
UInt64 reschedule_processing_interval_ms;
std::optional<FormatSettings> format_settings;
ASTPtr partition_by;
String zk_path;
mutable zkutil::ZooKeeperPtr zk_client;
mutable std::mutex zk_mutex;
std::atomic<bool> mv_attached = false;
std::atomic<bool> shutdown_called{false};
Poco::Logger * log;
bool supportsSubcolumns() const override;
bool withGlobs() const { return configuration.url.key.find_first_of("*?{") != std::string::npos; }
void threadFunc();
size_t getTableDependentCount() const;
bool hasDependencies(const StorageID & table_id);
void startup() override;
void shutdown() override;
void drop() override;
struct TaskContext
{
BackgroundSchedulePool::TaskHolder holder;
std::atomic<bool> stream_cancelled{false};
explicit TaskContext(BackgroundSchedulePool::TaskHolder && task_) : holder(std::move(task_)) { }
};
std::shared_ptr<TaskContext> task;
bool supportsSubsetOfColumns() const override;
const UInt32 zk_create_table_retries = 1000;
bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot);
void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot);
using KeysWithInfo = StorageS3QueueSource::KeysWithInfo;
std::shared_ptr<StorageS3QueueSource::IIterator>
createFileIterator(ContextPtr local_context, ASTPtr query);
void streamToViews();
Configuration updateConfigurationAndGetCopy(ContextPtr local_context);
};
}
#endif

View File

@ -1154,16 +1154,25 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate(
}
TransactionID first_mutation_tid = mutations_begin_it->second.tid;
MergeTreeTransactionPtr txn = tryGetTransactionForMutation(mutations_begin_it->second, log);
assert(txn || first_mutation_tid.isPrehistoric());
MergeTreeTransactionPtr txn;
if (txn)
if (!first_mutation_tid.isPrehistoric())
{
/// Mutate visible parts only
/// NOTE Do not mutate visible parts in Outdated state, because it does not make sense:
/// mutation will fail anyway due to serialization error.
if (!part->version.isVisible(*txn))
/// It's possible that both mutation and transaction are already finished,
/// because that part should not be mutated because it was not visible for that transaction.
if (!part->version.isVisible(first_mutation_tid.start_csn, first_mutation_tid))
continue;
txn = tryGetTransactionForMutation(mutations_begin_it->second, log);
if (!txn)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find transaction {} that has started mutation {} "
"that is going to be applied to part {}",
first_mutation_tid, mutations_begin_it->second.file_name, part->name);
}
auto commands = std::make_shared<MutationCommands>();

View File

@ -596,7 +596,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader()
auto pipeline = std::make_unique<QueryPipeline>(QueryPipelineBuilder::getPipeline(std::move(builder)));
auto current_reader = std::make_unique<PullingPipelineExecutor>(*pipeline);
return ReaderHolder{fs::path(bucket) / key_with_info.key, std::move(read_buf), std::move(input_format), std::move(pipeline), std::move(current_reader)};
return ReaderHolder{key_with_info.key, bucket, std::move(read_buf), std::move(input_format), std::move(pipeline), std::move(current_reader)};
}
std::future<StorageS3Source::ReaderHolder> StorageS3Source::createReaderAsync()

View File

@ -141,6 +141,8 @@ public:
Chunk generate() override;
private:
friend class StorageS3QueueSource;
String name;
String bucket;
String version_id;
@ -157,12 +159,14 @@ private:
{
public:
ReaderHolder(
String path_,
String key_,
String bucket_,
std::unique_ptr<ReadBuffer> read_buf_,
std::shared_ptr<IInputFormat> input_format_,
std::unique_ptr<QueryPipeline> pipeline_,
std::unique_ptr<PullingPipelineExecutor> reader_)
: path(std::move(path_))
: key(std::move(key_))
, bucket(std::move(bucket_))
, read_buf(std::move(read_buf_))
, input_format(std::move(input_format_))
, pipeline(std::move(pipeline_))
@ -187,19 +191,22 @@ private:
pipeline = std::move(other.pipeline);
input_format = std::move(other.input_format);
read_buf = std::move(other.read_buf);
path = std::move(other.path);
key = std::move(other.key);
bucket = std::move(other.bucket);
return *this;
}
explicit operator bool() const { return reader != nullptr; }
PullingPipelineExecutor * operator->() { return reader.get(); }
const PullingPipelineExecutor * operator->() const { return reader.get(); }
const String & getPath() const { return path; }
String getPath() const { return fs::path(bucket) / key; }
const String & getFile() const { return key; }
const IInputFormat * getInputFormat() const { return input_format.get(); }
private:
String path;
String key;
String bucket;
std::unique_ptr<ReadBuffer> read_buf;
std::shared_ptr<IInputFormat> input_format;
std::unique_ptr<QueryPipeline> pipeline;
@ -323,6 +330,7 @@ protected:
private:
friend class StorageS3Cluster;
friend class TableFunctionS3Cluster;
friend class StorageS3Queue;
Configuration configuration;
std::mutex configuration_update_mutex;

View File

@ -34,6 +34,8 @@ void registerStorageS3(StorageFactory & factory);
void registerStorageCOS(StorageFactory & factory);
void registerStorageOSS(StorageFactory & factory);
void registerStorageHudi(StorageFactory & factory);
void registerStorageS3Queue(StorageFactory & factory);
#if USE_PARQUET
void registerStorageDeltaLake(StorageFactory & factory);
#endif
@ -133,6 +135,7 @@ void registerStorages()
registerStorageCOS(factory);
registerStorageOSS(factory);
registerStorageHudi(factory);
registerStorageS3Queue(factory);
#if USE_PARQUET
registerStorageDeltaLake(factory);

View File

@ -50,7 +50,7 @@ def _can_export_binaries(build_config: BuildConfig) -> bool:
return False
if build_config["sanitizer"] != "":
return True
if build_config["build_type"] != "":
if build_config["debug_build"]:
return True
return False
@ -71,8 +71,8 @@ def get_packager_cmd(
f"--package-type={package_type} --compiler={comp}"
)
if build_config["build_type"]:
cmd += f" --build-type={build_config['build_type']}"
if build_config["debug_build"]:
cmd += " --debug-build"
if build_config["sanitizer"]:
cmd += f" --sanitizer={build_config['sanitizer']}"
if build_config["tidy"] == "enable":

View File

@ -70,7 +70,7 @@ def get_failed_report(
message = f"{job_name} failed"
build_result = BuildResult(
compiler="unknown",
build_type="unknown",
debug_build=False,
sanitizer="unknown",
status=message,
elapsed_seconds=0,
@ -85,7 +85,7 @@ def process_report(
build_config = build_report["build_config"]
build_result = BuildResult(
compiler=build_config["compiler"],
build_type=build_config["build_type"],
debug_build=build_config["debug_build"],
sanitizer=build_config["sanitizer"],
status="success" if build_report["status"] else "failure",
elapsed_seconds=build_report["elapsed_seconds"],

View File

@ -10,7 +10,7 @@ CI_CONFIG = {
"build_config": {
"package_release": {
"compiler": "clang-16",
"build_type": "",
"debug_build": False,
"sanitizer": "",
"package_type": "deb",
"static_binary_name": "amd64",
@ -21,7 +21,7 @@ CI_CONFIG = {
},
"package_aarch64": {
"compiler": "clang-16-aarch64",
"build_type": "",
"debug_build": False,
"sanitizer": "",
"package_type": "deb",
"static_binary_name": "aarch64",
@ -32,7 +32,7 @@ CI_CONFIG = {
},
"package_asan": {
"compiler": "clang-16",
"build_type": "",
"debug_build": False,
"sanitizer": "address",
"package_type": "deb",
"tidy": "disable",
@ -41,7 +41,7 @@ CI_CONFIG = {
},
"package_ubsan": {
"compiler": "clang-16",
"build_type": "",
"debug_build": False,
"sanitizer": "undefined",
"package_type": "deb",
"tidy": "disable",
@ -50,7 +50,7 @@ CI_CONFIG = {
},
"package_tsan": {
"compiler": "clang-16",
"build_type": "",
"debug_build": False,
"sanitizer": "thread",
"package_type": "deb",
"tidy": "disable",
@ -59,7 +59,7 @@ CI_CONFIG = {
},
"package_msan": {
"compiler": "clang-16",
"build_type": "",
"debug_build": False,
"sanitizer": "memory",
"package_type": "deb",
"tidy": "disable",
@ -68,7 +68,7 @@ CI_CONFIG = {
},
"package_debug": {
"compiler": "clang-16",
"build_type": "debug",
"debug_build": True,
"sanitizer": "",
"package_type": "deb",
"tidy": "disable",
@ -77,7 +77,7 @@ CI_CONFIG = {
},
"binary_release": {
"compiler": "clang-16",
"build_type": "",
"debug_build": False,
"sanitizer": "",
"package_type": "binary",
"tidy": "disable",
@ -86,7 +86,7 @@ CI_CONFIG = {
},
"binary_tidy": {
"compiler": "clang-16",
"build_type": "debug",
"debug_build": True,
"sanitizer": "",
"package_type": "binary",
"static_binary_name": "debug-amd64",
@ -96,7 +96,7 @@ CI_CONFIG = {
},
"binary_darwin": {
"compiler": "clang-16-darwin",
"build_type": "",
"debug_build": False,
"sanitizer": "",
"package_type": "binary",
"static_binary_name": "macos",
@ -106,7 +106,7 @@ CI_CONFIG = {
},
"binary_aarch64": {
"compiler": "clang-16-aarch64",
"build_type": "",
"debug_build": False,
"sanitizer": "",
"package_type": "binary",
"tidy": "disable",
@ -115,7 +115,7 @@ CI_CONFIG = {
},
"binary_aarch64_v80compat": {
"compiler": "clang-16-aarch64-v80compat",
"build_type": "",
"debug_build": False,
"sanitizer": "",
"package_type": "binary",
"static_binary_name": "aarch64v80compat",
@ -125,7 +125,7 @@ CI_CONFIG = {
},
"binary_freebsd": {
"compiler": "clang-16-freebsd",
"build_type": "",
"debug_build": False,
"sanitizer": "",
"package_type": "binary",
"static_binary_name": "freebsd",
@ -135,7 +135,7 @@ CI_CONFIG = {
},
"binary_darwin_aarch64": {
"compiler": "clang-16-darwin-aarch64",
"build_type": "",
"debug_build": False,
"sanitizer": "",
"package_type": "binary",
"static_binary_name": "macos-aarch64",
@ -145,7 +145,7 @@ CI_CONFIG = {
},
"binary_ppc64le": {
"compiler": "clang-16-ppc64le",
"build_type": "",
"debug_build": False,
"sanitizer": "",
"package_type": "binary",
"static_binary_name": "powerpc64le",
@ -155,7 +155,7 @@ CI_CONFIG = {
},
"binary_amd64_compat": {
"compiler": "clang-16-amd64-compat",
"build_type": "",
"debug_build": False,
"sanitizer": "",
"package_type": "binary",
"static_binary_name": "amd64compat",
@ -165,7 +165,7 @@ CI_CONFIG = {
},
"binary_riscv64": {
"compiler": "clang-16-riscv64",
"build_type": "",
"debug_build": False,
"sanitizer": "",
"package_type": "binary",
"static_binary_name": "riscv64",

View File

@ -18,7 +18,7 @@ import docker_server as ds
class TestDockerImageCheck(unittest.TestCase):
docker_images_path = os.path.join(
os.path.dirname(__file__), "tests/docker_images.json"
os.path.dirname(__file__), "tests/docker_images_for_tests.json"
)
def test_get_changed_docker_images(self):

View File

@ -219,3 +219,12 @@ def list_runners(access_token: str) -> RunnerDescriptions:
result.append(desc)
return result
def cached_value_is_valid(updated_at: float, ttl: float) -> bool:
"a common function to identify if cachable value is still valid"
if updated_at == 0:
return False
if time.time() - ttl < updated_at:
return True
return False

View File

@ -8,6 +8,8 @@ import boto3 # type: ignore
import jwt
import requests # type: ignore
from . import cached_value_is_valid
def get_key_and_app_from_aws() -> Tuple[str, int]:
secret_name = "clickhouse_github_secret_key"
@ -68,7 +70,7 @@ def get_access_token_by_key_app(private_key: str, app_id: int) -> str:
@dataclass
class CachedToken:
time: int
time: float
value: str
updating: bool = False
@ -81,12 +83,9 @@ def get_cached_access_token() -> str:
return _cached_token.value
# Indicate that the value is updating now, so the cached value can be
# used. The first setting and close-to-ttl are not counted as update
if _cached_token.time != 0 or time.time() - 590 < _cached_token.time:
_cached_token.updating = True
else:
_cached_token.updating = False
_cached_token.updating = cached_value_is_valid(_cached_token.time, 590)
private_key, app_id = get_key_and_app_from_aws()
_cached_token.time = int(time.time())
_cached_token.time = time.time()
_cached_token.value = get_access_token_by_key_app(private_key, app_id)
_cached_token.updating = False
return _cached_token.value

View File

@ -239,7 +239,7 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes
@dataclass
class BuildResult:
compiler: str
build_type: str
debug_build: bool
sanitizer: str
status: str
elapsed_seconds: int
@ -484,8 +484,8 @@ def create_build_html_report(
):
row = "<tr>"
row += f"<td>{build_result.compiler}</td>"
if build_result.build_type:
row += f"<td>{build_result.build_type}</td>"
if build_result.debug_build:
row += "<td>debug</td>"
else:
row += "<td>relwithdebuginfo</td>"
if build_result.sanitizer:

View File

@ -125,6 +125,8 @@ def prepare_for_hung_check(drop_databases):
# However, it obstruct checking for hung queries.
logging.info("Will terminate gdb (if any)")
call_with_retry("kill -TERM $(pidof gdb)")
# Sometimes there is a message `Child process was stopped by signal 19` in logs after stopping gdb
call_with_retry("kill -CONT $(lsof -ti:9000)")
# ThreadFuzzer significantly slows down server and causes false-positive hung check failures
call_with_retry("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'")

View File

@ -9,13 +9,13 @@ from typing import Any, Dict, List
import boto3 # type: ignore
from lambda_shared import RunnerDescriptions, list_runners
from lambda_shared import RunnerDescriptions, list_runners, cached_value_is_valid
from lambda_shared.token import get_access_token_by_key_app, get_cached_access_token
@dataclass
class CachedInstances:
time: int
time: float
value: dict
updating: bool = False
@ -27,17 +27,12 @@ def get_cached_instances() -> dict:
"""return cached instances description with updating it once per five minutes"""
if time.time() - 250 < cached_instances.time or cached_instances.updating:
return cached_instances.value
# Indicate that the value is updating now, so the cached value can be
# used. The first setting and close-to-ttl are not counted as update
if cached_instances.time != 0 or time.time() - 300 < cached_instances.time:
cached_instances.updating = True
else:
cached_instances.updating = False
cached_instances.updating = cached_value_is_valid(cached_instances.time, 300)
ec2_client = boto3.client("ec2")
instances_response = ec2_client.describe_instances(
Filters=[{"Name": "instance-state-name", "Values": ["running"]}]
)
cached_instances.time = int(time.time())
cached_instances.time = time.time()
cached_instances.value = {
instance["InstanceId"]: instance
for reservation in instances_response["Reservations"]
@ -47,6 +42,28 @@ def get_cached_instances() -> dict:
return cached_instances.value
@dataclass
class CachedRunners:
time: float
value: RunnerDescriptions
updating: bool = False
cached_runners = CachedRunners(0, [])
def get_cached_runners(access_token: str) -> RunnerDescriptions:
"""From time to time request to GH api costs up to 3 seconds, and
it's a disaster from the termination lambda perspective"""
if time.time() - 5 < cached_runners.time or cached_instances.updating:
return cached_runners.value
cached_runners.updating = cached_value_is_valid(cached_runners.time, 15)
cached_runners.value = list_runners(access_token)
cached_runners.time = time.time()
cached_runners.updating = False
return cached_runners.value
def how_many_instances_to_kill(event_data: dict) -> Dict[str, int]:
data_array = event_data["CapacityToTerminate"]
to_kill_by_zone = {} # type: Dict[str, int]
@ -104,7 +121,7 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]:
)
print("Time spent on the requests to AWS: ", time.time() - start)
runners = list_runners(access_token)
runners = get_cached_runners(access_token)
runner_ids = set(runner.name for runner in runners)
# We used to delete potential hosts to terminate from GitHub runners pool,
# but the documentation states:

View File

@ -12,7 +12,8 @@ echo "Running init script"
export DEBIAN_FRONTEND=noninteractive
export RUNNER_HOME=/home/ubuntu/actions-runner
export RUNNER_URL="https://github.com/ClickHouse"
export RUNNER_ORG="ClickHouse"
export RUNNER_URL="https://github.com/${RUNNER_ORG}"
# Funny fact, but metadata service has fixed IP
INSTANCE_ID=$(ec2metadata --instance-id)
export INSTANCE_ID
@ -102,7 +103,8 @@ check_proceed_spot_termination() {
runner_pid=$(pgrep Runner.Listener)
if [ -n "$runner_pid" ]; then
# Kill the runner to not allow it cancelling the job
kill -9 "$runner_pid"
# shellcheck disable=SC2046
kill -9 $(list_children "$runner_pid")
fi
sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)"
terminate_and_exit
@ -171,6 +173,7 @@ set -uo pipefail
echo "Runner's public DNS: $(ec2metadata --public-hostname)"
echo "Runner's labels: ${LABELS}"
echo "Runner's instance type: $(ec2metadata --instance-type)"
EOF
# Create a post-run script that will restart docker daemon before the job started
@ -234,6 +237,19 @@ is_job_assigned() {
|| return 1
}
list_children () {
local children
children=$(ps --ppid "$1" -o pid=)
if [ -z "$children" ]; then
return
fi
for pid in $children; do
list_children "$pid"
done
echo "$children"
}
while true; do
runner_pid=$(pgrep Runner.Listener)
echo "Got runner pid '$runner_pid'"
@ -268,17 +284,11 @@ while true; do
RUNNER_AGE=$(( $(date +%s) - $(stat -c +%Y /proc/"$runner_pid" 2>/dev/null || date +%s) ))
echo "The runner is launched $RUNNER_AGE seconds ago and still has hot received the job"
if (( 60 < RUNNER_AGE )); then
echo "Check if the instance should tear down"
if ! no_terminating_metadata; then
# Another check if the worker still didn't start
if is_job_assigned; then
echo "During the metadata check the job was assigned, continue"
continue
fi
kill -9 "$runner_pid"
sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)"
terminate_on_event
fi
echo "Attempt to delete the runner for a graceful shutdown"
sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" \
|| continue
echo "Runner didn't launch or have assigned jobs after ${RUNNER_AGE} seconds, shutting down"
terminate_and_exit
fi
fi
sleep 5

View File

@ -957,6 +957,16 @@ class TestCase:
elif tags and ("no-s3-storage" in tags) and args.s3_storage:
return FailureReason.S3_STORAGE
elif (
tags
and ("no-s3-storage-with-slow-build" in tags)
and args.s3_storage
and (
BuildFlags.THREAD in args.build_flags
or BuildFlags.DEBUG in args.build_flags
)
):
return FailureReason.S3_STORAGE
elif tags:
for build_flag in args.build_flags:

View File

@ -6,14 +6,18 @@
<endpoint>http://localhost:11111/test/test/</endpoint>
<access_key_id>clickhouse</access_key_id>
<secret_access_key>clickhouse</secret_access_key>
<data_cache_enabled>1</data_cache_enabled>
<data_cache_max_size>22548578304</data_cache_max_size>
</s3>
<cached_s3>
<type>cache</type>
<max_size>1Gi</max_size>
<path>/var/lib/clickhouse/s3_cache/</path>
<disk>s3</disk>
</cached_s3>
</disks>
<policies>
<s3>
<volumes>
<main><disk>s3</disk></main>
<main><disk>cached_s3</disk></main>
</volumes>
</s3>
</policies>

View File

@ -2,6 +2,6 @@
<text_log>
<database>system</database>
<table>text_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<flush_interval_milliseconds>4000</flush_interval_milliseconds>
</text_log>
</clickhouse>

View File

@ -43,7 +43,7 @@ ln -sf $SRC_PATH/config.d/transactions.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/encryption.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/CORS.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/zookeeper_log.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/logger_test.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/logger_trace.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/named_collection.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/ssl_certs.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/filesystem_cache_log.xml $DEST_SERVER_PATH/config.d/

View File

@ -91,6 +91,5 @@
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_http_named_session",
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_grpc",
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp_and_others",
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp",
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_end_session"
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_setting_in_query"
]

View File

@ -1,6 +1,7 @@
import pytest
from helpers.client import CommandRequest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
@ -13,7 +14,7 @@ node = cluster.add_instance(
)
@pytest.fixture(scope="module")
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
@ -23,41 +24,122 @@ def started_cluster():
cluster.shutdown()
def test_base_commands(started_cluster):
_ = started_cluster
command = CommandRequest(
def keeper_query(query: str):
return CommandRequest(
[
started_cluster.server_bin_path,
cluster.server_bin_path,
"keeper-client",
"--host",
str(cluster.get_instance_ip("zoo1")),
"--port",
str(cluster.zookeeper_port),
"-q",
"create test_create_zk_node1 testvalue1;create test_create_zk_node_2 testvalue2;get test_create_zk_node1;",
query,
],
stdin="",
)
def test_big_family():
command = keeper_query(
"create test_big_family foo;"
"create test_big_family/1 foo;"
"create test_big_family/1/1 foo;"
"create test_big_family/1/2 foo;"
"create test_big_family/1/3 foo;"
"create test_big_family/1/4 foo;"
"create test_big_family/1/5 foo;"
"create test_big_family/2 foo;"
"create test_big_family/2/1 foo;"
"create test_big_family/2/2 foo;"
"create test_big_family/2/3 foo;"
"find_big_family test_big_family;"
)
assert command.get_answer() == TSV(
[
["/test_big_family/1", "5"],
["/test_big_family/2", "3"],
["/test_big_family/2/3", "0"],
["/test_big_family/2/2", "0"],
["/test_big_family/2/1", "0"],
["/test_big_family/1/5", "0"],
["/test_big_family/1/4", "0"],
["/test_big_family/1/3", "0"],
["/test_big_family/1/2", "0"],
["/test_big_family/1/1", "0"],
]
)
command = keeper_query("find_big_family test_big_family 1;")
assert command.get_answer() == TSV(
[
["/test_big_family/1", "5"],
]
)
def test_find_super_nodes():
command = keeper_query(
"create test_find_super_nodes foo;"
"create test_find_super_nodes/1 foo;"
"create test_find_super_nodes/1/1 foo;"
"create test_find_super_nodes/1/2 foo;"
"create test_find_super_nodes/1/3 foo;"
"create test_find_super_nodes/1/4 foo;"
"create test_find_super_nodes/1/5 foo;"
"create test_find_super_nodes/2 foo;"
"create test_find_super_nodes/2/1 foo;"
"create test_find_super_nodes/2/2 foo;"
"create test_find_super_nodes/2/3 foo;"
"create test_find_super_nodes/2/4 foo;"
"cd test_find_super_nodes;"
"find_super_nodes 4;"
)
assert command.get_answer() == TSV(
[
["/test_find_super_nodes/1", "5"],
["/test_find_super_nodes/2", "4"],
]
)
def test_delete_stable_backups():
command = keeper_query(
"create /clickhouse/backups foo;"
"create /clickhouse/backups/1 foo;"
"create /clickhouse/backups/1/stage foo;"
"create /clickhouse/backups/1/stage/alive123 foo;"
"create /clickhouse/backups/2 foo;"
"create /clickhouse/backups/2/stage foo;"
"create /clickhouse/backups/2/stage/dead123 foo;"
"delete_stable_backups;"
"y;"
"ls clickhouse/backups;"
)
assert command.get_answer() == (
"You are going to delete all inactive backups in /clickhouse/backups. Continue?\n"
'Found backup "/clickhouse/backups/1", checking if it\'s active\n'
'Backup "/clickhouse/backups/1" is active, not going to delete\n'
'Found backup "/clickhouse/backups/2", checking if it\'s active\n'
'Backup "/clickhouse/backups/2" is not active, deleting it\n'
"1\n"
)
def test_base_commands():
command = keeper_query(
"create test_create_zk_node1 testvalue1;"
"create test_create_zk_node_2 testvalue2;"
"get test_create_zk_node1;"
)
assert command.get_answer() == "testvalue1\n"
def test_four_letter_word_commands(started_cluster):
_ = started_cluster
command = CommandRequest(
[
started_cluster.server_bin_path,
"keeper-client",
"--host",
str(cluster.get_instance_ip("zoo1")),
"--port",
str(cluster.zookeeper_port),
"-q",
"ruok",
],
stdin="",
)
def test_four_letter_word_commands():
command = keeper_query("ruok")
assert command.get_answer() == "imok\n"

View File

@ -43,6 +43,7 @@ def get_fake_zk(node):
return ku.get_fake_zk(cluster, node)
@pytest.mark.skip(reason="test is flaky because changes are not properly waited for")
def test_reconfig_replace_leader_in_one_command(started_cluster):
"""
Remove leader from a cluster of 3 and add a new node to this cluster in a single command

View File

@ -8,6 +8,8 @@ import sys
import threading
from helpers.cluster import ClickHouseCluster, run_and_check
from helpers.test_tools import assert_logs_contain_with_retry
MAX_SESSIONS_FOR_USER = 2
POSTGRES_SERVER_PORT = 5433
@ -54,7 +56,7 @@ instance = cluster.add_instance(
def get_query(name, id):
return f"SElECT '{name}', {id}, sleep(1)"
return f"SElECT '{name}', {id}, number from system.numbers"
def grpc_get_url():
@ -83,22 +85,21 @@ def grpc_query(query_text, channel, session_id_):
def threaded_run_test(sessions):
instance.rotate_logs()
thread_list = []
for i in range(len(sessions)):
thread = ThreadWithException(target=sessions[i], args=(i,))
thread_list.append(thread)
thread.start()
if len(sessions) > MAX_SESSIONS_FOR_USER:
assert_logs_contain_with_retry(instance, "overflown session count")
instance.query(f"KILL QUERY WHERE user='{TEST_USER}' SYNC")
for thread in thread_list:
thread.join()
exception_count = 0
for i in range(len(sessions)):
if thread_list[i].run_exception != None:
exception_count += 1
assert exception_count == 1
@pytest.fixture(scope="module")
def started_cluster():
@ -110,16 +111,11 @@ def started_cluster():
class ThreadWithException(threading.Thread):
run_exception = None
def run(self):
try:
super().run()
except:
self.run_exception = sys.exc_info()
def join(self):
super().join()
pass
def postgres_session(id):
@ -206,17 +202,5 @@ def test_profile_max_sessions_for_user_tcp_and_others(started_cluster):
threaded_run_test([tcp_session, postgres_session, http_session])
def test_profile_max_sessions_for_user_end_session(started_cluster):
for conection_func in [
tcp_session,
http_session,
grpc_session,
mysql_session,
postgres_session,
]:
threaded_run_test([conection_func] * MAX_SESSIONS_FOR_USER)
threaded_run_test([conection_func] * MAX_SESSIONS_FOR_USER)
def test_profile_max_sessions_for_user_end_session(started_cluster):
def test_profile_max_sessions_for_user_setting_in_query(started_cluster):
instance.query_and_get_error("SET max_sessions_for_user = 10")

View File

@ -0,0 +1,11 @@
<clickhouse>
<s3>
<s3_mock>
<endpoint>http://resolver:8080</endpoint>
<header>Authorization: Bearer TOKEN</header>
</s3_mock>
<s3_mock_restricted_directory>
<endpoint>http://resolver:8080/root-with-auth/restricteddirectory/</endpoint>
</s3_mock_restricted_directory>
</s3>
</clickhouse>

View File

@ -0,0 +1,43 @@
<clickhouse>
<named_collections>
<s3_conf1>
<url>http://minio1:9001/root/test_table</url>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</s3_conf1>
<s3_parquet>
<url>http://minio1:9001/root/test_parquet</url>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</s3_parquet>
<s3_parquet_gz>
<url>http://minio1:9001/root/test_parquet_gz</url>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</s3_parquet_gz>
<s3_orc>
<url>http://minio1:9001/root/test_orc</url>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</s3_orc>
<s3_native>
<url>http://minio1:9001/root/test_native</url>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</s3_native>
<s3_arrow>
<url>http://minio1:9001/root/test.arrow</url>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</s3_arrow>
<s3_parquet2>
<url>http://minio1:9001/root/test.parquet</url>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</s3_parquet2>
<s3_json_no_sign>
<url>http://minio1:9001/root/test_cache4.jsonl</url>
<no_sign_request>true</no_sign_request>
</s3_json_no_sign>
</named_collections>
</clickhouse>

View File

@ -0,0 +1,7 @@
<clickhouse>
<profiles>
<default>
<stream_like_engine_allow_direct_select>1</stream_like_engine_allow_direct_select>
</default>
</profiles>
</clickhouse>

View File

@ -0,0 +1,873 @@
import io
import logging
import os
import random
import time
import pytest
from helpers.client import QueryRuntimeException
from helpers.cluster import ClickHouseCluster, ClickHouseInstance
import json
"""
export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-server
export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-client
export CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-odbc-bridge
export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/home/sergey/vkr/ClickHouse/programs/server
"""
def prepare_s3_bucket(started_cluster):
# Allows read-write access for bucket without authorization.
bucket_read_write_policy = {
"Version": "2012-10-17",
"Statement": [
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:GetBucketLocation",
"Resource": "arn:aws:s3:::root",
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:ListBucket",
"Resource": "arn:aws:s3:::root",
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:GetObject",
"Resource": "arn:aws:s3:::root/*",
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:PutObject",
"Resource": "arn:aws:s3:::root/*",
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:DeleteObject",
"Resource": "arn:aws:s3:::root/*",
},
],
}
minio_client = started_cluster.minio_client
minio_client.set_bucket_policy(
started_cluster.minio_bucket, json.dumps(bucket_read_write_policy)
)
started_cluster.minio_restricted_bucket = "{}-with-auth".format(
started_cluster.minio_bucket
)
if minio_client.bucket_exists(started_cluster.minio_restricted_bucket):
minio_client.remove_bucket(started_cluster.minio_restricted_bucket)
minio_client.make_bucket(started_cluster.minio_restricted_bucket)
@pytest.fixture(autouse=True)
def s3_queue_setup_teardown(started_cluster):
instance = started_cluster.instances["instance"]
instance_2 = started_cluster.instances["instance2"]
instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;")
instance_2.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;")
minio = started_cluster.minio_client
objects = list(
minio.list_objects(started_cluster.minio_restricted_bucket, recursive=True)
)
for obj in objects:
minio.remove_object(started_cluster.minio_restricted_bucket, obj.object_name)
yield # run test
MINIO_INTERNAL_PORT = 9001
AVAILABLE_MODES = ["unordered", "ordered"]
AUTH = "'minio','minio123',"
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
def put_s3_file_content(started_cluster, bucket, filename, data):
buf = io.BytesIO(data)
started_cluster.minio_client.put_object(bucket, filename, buf, len(data))
def generate_random_files(
count, prefix, cluster, bucket, column_num=3, row_num=10, start_ind=0
):
total_values = []
to_generate = [
(f"{prefix}/test_{i}.csv", i) for i in range(start_ind, start_ind + count)
]
to_generate.sort(key=lambda x: x[0])
for filename, i in to_generate:
rand_values = [
[random.randint(0, 50) for _ in range(column_num)] for _ in range(row_num)
]
total_values += rand_values
values_csv = (
"\n".join((",".join(map(str, row)) for row in rand_values)) + "\n"
).encode()
put_s3_file_content(cluster, bucket, filename, values_csv)
return total_values
# Returns content of given S3 file as string.
def get_s3_file_content(started_cluster, bucket, filename, decode=True):
# type: (ClickHouseCluster, str, str, bool) -> str
data = started_cluster.minio_client.get_object(bucket, filename)
data_str = b""
for chunk in data.stream():
data_str += chunk
if decode:
return data_str.decode()
return data_str
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance(
"instance",
user_configs=["configs/users.xml"],
with_minio=True,
with_zookeeper=True,
main_configs=["configs/defaultS3.xml", "configs/named_collections.xml"],
)
cluster.add_instance(
"instance2",
user_configs=["configs/users.xml"],
with_minio=True,
with_zookeeper=True,
main_configs=["configs/defaultS3.xml", "configs/named_collections.xml"],
)
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
prepare_s3_bucket(cluster)
yield cluster
finally:
cluster.shutdown()
def run_query(instance, query, stdin=None, settings=None):
# type: (ClickHouseInstance, str, object, dict) -> str
logging.info("Running query '{}'...".format(query))
result = instance.query(query, stdin=stdin, settings=settings)
logging.info("Query finished")
return result
@pytest.mark.parametrize("mode", AVAILABLE_MODES)
def test_delete_after_processing(started_cluster, mode):
prefix = "delete"
bucket = started_cluster.minio_bucket
instance = started_cluster.instances["instance"]
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
total_values = generate_random_files(5, prefix, started_cluster, bucket)
instance.query(
f"""
DROP TABLE IF EXISTS test.s3_queue;
CREATE TABLE test.s3_queue ({table_format})
ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = '{mode}',
keeper_path = '/clickhouse/test_delete_{mode}',
s3queue_loading_retries = 3,
after_processing='delete';
"""
)
get_query = f"SELECT * FROM test.s3_queue"
assert [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
] == total_values
minio = started_cluster.minio_client
objects = list(minio.list_objects(started_cluster.minio_bucket, recursive=True))
assert len(objects) == 0
@pytest.mark.parametrize("mode", AVAILABLE_MODES)
def test_failed_retry(started_cluster, mode):
bucket = started_cluster.minio_restricted_bucket
instance = started_cluster.instances["instance"]
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
values = [
["failed", 1, 1],
]
values_csv = (
"\n".join((",".join(map(str, row)) for row in values)) + "\n"
).encode()
filename = f"test.csv"
put_s3_file_content(started_cluster, bucket, filename, values_csv)
instance.query(
f"""
DROP TABLE IF EXISTS test.s3_queue;
CREATE TABLE test.s3_queue ({table_format})
ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {AUTH}'CSV')
SETTINGS
mode = '{mode}',
keeper_path = '/clickhouse/select_failed_retry_{mode}',
s3queue_loading_retries = 3;
"""
)
# first try
get_query = f"SELECT * FROM test.s3_queue"
assert [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
] == []
# second try
assert [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
] == []
# upload correct file
values = [
[1, 1, 1],
]
values_csv = (
"\n".join((",".join(map(str, row)) for row in values)) + "\n"
).encode()
put_s3_file_content(started_cluster, bucket, filename, values_csv)
assert [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
] == values
assert [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
] == []
@pytest.mark.parametrize("mode", AVAILABLE_MODES)
def test_direct_select_file(started_cluster, mode):
auth = "'minio','minio123',"
bucket = started_cluster.minio_restricted_bucket
instance = started_cluster.instances["instance"]
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
values = [
[12549, 2463, 19893],
[64021, 38652, 66703],
[81611, 39650, 83516],
]
values_csv = (
"\n".join((",".join(map(str, row)) for row in values)) + "\n"
).encode()
filename = f"test.csv"
put_s3_file_content(started_cluster, bucket, filename, values_csv)
instance.query(
"""
DROP TABLE IF EXISTS test.s3_queue;
DROP TABLE IF EXISTS test.s3_queue_2;
DROP TABLE IF EXISTS test.s3_queue_3;
"""
)
instance.query(
f"""
CREATE TABLE test.s3_queue ({table_format})
ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV')
SETTINGS
mode = '{mode}',
keeper_path = '/clickhouse/select_{mode}'
"""
)
get_query = f"SELECT * FROM test.s3_queue"
assert [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
] == values
instance.query(
f"""
CREATE TABLE test.s3_queue_2 ({table_format})
ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV')
SETTINGS
mode = '{mode}',
keeper_path = '/clickhouse/select_{mode}'
"""
)
get_query = f"SELECT * FROM test.s3_queue"
assert [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
] == []
# New table with same zookeeper path
get_query = f"SELECT * FROM test.s3_queue_2"
assert [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
] == []
# New table with different zookeeper path
instance.query(
f"""
CREATE TABLE test.s3_queue_3 ({table_format})
ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV')
SETTINGS
mode = '{mode}',
keeper_path='/clickhouse/select_{mode}_2'
"""
)
get_query = f"SELECT * FROM test.s3_queue_3"
assert [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
] == values
values = [
[1, 1, 1],
]
values_csv = (
"\n".join((",".join(map(str, row)) for row in values)) + "\n"
).encode()
filename = f"t.csv"
put_s3_file_content(started_cluster, bucket, filename, values_csv)
get_query = f"SELECT * FROM test.s3_queue_3"
if mode == "unordered":
assert [
list(map(int, l.split()))
for l in run_query(instance, get_query).splitlines()
] == values
elif mode == "ordered":
assert [
list(map(int, l.split()))
for l in run_query(instance, get_query).splitlines()
] == []
@pytest.mark.parametrize("mode", AVAILABLE_MODES)
def test_direct_select_multiple_files(started_cluster, mode):
prefix = f"multiple_files_{mode}"
bucket = started_cluster.minio_restricted_bucket
instance = started_cluster.instances["instance"]
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
instance.query("drop table if exists test.s3_queue")
instance.query(
f"""
CREATE TABLE test.s3_queue ({table_format})
ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = '{mode}',
keeper_path = '/clickhouse/select_multiple_{mode}'
"""
)
for i in range(5):
rand_values = [[random.randint(0, 50) for _ in range(3)] for _ in range(10)]
values_csv = (
"\n".join((",".join(map(str, row)) for row in rand_values)) + "\n"
).encode()
filename = f"{prefix}/test_{i}.csv"
put_s3_file_content(started_cluster, bucket, filename, values_csv)
get_query = f"SELECT * FROM test.s3_queue"
assert [
list(map(int, l.split()))
for l in run_query(instance, get_query).splitlines()
] == rand_values
total_values = generate_random_files(
4, prefix, started_cluster, bucket, start_ind=5
)
get_query = f"SELECT * FROM test.s3_queue"
assert {
tuple(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
} == set([tuple(i) for i in total_values])
@pytest.mark.parametrize("mode", AVAILABLE_MODES)
def test_streaming_to_view_(started_cluster, mode):
prefix = f"streaming_files_{mode}"
bucket = started_cluster.minio_restricted_bucket
instance = started_cluster.instances["instance"]
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
total_values = generate_random_files(10, prefix, started_cluster, bucket)
instance.query(
f"""
DROP TABLE IF EXISTS test.s3_queue_persistent;
DROP TABLE IF EXISTS test.s3_queue;
DROP TABLE IF EXISTS test.persistent_s3_queue_mv;
CREATE TABLE test.s3_queue_persistent ({table_format})
ENGINE = MergeTree()
ORDER BY column1;
CREATE TABLE test.s3_queue ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = '{mode}',
keeper_path = '/clickhouse/view_{mode}';
CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS
SELECT
*
FROM test.s3_queue;
"""
)
expected_values = set([tuple(i) for i in total_values])
for i in range(10):
get_query = f"SELECT * FROM test.persistent_s3_queue_mv"
selected_values = {
tuple(map(int, l.split()))
for l in run_query(instance, get_query).splitlines()
}
if selected_values != expected_values:
time.sleep(1)
else:
break
assert selected_values == expected_values
@pytest.mark.parametrize("mode", AVAILABLE_MODES)
def test_streaming_to_many_views(started_cluster, mode):
prefix = f"streaming_files_{mode}"
bucket = started_cluster.minio_restricted_bucket
instance = started_cluster.instances["instance"]
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
retry_cnt = 10
instance.query(
f"""
DROP TABLE IF EXISTS test.s3_queue_persistent;
DROP TABLE IF EXISTS test.s3_queue_persistent_2;
DROP TABLE IF EXISTS test.s3_queue_persistent_3;
DROP TABLE IF EXISTS test.s3_queue;
DROP TABLE IF EXISTS test.persistent_s3_queue_mv;
DROP TABLE IF EXISTS test.persistent_s3_queue_mv_2;
DROP TABLE IF EXISTS test.persistent_s3_queue_mv_3;
CREATE TABLE test.s3_queue_persistent ({table_format})
ENGINE = MergeTree()
ORDER BY column1;
CREATE TABLE test.s3_queue_persistent_2 ({table_format})
ENGINE = MergeTree()
ORDER BY column1;
CREATE TABLE test.s3_queue_persistent_3 ({table_format})
ENGINE = MergeTree()
ORDER BY column1;
CREATE TABLE test.s3_queue ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = '{mode}',
keeper_path = '/clickhouse/multiple_view_{mode}';
CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS
SELECT
*
FROM test.s3_queue;
CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_2 TO test.s3_queue_persistent_2 AS
SELECT
*
FROM test.s3_queue;
CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_3 TO test.s3_queue_persistent_3 AS
SELECT
*
FROM test.s3_queue;
"""
)
total_values = generate_random_files(5, prefix, started_cluster, bucket)
expected_values = set([tuple(i) for i in total_values])
for i in range(retry_cnt):
retry = False
for get_query in [
f"SELECT * FROM test.s3_queue_persistent",
f"SELECT * FROM test.s3_queue_persistent_2",
f"SELECT * FROM test.s3_queue_persistent_3",
]:
selected_values = {
tuple(map(int, l.split()))
for l in run_query(instance, get_query).splitlines()
}
if i == retry_cnt - 1:
assert selected_values == expected_values
if selected_values != expected_values:
retry = True
break
if retry:
time.sleep(1)
else:
break
def test_multiple_tables_meta_mismatch(started_cluster):
prefix = f"test_meta"
bucket = started_cluster.minio_restricted_bucket
instance = started_cluster.instances["instance"]
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
instance.query(
f"""
DROP TABLE IF EXISTS test.s3_queue;
CREATE TABLE test.s3_queue ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = 'ordered',
keeper_path = '/clickhouse/test_meta';
"""
)
# check mode
failed = False
try:
instance.query(
f"""
CREATE TABLE test.s3_queue_copy ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = 'unordered',
keeper_path = '/clickhouse/test_meta';
"""
)
except QueryRuntimeException as e:
assert "Existing table metadata in ZooKeeper differs in engine mode" in str(e)
failed = True
assert failed is True
# check columns
table_format_copy = table_format + ", column4 UInt32"
try:
instance.query(
f"""
CREATE TABLE test.s3_queue_copy ({table_format_copy})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = 'ordered',
keeper_path = '/clickhouse/test_meta';
"""
)
except QueryRuntimeException as e:
assert (
"Table columns structure in ZooKeeper is different from local table structure"
in str(e)
)
failed = True
assert failed is True
# check format
try:
instance.query(
f"""
CREATE TABLE test.s3_queue_copy ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'TSV')
SETTINGS
mode = 'ordered',
keeper_path = '/clickhouse/test_meta';
"""
)
except QueryRuntimeException as e:
assert "Existing table metadata in ZooKeeper differs in format name" in str(e)
failed = True
assert failed is True
# create working engine
instance.query(
f"""
CREATE TABLE test.s3_queue_copy ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = 'ordered',
keeper_path = '/clickhouse/test_meta';
"""
)
def test_max_set_age(started_cluster):
files_to_generate = 10
max_age = 1
prefix = f"test_multiple"
bucket = started_cluster.minio_restricted_bucket
instance = started_cluster.instances["instance"]
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
instance.query(
f"""
DROP TABLE IF EXISTS test.s3_queue;
CREATE TABLE test.s3_queue ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = 'unordered',
keeper_path = '/clickhouse/test_set_age',
s3queue_tracked_files_limit = 10,
s3queue_tracked_file_ttl_sec = {max_age};
"""
)
total_values = generate_random_files(
files_to_generate, prefix, started_cluster, bucket, row_num=1
)
get_query = f"SELECT * FROM test.s3_queue"
res1 = [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
]
assert res1 == total_values
time.sleep(max_age + 1)
get_query = f"SELECT * FROM test.s3_queue"
res1 = [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
]
assert res1 == total_values
@pytest.mark.parametrize("mode", AVAILABLE_MODES)
def test_multiple_tables_streaming_sync(started_cluster, mode):
files_to_generate = 300
poll_size = 30
prefix = f"test_multiple_{mode}"
bucket = started_cluster.minio_restricted_bucket
instance = started_cluster.instances["instance"]
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
instance.query(
f"""
DROP TABLE IF EXISTS test.s3_queue;
DROP TABLE IF EXISTS test.s3_queue_copy;
DROP TABLE IF EXISTS test.s3_queue_copy_2;
DROP TABLE IF EXISTS test.s3_queue_persistent;
DROP TABLE IF EXISTS test.s3_queue_persistent_copy;
DROP TABLE IF EXISTS test.s3_queue_persistent_copy_2;
DROP TABLE IF EXISTS test.persistent_s3_queue_mv;
DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy;
DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy_2;
CREATE TABLE test.s3_queue ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = '{mode}',
keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}',
s3queue_polling_size = {poll_size};
CREATE TABLE test.s3_queue_copy ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = '{mode}',
keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}',
s3queue_polling_size = {poll_size};
CREATE TABLE test.s3_queue_copy_2 ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = '{mode}',
keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}',
s3queue_polling_size = {poll_size};
CREATE TABLE test.s3_queue_persistent ({table_format})
ENGINE = MergeTree()
ORDER BY column1;
CREATE TABLE test.s3_queue_persistent_copy ({table_format})
ENGINE = MergeTree()
ORDER BY column1;
CREATE TABLE test.s3_queue_persistent_copy_2 ({table_format})
ENGINE = MergeTree()
ORDER BY column1;
CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS
SELECT
*
FROM test.s3_queue;
CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy TO test.s3_queue_persistent_copy AS
SELECT
*
FROM test.s3_queue_copy;
CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy_2 TO test.s3_queue_persistent_copy_2 AS
SELECT
*
FROM test.s3_queue_copy_2;
"""
)
total_values = generate_random_files(
files_to_generate, prefix, started_cluster, bucket, row_num=1
)
def get_count(table_name):
return int(run_query(instance, f"SELECT count() FROM {table_name}"))
for _ in range(100):
if (
get_count("test.s3_queue_persistent")
+ get_count("test.s3_queue_persistent_copy")
+ get_count("test.s3_queue_persistent_copy_2")
) == files_to_generate:
break
time.sleep(1)
get_query = f"SELECT * FROM test.s3_queue_persistent"
res1 = [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
]
get_query_copy = f"SELECT * FROM test.s3_queue_persistent_copy"
res2 = [
list(map(int, l.split()))
for l in run_query(instance, get_query_copy).splitlines()
]
get_query_copy_2 = f"SELECT * FROM test.s3_queue_persistent_copy_2"
res3 = [
list(map(int, l.split()))
for l in run_query(instance, get_query_copy_2).splitlines()
]
assert {tuple(v) for v in res1 + res2 + res3} == set(
[tuple(i) for i in total_values]
)
# Checking that all files were processed only once
time.sleep(10)
assert (
get_count("test.s3_queue_persistent")
+ get_count("test.s3_queue_persistent_copy")
+ get_count("test.s3_queue_persistent_copy_2")
) == files_to_generate
@pytest.mark.parametrize("mode", AVAILABLE_MODES)
def test_multiple_tables_streaming_sync_distributed(started_cluster, mode):
files_to_generate = 100
poll_size = 10
prefix = f"test_multiple_{mode}"
bucket = started_cluster.minio_restricted_bucket
instance = started_cluster.instances["instance"]
instance_2 = started_cluster.instances["instance2"]
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
for inst in [instance, instance_2]:
inst.query(
f"""
DROP TABLE IF EXISTS test.s3_queue;
DROP TABLE IF EXISTS test.s3_queue_persistent;
DROP TABLE IF EXISTS test.persistent_s3_queue_mv;
CREATE TABLE test.s3_queue ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = '{mode}',
keeper_path = '/clickhouse/test_multiple_consumers_{mode}',
s3queue_polling_size = {poll_size};
CREATE TABLE test.s3_queue_persistent ({table_format})
ENGINE = MergeTree()
ORDER BY column1;
CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS
SELECT
*
FROM test.s3_queue;
"""
)
total_values = generate_random_files(
files_to_generate, prefix, started_cluster, bucket, row_num=1
)
def get_count(node, table_name):
return int(run_query(node, f"SELECT count() FROM {table_name}"))
for _ in range(100):
if (
get_count(instance, "test.s3_queue_persistent")
+ get_count(instance_2, "test.s3_queue_persistent")
) == files_to_generate:
break
time.sleep(1)
get_query = f"SELECT * FROM test.s3_queue_persistent"
res1 = [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
]
res2 = [
list(map(int, l.split())) for l in run_query(instance_2, get_query).splitlines()
]
# Checking that all engines have made progress
assert len(res1) > 0
assert len(res2) > 0
assert len(res1) + len(res2) == files_to_generate
assert {tuple(v) for v in res1 + res2} == set([tuple(i) for i in total_values])
# Checking that all files were processed only once
time.sleep(10)
assert (
get_count(instance, "test.s3_queue_persistent")
+ get_count(instance_2, "test.s3_queue_persistent")
) == files_to_generate
def test_max_set_size(started_cluster):
files_to_generate = 10
prefix = f"test_multiple"
bucket = started_cluster.minio_restricted_bucket
instance = started_cluster.instances["instance"]
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
instance.query(
f"""
DROP TABLE IF EXISTS test.s3_queue;
CREATE TABLE test.s3_queue ({table_format})
ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV')
SETTINGS
mode = 'unordered',
keeper_path = '/clickhouse/test_set_size',
s3queue_tracked_files_limit = {files_to_generate - 1};
"""
)
total_values = generate_random_files(
files_to_generate, prefix, started_cluster, bucket, start_ind=0, row_num=1
)
get_query = f"SELECT * FROM test.s3_queue"
res1 = [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
]
assert res1 == total_values
get_query = f"SELECT * FROM test.s3_queue"
res1 = [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
]
assert res1 == [total_values[0]]
get_query = f"SELECT * FROM test.s3_queue"
res1 = [
list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()
]
assert res1 == [total_values[1]]

View File

@ -25,5 +25,5 @@
9631199822919835226
4334672815104069193
4334672815104069193
1
1
6145F501578671E2877DBA2BE487AF7E
16FE7483905CCE7A85670E43E4678877

View File

@ -32,7 +32,5 @@ SELECT gccMurmurHash('foo');
SELECT gccMurmurHash('\x01');
SELECT gccMurmurHash(1);
-- Comparison with reverse for big endian
SELECT hex(murmurHash3_128('foo')) = hex(reverse(unhex('6145F501578671E2877DBA2BE487AF7E'))) or hex(murmurHash3_128('foo')) = '6145F501578671E2877DBA2BE487AF7E';
-- Comparison with reverse for big endian
SELECT hex(murmurHash3_128('\x01')) = hex(reverse(unhex('16FE7483905CCE7A85670E43E4678877'))) or hex(murmurHash3_128('\x01')) = '16FE7483905CCE7A85670E43E4678877';
SELECT hex(murmurHash3_128('foo'));
SELECT hex(murmurHash3_128('\x01'));

View File

@ -1,11 +1,11 @@
12940785793559895259
17926972817233444501
7456555839952096623
1
1
1
1
1
CC45107CC4B79F62D831BEF2103C7CBF
DF2EC2F0669B000EDFF6ADEE264E7D68
4CD1C30C38AB935D418B5269EF197B9E
9D78134EE48654D753CCA1B76185CF8E
389D16428D2AADEC9713905572F42864
955237314186186656
8175794665478042155
9325786087413524176
@ -18,8 +18,8 @@
8163029322371165472
8788309436660676487
236561483980029756
1
1
8DD5527CC43D76F4760D26BE0F641F7E
F8F7AD9B6CD4CF117A71E277E2EC2931
12384823029245979431
4507350192761038840
1188926775431157506

View File

@ -4,11 +4,11 @@ SELECT sipHash64(1, 2, 3);
SELECT sipHash64(1, 3, 2);
SELECT sipHash64(('a', [1, 2, 3], 4, (4, ['foo', 'bar'], 1, (1, 2))));
SELECT hex(sipHash128('foo')) = hex(reverse(unhex('CC45107CC4B79F62D831BEF2103C7CBF'))) or hex(sipHash128('foo')) = 'CC45107CC4B79F62D831BEF2103C7CBF';
SELECT hex(sipHash128('\x01')) = hex(reverse(unhex('DF2EC2F0669B000EDFF6ADEE264E7D68'))) or hex(sipHash128('\x01')) = 'DF2EC2F0669B000EDFF6ADEE264E7D68';
SELECT hex(sipHash128('foo', 'foo')) = hex(reverse(unhex('4CD1C30C38AB935D418B5269EF197B9E'))) or hex(sipHash128('foo', 'foo')) = '4CD1C30C38AB935D418B5269EF197B9E';
SELECT hex(sipHash128('foo', 'foo', 'foo')) = hex(reverse(unhex('9D78134EE48654D753CCA1B76185CF8E'))) or hex(sipHash128('foo', 'foo', 'foo')) = '9D78134EE48654D753CCA1B76185CF8E';
SELECT hex(sipHash128(1, 2, 3)) = hex(reverse(unhex('389D16428D2AADEC9713905572F42864'))) or hex(sipHash128(1, 2, 3)) = '389D16428D2AADEC9713905572F42864';
SELECT hex(sipHash128('foo'));
SELECT hex(sipHash128('\x01'));
SELECT hex(sipHash128('foo', 'foo'));
SELECT hex(sipHash128('foo', 'foo', 'foo'));
SELECT hex(sipHash128(1, 2, 3));
SELECT halfMD5(1, 2, 3);
SELECT halfMD5(1, 3, 2);
@ -26,8 +26,8 @@ SELECT murmurHash3_64(1, 2, 3);
SELECT murmurHash3_64(1, 3, 2);
SELECT murmurHash3_64(('a', [1, 2, 3], 4, (4, ['foo', 'bar'], 1, (1, 2))));
SELECT hex(murmurHash3_128('foo', 'foo')) = hex(reverse(unhex('8DD5527CC43D76F4760D26BE0F641F7E'))) or hex(murmurHash3_128('foo', 'foo')) = '8DD5527CC43D76F4760D26BE0F641F7E';
SELECT hex(murmurHash3_128('foo', 'foo', 'foo')) = hex(reverse(unhex('F8F7AD9B6CD4CF117A71E277E2EC2931'))) or hex(murmurHash3_128('foo', 'foo', 'foo')) = 'F8F7AD9B6CD4CF117A71E277E2EC2931';
SELECT hex(murmurHash3_128('foo', 'foo'));
SELECT hex(murmurHash3_128('foo', 'foo', 'foo'));
SELECT gccMurmurHash(1, 2, 3);
SELECT gccMurmurHash(1, 3, 2);

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long, no-fasttest
# Tags: long, no-fasttest, no-debug
#
# Load all possible .parquet files found in submodules.

View File

@ -36,3 +36,5 @@ tx14 10 22 all_1_14_2_18
tx14 10 42 all_1_14_2_18
tx14 10 62 all_1_14_2_18
tx14 10 82 all_1_14_2_18
11 2 all_2_2_0
11 10 all_1_1_0_3

View File

@ -94,3 +94,16 @@ tx 14 "begin transaction"
tx 14 "select 10, n, _part from mt order by n" | accept_both_parts
$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table mt"
$CLICKHOUSE_CLIENT -q "create table mt (n int) engine=MergeTree order by tuple()"
$CLICKHOUSE_CLIENT --implicit_transaction=1 -q "insert into mt values (1)"
tx 15 "begin transaction"
tx 16 "begin transaction"
tx 16 "insert into mt values (2)"
tx 15 "alter table mt update n = 10*n where 1"
tx 15 "commit"
tx 16 "commit"
$CLICKHOUSE_CLIENT --implicit_transaction=1 -q "select 11, n, _part from mt order by n"
$CLICKHOUSE_CLIENT -q "drop table mt"

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long
# Tags: long, no-s3-storage-with-slow-build
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,6 +1,8 @@
-- Tags: no-tsan
-- Tag no-tsan: Too long for TSan
set enable_filesystem_cache=0;
set enable_filesystem_cache_on_write_operations=0;
drop table if exists t;
create table t (x UInt64, s String) engine = MergeTree order by x;

View File

@ -24,8 +24,8 @@ state: DOWNLOADED
8 2014
8 2014
24 84045
35 168815
44 252113
32 167243
41 250541
5010500
18816
Using storage policy: local_cache
@ -54,7 +54,7 @@ state: DOWNLOADED
8 2014
8 2014
24 84045
35 168815
44 252113
32 167243
41 250541
5010500
18816

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-parallel, no-fasttest
# Tags: no-parallel, no-fasttest, no-debug
# Tag no-parallel -- to avoid running it in parallel, this will avoid possible issues due to high pressure
# Test that ensures that WRITE lock failure notifies READ.

View File

@ -191,9 +191,9 @@ E51B38608EF25F57
1
1
E28DBDE7FE22E41C
1
1CE422FEE7BD8DE20000000000000000
E28DBDE7FE22E41C
1
1CE422FEE7BD8DE20000000000000000
Check bug with hashing of const integer values
11862823756610506724
11862823756610506724

View File

@ -269,9 +269,9 @@ select sipHash64Keyed(toUInt64(0), '1'); -- { serverError 48 }
select sipHash128Keyed(toUInt64(0), '1'); -- { serverError 48 }
select hex(sipHash64());
SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000';
SELECT hex(sipHash128());
select hex(sipHash64Keyed());
SELECT hex(sipHash128Keyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000';
SELECT hex(sipHash128Keyed());
SELECT 'Check bug with hashing of const integer values';
DROP TABLE IF EXISTS tab;

View File

@ -1 +1 @@
1
1CE422FEE7BD8DE20000000000000000

View File

@ -1 +1 @@
SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000';
SELECT hex(sipHash128Reference());

View File

@ -1 +1,2 @@
select match('default/k8s1', '\\A(?:(?:[-0-9_a-z]+(?:\\.[-0-9_a-z]+)*)/k8s1)\\z');
select match('abc123', '[a-zA-Z]+(?P<num>\\d+)');

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