Merge branch 'master' into local-assert

This commit is contained in:
Kruglov Pavel 2023-08-02 12:31:06 +02:00 committed by GitHub
commit cd33bd1e88
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
232 changed files with 7941 additions and 704 deletions

View File

@ -8,6 +8,7 @@
#include <functional>
#include <iosfwd>
#include <base/defines.h>
#include <base/types.h>
#include <base/unaligned.h>
@ -274,6 +275,8 @@ struct CRC32Hash
if (size == 0)
return 0;
chassert(pos);
if (size < 8)
{
return static_cast<unsigned>(hashLessThan8(x.data, x.size));

View File

@ -115,8 +115,15 @@
/// because SIGABRT is easier to debug than SIGTRAP (the second one makes gdb crazy)
#if !defined(chassert)
#if defined(ABORT_ON_LOGICAL_ERROR)
// clang-format off
#include <base/types.h>
namespace DB
{
void abortOnFailedAssertion(const String & description);
}
#define chassert(x) static_cast<bool>(x) ? void(0) : ::DB::abortOnFailedAssertion(#x)
#define UNREACHABLE() abort()
// clang-format off
#else
/// Here sizeof() trick is used to suppress unused warning for result,
/// since simple "(void)x" will evaluate the expression, while

View File

@ -57,7 +57,7 @@ public:
URI();
/// Creates an empty URI.
explicit URI(const std::string & uri, bool disable_url_encoding = false);
explicit URI(const std::string & uri, bool enable_url_encoding = true);
/// Parses an URI from the given string. Throws a
/// SyntaxException if the uri is not valid.
@ -362,7 +362,7 @@ private:
std::string _query;
std::string _fragment;
bool _disable_url_encoding = false;
bool _enable_url_encoding = true;
};

View File

@ -36,8 +36,8 @@ URI::URI():
}
URI::URI(const std::string& uri, bool decode_and_encode_path):
_port(0), _disable_url_encoding(decode_and_encode_path)
URI::URI(const std::string& uri, bool enable_url_encoding):
_port(0), _enable_url_encoding(enable_url_encoding)
{
parse(uri);
}
@ -108,7 +108,7 @@ URI::URI(const URI& uri):
_path(uri._path),
_query(uri._query),
_fragment(uri._fragment),
_disable_url_encoding(uri._disable_url_encoding)
_enable_url_encoding(uri._enable_url_encoding)
{
}
@ -121,7 +121,7 @@ URI::URI(const URI& baseURI, const std::string& relativeURI):
_path(baseURI._path),
_query(baseURI._query),
_fragment(baseURI._fragment),
_disable_url_encoding(baseURI._disable_url_encoding)
_enable_url_encoding(baseURI._enable_url_encoding)
{
resolve(relativeURI);
}
@ -153,7 +153,7 @@ URI& URI::operator = (const URI& uri)
_path = uri._path;
_query = uri._query;
_fragment = uri._fragment;
_disable_url_encoding = uri._disable_url_encoding;
_enable_url_encoding = uri._enable_url_encoding;
}
return *this;
}
@ -184,7 +184,7 @@ void URI::swap(URI& uri)
std::swap(_path, uri._path);
std::swap(_query, uri._query);
std::swap(_fragment, uri._fragment);
std::swap(_disable_url_encoding, uri._disable_url_encoding);
std::swap(_enable_url_encoding, uri._enable_url_encoding);
}
@ -687,18 +687,18 @@ void URI::decode(const std::string& str, std::string& decodedStr, bool plusAsSpa
void URI::encodePath(std::string & encodedStr) const
{
if (_disable_url_encoding)
encodedStr = _path;
else
if (_enable_url_encoding)
encode(_path, RESERVED_PATH, encodedStr);
else
encodedStr = _path;
}
void URI::decodePath(const std::string & encodedStr)
{
if (_disable_url_encoding)
_path = encodedStr;
else
if (_enable_url_encoding)
decode(encodedStr, _path);
else
_path = encodedStr;
}
bool URI::isWellKnownPort() const

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

@ -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

@ -106,4 +106,4 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da
## Storage Settings {#storage-settings}
- [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default.
- [disable_url_encoding](/docs/en/operations/settings/settings.md#disable_url_encoding) -allows to disable decoding/encoding path in uri. Disabled by default.
- [enable_url_encoding](/docs/en/operations/settings/settings.md#enable_url_encoding) - allows to enable/disable decoding/encoding path in uri. Enabled by default.

View File

@ -1723,6 +1723,34 @@ You can select data from a ClickHouse table and save them into some file in the
``` bash
$ clickhouse-client --query = "SELECT * FROM test.hits FORMAT CapnProto SETTINGS format_schema = 'schema:Message'"
```
### Using autogenerated schema {#using-autogenerated-capn-proto-schema}
If you don't have an external CapnProto schema for your data, you can still output/input data in CapnProto format using autogenerated schema.
For example:
```sql
SELECT * FROM test.hits format CapnProto SETTINGS format_capn_proto_use_autogenerated_schema=1
```
In this case ClickHouse will autogenerate CapnProto schema according to the table structure using function [structureToCapnProtoSchema](../sql-reference/functions/other-functions.md#structure_to_capn_proto_schema) and will use this schema to serialize data in CapnProto format.
You can also read CapnProto file with autogenerated schema (in this case the file must be created using the same schema):
```bash
$ cat hits.bin | clickhouse-client --query "INSERT INTO test.hits SETTINGS format_capn_proto_use_autogenerated_schema=1 FORMAT CapnProto"
```
The setting [format_capn_proto_use_autogenerated_schema](../operations/settings/settings-formats.md#format_capn_proto_use_autogenerated_schema) is enabled by default and applies if [format_schema](../operations/settings/settings-formats.md#formatschema-format-schema) is not set.
You can also save autogenerated schema in the file during input/output using setting [output_format_schema](../operations/settings/settings-formats.md#outputformatschema-output-format-schema). For example:
```sql
SELECT * FROM test.hits format CapnProto SETTINGS format_capn_proto_use_autogenerated_schema=1, output_format_schema='path/to/schema/schema.capnp'
```
In this case autogenerated CapnProto schema will be saved in file `path/to/schema/schema.capnp`.
## Prometheus {#prometheus}
Expose metrics in [Prometheus text-based exposition format](https://prometheus.io/docs/instrumenting/exposition_formats/#text-based-format).
@ -1861,6 +1889,33 @@ ClickHouse inputs and outputs protobuf messages in the `length-delimited` format
It means before every message should be written its length as a [varint](https://developers.google.com/protocol-buffers/docs/encoding#varints).
See also [how to read/write length-delimited protobuf messages in popular languages](https://cwiki.apache.org/confluence/display/GEODE/Delimiting+Protobuf+Messages).
### Using autogenerated schema {#using-autogenerated-protobuf-schema}
If you don't have an external Protobuf schema for your data, you can still output/input data in Protobuf format using autogenerated schema.
For example:
```sql
SELECT * FROM test.hits format Protobuf SETTINGS format_protobuf_use_autogenerated_schema=1
```
In this case ClickHouse will autogenerate Protobuf schema according to the table structure using function [structureToProtobufSchema](../sql-reference/functions/other-functions.md#structure_to_protobuf_schema) and will use this schema to serialize data in Protobuf format.
You can also read Protobuf file with autogenerated schema (in this case the file must be created using the same schema):
```bash
$ cat hits.bin | clickhouse-client --query "INSERT INTO test.hits SETTINGS format_protobuf_use_autogenerated_schema=1 FORMAT Protobuf"
```
The setting [format_protobuf_use_autogenerated_schema](../operations/settings/settings-formats.md#format_protobuf_use_autogenerated_schema) is enabled by default and applies if [format_schema](../operations/settings/settings-formats.md#formatschema-format-schema) is not set.
You can also save autogenerated schema in the file during input/output using setting [output_format_schema](../operations/settings/settings-formats.md#outputformatschema-output-format-schema). For example:
```sql
SELECT * FROM test.hits format Protobuf SETTINGS format_protobuf_use_autogenerated_schema=1, output_format_schema='path/to/schema/schema.proto'
```
In this case autogenerated Protobuf schema will be saved in file `path/to/schema/schema.capnp`.
## ProtobufSingle {#protobufsingle}
Same as [Protobuf](#protobuf) but for storing/parsing single Protobuf message without length delimiters.

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

@ -321,6 +321,10 @@ If both `input_format_allow_errors_num` and `input_format_allow_errors_ratio` ar
This parameter is useful when you are using formats that require a schema definition, such as [Capn Proto](https://capnproto.org/) or [Protobuf](https://developers.google.com/protocol-buffers/). The value depends on the format.
## output_format_schema {#output-format-schema}
The path to the file where the automatically generated schema will be saved in [Capn Proto](../../interfaces/formats.md#capnproto-capnproto) or [Protobuf](../../interfaces/formats.md#protobuf-protobuf) formats.
## output_format_enable_streaming {#output_format_enable_streaming}
Enable streaming in output formats that support it.
@ -1330,6 +1334,11 @@ When serializing Nullable columns with Google wrappers, serialize default values
Disabled by default.
### format_protobuf_use_autogenerated_schema {#format_capn_proto_use_autogenerated_schema}
Use autogenerated Protobuf schema when [format_schema](#formatschema-format-schema) is not set.
The schema is generated from ClickHouse table structure using function [structureToProtobufSchema](../../sql-reference/functions/other-functions.md#structure_to_protobuf_schema)
## Avro format settings {#avro-format-settings}
### input_format_avro_allow_missing_fields {#input_format_avro_allow_missing_fields}
@ -1626,6 +1635,11 @@ Possible values:
Default value: `'by_values'`.
### format_capn_proto_use_autogenerated_schema {#format_capn_proto_use_autogenerated_schema}
Use autogenerated CapnProto schema when [format_schema](#formatschema-format-schema) is not set.
The schema is generated from ClickHouse table structure using function [structureToCapnProtoSchema](../../sql-reference/functions/other-functions.md#structure_to_capnproto_schema)
## MySQLDump format settings {#musqldump-format-settings}
### input_format_mysql_dump_table_name (#input_format_mysql_dump_table_name)

View File

@ -3468,11 +3468,11 @@ Possible values:
Default value: `0`.
## disable_url_encoding {#disable_url_encoding}
## enable_url_encoding {#enable_url_encoding}
Allows to disable decoding/encoding path in uri in [URL](../../engines/table-engines/special/url.md) engine tables.
Allows to enable/disable decoding/encoding path in uri in [URL](../../engines/table-engines/special/url.md) engine tables.
Disabled by default.
Enabled by default.
## database_atomic_wait_for_drop_and_detach_synchronously {#database_atomic_wait_for_drop_and_detach_synchronously}
@ -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

@ -2552,3 +2552,187 @@ Result:
This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables.
## structureToCapnProtoSchema {#structure_to_capn_proto_schema}
Converts ClickHouse table structure to CapnProto schema.
**Syntax**
``` sql
structureToCapnProtoSchema(structure)
```
**Arguments**
- `structure` — Table structure in a format `column1_name column1_type, column2_name column2_type, ...`.
- `root_struct_name` — Name for root struct in CapnProto schema. Default value - `Message`;
**Returned value**
- CapnProto schema
Type: [String](../../sql-reference/data-types/string.md).
**Examples**
Query:
``` sql
SELECT structureToCapnProtoSchema('column1 String, column2 UInt32, column3 Array(String)') FORMAT RawBLOB
```
Result:
``` text
@0xf96402dd754d0eb7;
struct Message
{
column1 @0 : Data;
column2 @1 : UInt32;
column3 @2 : List(Data);
}
```
Query:
``` sql
SELECT structureToCapnProtoSchema('column1 Nullable(String), column2 Tuple(element1 UInt32, element2 Array(String)), column3 Map(String, String)') FORMAT RawBLOB
```
Result:
``` text
@0xd1c8320fecad2b7f;
struct Message
{
struct Column1
{
union
{
value @0 : Data;
null @1 : Void;
}
}
column1 @0 : Column1;
struct Column2
{
element1 @0 : UInt32;
element2 @1 : List(Data);
}
column2 @1 : Column2;
struct Column3
{
struct Entry
{
key @0 : Data;
value @1 : Data;
}
entries @0 : List(Entry);
}
column3 @2 : Column3;
}
```
Query:
``` sql
SELECT structureToCapnProtoSchema('column1 String, column2 UInt32', 'Root') FORMAT RawBLOB
```
Result:
``` text
@0x96ab2d4ab133c6e1;
struct Root
{
column1 @0 : Data;
column2 @1 : UInt32;
}
```
## structureToProtobufSchema {#structure_to_protobuf_schema}
Converts ClickHouse table structure to Protobuf schema.
**Syntax**
``` sql
structureToProtobufSchema(structure)
```
**Arguments**
- `structure` — Table structure in a format `column1_name column1_type, column2_name column2_type, ...`.
- `root_message_name` — Name for root message in Protobuf schema. Default value - `Message`;
**Returned value**
- Protobuf schema
Type: [String](../../sql-reference/data-types/string.md).
**Examples**
Query:
``` sql
SELECT structureToProtobufSchema('column1 String, column2 UInt32, column3 Array(String)') FORMAT RawBLOB
```
Result:
``` text
syntax = "proto3";
message Message
{
bytes column1 = 1;
uint32 column2 = 2;
repeated bytes column3 = 3;
}
```
Query:
``` sql
SELECT structureToProtobufSchema('column1 Nullable(String), column2 Tuple(element1 UInt32, element2 Array(String)), column3 Map(String, String)') FORMAT RawBLOB
```
Result:
``` text
syntax = "proto3";
message Message
{
bytes column1 = 1;
message Column2
{
uint32 element1 = 1;
repeated bytes element2 = 2;
}
Column2 column2 = 2;
map<string, bytes> column3 = 3;
}
```
Query:
``` sql
SELECT structureToProtobufSchema('column1 String, column2 UInt32', 'Root') FORMAT RawBLOB
```
Result:
``` text
syntax = "proto3";
message Root
{
bytes column1 = 1;
uint32 column2 = 2;
}
```

View File

@ -56,7 +56,7 @@ Character `|` inside patterns is used to specify failover addresses. They are it
## Storage Settings {#storage-settings}
- [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default.
- [disable_url_encoding](/docs/en/operations/settings/settings.md#disable_url_encoding) - allows to disable decoding/encoding path in uri. Disabled by default.
- [enable_url_encoding](/docs/en/operations/settings/settings.md#enable_url_encoding) - allows to enable/disable decoding/encoding path in uri. Enabled by default.
**See Also**

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

@ -0,0 +1,221 @@
#include <Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.h>
#include <Functions/FunctionFactory.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
#include <Common/DateLUT.h>
#include <Common/DateLUTImpl.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
class OptimizeDateOrDateTimeConverterWithPreimageVisitor : public InDepthQueryTreeVisitorWithContext<OptimizeDateOrDateTimeConverterWithPreimageVisitor>
{
public:
using Base = InDepthQueryTreeVisitorWithContext<OptimizeDateOrDateTimeConverterWithPreimageVisitor>;
explicit OptimizeDateOrDateTimeConverterWithPreimageVisitor(ContextPtr context)
: Base(std::move(context))
{}
static bool needChildVisit(QueryTreeNodePtr & node, QueryTreeNodePtr & /*child*/)
{
const static std::unordered_set<String> relations = {
"equals",
"notEquals",
"less",
"greater",
"lessOrEquals",
"greaterOrEquals",
};
if (const auto * function = node->as<FunctionNode>())
{
return !relations.contains(function->getFunctionName());
}
return true;
}
void enterImpl(QueryTreeNodePtr & node) const
{
const static std::unordered_map<String, String> swap_relations = {
{"equals", "equals"},
{"notEquals", "notEquals"},
{"less", "greater"},
{"greater", "less"},
{"lessOrEquals", "greaterOrEquals"},
{"greaterOrEquals", "lessOrEquals"},
};
const auto * function = node->as<FunctionNode>();
if (!function || !swap_relations.contains(function->getFunctionName())) return;
if (function->getArguments().getNodes().size() != 2) return;
size_t func_id = function->getArguments().getNodes().size();
for (size_t i = 0; i < function->getArguments().getNodes().size(); i++)
{
if (const auto * func = function->getArguments().getNodes()[i]->as<FunctionNode>())
{
func_id = i;
}
}
if (func_id == function->getArguments().getNodes().size()) return;
size_t literal_id = 1 - func_id;
const auto * literal = function->getArguments().getNodes()[literal_id]->as<ConstantNode>();
if (!literal || literal->getValue().getType() != Field::Types::UInt64) return;
String comparator = literal_id > func_id ? function->getFunctionName(): swap_relations.at(function->getFunctionName());
const auto * func_node = function->getArguments().getNodes()[func_id]->as<FunctionNode>();
/// Currently we only handle single-argument functions.
if (!func_node || func_node->getArguments().getNodes().size() != 1) return;
const auto * column_id = func_node->getArguments().getNodes()[0]->as<ColumnNode>();
if (!column_id) return;
const auto * column_type = column_id->getColumnType().get();
if (!isDateOrDate32(column_type) && !isDateTime(column_type) && !isDateTime64(column_type)) return;
const auto & converter = FunctionFactory::instance().tryGet(func_node->getFunctionName(), getContext());
if (!converter) return;
ColumnsWithTypeAndName args;
args.emplace_back(column_id->getColumnType(), "tmp");
auto converter_base = converter->build(args);
if (!converter_base || !converter_base->hasInformationAboutPreimage()) return;
auto preimage_range = converter_base->getPreimage(*(column_id->getColumnType()), literal->getValue());
if (!preimage_range) return;
const auto new_node = generateOptimizedDateFilter(comparator, *column_id, *preimage_range);
if (!new_node) return;
node = new_node;
}
private:
QueryTreeNodePtr generateOptimizedDateFilter(const String & comparator, const ColumnNode & column_node, const std::pair<Field, Field>& range) const
{
const DateLUTImpl & date_lut = DateLUT::instance("UTC");
String start_date_or_date_time;
String end_date_or_date_time;
if (isDateOrDate32(column_node.getColumnType().get()))
{
start_date_or_date_time = date_lut.dateToString(range.first.get<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.dateToString(range.second.get<DateLUTImpl::Time>());
}
else if (isDateTime(column_node.getColumnType().get()) || isDateTime64(column_node.getColumnType().get()))
{
start_date_or_date_time = date_lut.timeToString(range.first.get<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.timeToString(range.second.get<DateLUTImpl::Time>());
}
else [[unlikely]] return {};
if (comparator == "equals")
{
const auto lhs = std::make_shared<FunctionNode>("greaterOrEquals");
lhs->getArguments().getNodes().push_back(std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
lhs->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(start_date_or_date_time));
resolveOrdinaryFunctionNode(*lhs, lhs->getFunctionName());
const auto rhs = std::make_shared<FunctionNode>("less");
rhs->getArguments().getNodes().push_back(std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
rhs->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(end_date_or_date_time));
resolveOrdinaryFunctionNode(*rhs, rhs->getFunctionName());
const auto new_date_filter = std::make_shared<FunctionNode>("and");
new_date_filter->getArguments().getNodes() = {lhs, rhs};
resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName());
return new_date_filter;
}
else if (comparator == "notEquals")
{
const auto lhs = std::make_shared<FunctionNode>("less");
lhs->getArguments().getNodes().push_back(std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
lhs->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(start_date_or_date_time));
resolveOrdinaryFunctionNode(*lhs, lhs->getFunctionName());
const auto rhs = std::make_shared<FunctionNode>("greaterOrEquals");
rhs->getArguments().getNodes().push_back(std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
rhs->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(end_date_or_date_time));
resolveOrdinaryFunctionNode(*rhs, rhs->getFunctionName());
const auto new_date_filter = std::make_shared<FunctionNode>("or");
new_date_filter->getArguments().getNodes() = {lhs, rhs};
resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName());
return new_date_filter;
}
else if (comparator == "greater")
{
const auto new_date_filter = std::make_shared<FunctionNode>("greaterOrEquals");
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(end_date_or_date_time));
resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName());
return new_date_filter;
}
else if (comparator == "lessOrEquals")
{
const auto new_date_filter = std::make_shared<FunctionNode>("less");
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(end_date_or_date_time));
resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName());
return new_date_filter;
}
else if (comparator == "less" || comparator == "greaterOrEquals")
{
const auto new_date_filter = std::make_shared<FunctionNode>(comparator);
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(start_date_or_date_time));
resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName());
return new_date_filter;
}
else [[unlikely]]
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expected equals, notEquals, less, lessOrEquals, greater, greaterOrEquals. Actual {}",
comparator);
}
}
void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const
{
auto function = FunctionFactory::instance().get(function_name, getContext());
function_node.resolveAsFunction(function->build(function_node.getArgumentColumns()));
}
};
}
void OptimizeDateOrDateTimeConverterWithPreimagePass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
OptimizeDateOrDateTimeConverterWithPreimageVisitor visitor(std::move(context));
visitor.visit(query_tree_node);
}
}

View File

@ -0,0 +1,24 @@
#pragma once
#include <Analyzer/IQueryTreePass.h>
namespace DB
{
/** Replace predicate having Date/DateTime converters with their preimages to improve performance.
* Given a Date column c, toYear(c) = 2023 -> c >= '2023-01-01' AND c < '2024-01-01'
* Or if c is a DateTime column, toYear(c) = 2023 -> c >= '2023-01-01 00:00:00' AND c < '2024-01-01 00:00:00'.
* The similar optimization also applies to other converters.
*/
class OptimizeDateOrDateTimeConverterWithPreimagePass final : public IQueryTreePass
{
public:
String getName() override { return "OptimizeDateOrDateTimeConverterWithPreimagePass"; }
String getDescription() override { return "Replace predicate having Date/DateTime converters with their preimages"; }
void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
};
}

View File

@ -42,6 +42,7 @@
#include <Analyzer/Passes/CrossToInnerJoinPass.h>
#include <Analyzer/Passes/ShardNumColumnToFunctionPass.h>
#include <Analyzer/Passes/ConvertQueryToCNFPass.h>
#include <Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.h>
namespace DB
{
@ -278,6 +279,7 @@ void addQueryTreePasses(QueryTreePassManager & manager)
manager.addPass(std::make_unique<AutoFinalOnQueryPass>());
manager.addPass(std::make_unique<CrossToInnerJoinPass>());
manager.addPass(std::make_unique<ShardNumColumnToFunctionPass>());
manager.addPass(std::make_unique<OptimizeDateOrDateTimeConverterWithPreimagePass>());
}
}

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

@ -564,15 +564,22 @@ void ColumnNullable::updatePermutationImpl(IColumn::PermutationSortDirection dir
else
getNestedColumn().updatePermutation(direction, stability, limit, null_direction_hint, res, new_ranges);
equal_ranges = std::move(new_ranges);
if (unlikely(stability == PermutationSortStability::Stable))
{
for (auto & null_range : null_ranges)
::sort(res.begin() + null_range.first, res.begin() + null_range.second);
}
std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges));
if (is_nulls_last || null_ranges.empty())
{
equal_ranges = std::move(new_ranges);
std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges));
}
else
{
equal_ranges = std::move(null_ranges);
std::move(new_ranges.begin(), new_ranges.end(), std::back_inserter(equal_ranges));
}
}
void ColumnNullable::getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,

View File

@ -439,7 +439,7 @@ void ColumnSparse::compareColumn(const IColumn & rhs, size_t rhs_row_num,
PaddedPODArray<UInt64> * row_indexes, PaddedPODArray<Int8> & compare_results,
int direction, int nan_direction_hint) const
{
if (row_indexes)
if (row_indexes || !typeid_cast<const ColumnSparse *>(&rhs))
{
/// TODO: implement without conversion to full column.
auto this_full = convertToFullColumnIfSparse();

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

@ -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) \
@ -626,7 +627,7 @@ class IColumn;
M(Bool, engine_file_allow_create_multiple_files, false, "Enables or disables creating a new file on each insert in file engine tables if format has suffix.", 0) \
M(Bool, engine_file_skip_empty_files, false, "Allows to skip empty files in file table engine", 0) \
M(Bool, engine_url_skip_empty_files, false, "Allows to skip empty files in url table engine", 0) \
M(Bool, disable_url_encoding, false, " Allows to disable decoding/encoding path in uri in URL table engine", 0) \
M(Bool, enable_url_encoding, true, " Allows to enable/disable decoding/encoding path in uri in URL table engine", 0) \
M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \
M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \
M(Bool, database_replicated_enforce_synchronous_settings, false, "Enforces synchronous waiting for some queries (see also database_atomic_wait_for_drop_and_detach_synchronously, mutation_sync, alter_sync). Not recommended to enable these settings.", 0) \
@ -1011,6 +1012,10 @@ class IColumn;
\
M(CapnProtoEnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::CapnProtoEnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \
\
M(Bool, format_capn_proto_use_autogenerated_schema, true, "Use autogenerated CapnProto schema when format_schema is not set", 0) \
M(Bool, format_protobuf_use_autogenerated_schema, true, "Use autogenerated Protobuf when format_schema is not set", 0) \
M(String, output_format_schema, "", "The path to the file where the automatically generated schema will be saved", 0) \
\
M(String, input_format_mysql_dump_table_name, "", "Name of the table in MySQL dump from which to read data", 0) \
M(Bool, input_format_mysql_dump_map_column_names, true, "Match columns from table in MySQL dump and columns from ClickHouse table by names", 0) \
\
@ -1027,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

@ -14,7 +14,7 @@ namespace ErrorCodes
}
WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(TemporaryFileOnDiskHolder && tmp_file_)
: WriteBufferFromFile(tmp_file_->getPath(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, /* throttler= */ {}, 0600)
: WriteBufferFromFile(tmp_file_->getAbsolutePath(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, /* throttler= */ {}, 0600)
, tmp_file(std::move(tmp_file_))
{
}

View File

@ -54,7 +54,7 @@ TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, const String & p
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file name is empty");
}
String TemporaryFileOnDisk::getPath() const
String TemporaryFileOnDisk::getAbsolutePath() const
{
return std::filesystem::path(disk->getPath()) / relative_path;
}

View File

@ -22,7 +22,10 @@ public:
~TemporaryFileOnDisk();
DiskPtr getDisk() const { return disk; }
String getPath() const;
/// Return absolute path (disk + relative_path)
String getAbsolutePath() const;
/// Return relative path (without disk)
const String & getRelativePath() const { return relative_path; }
private:
DiskPtr disk;

View File

@ -143,12 +143,14 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.protobuf.input_flatten_google_wrappers = settings.input_format_protobuf_flatten_google_wrappers;
format_settings.protobuf.output_nullables_with_google_wrappers = settings.output_format_protobuf_nullables_with_google_wrappers;
format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference;
format_settings.protobuf.use_autogenerated_schema = settings.format_protobuf_use_autogenerated_schema;
format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule;
format_settings.regexp.regexp = settings.format_regexp;
format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched;
format_settings.schema.format_schema = settings.format_schema;
format_settings.schema.format_schema_path = context->getFormatSchemaPath();
format_settings.schema.is_server = context->hasGlobalContext() && (context->getGlobalContext()->getApplicationType() == Context::ApplicationType::SERVER);
format_settings.schema.output_format_schema = settings.output_format_schema;
format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields;
format_settings.template_settings.resultset_format = settings.format_template_resultset;
format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter;
@ -190,6 +192,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields;
format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode;
format_settings.capn_proto.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference;
format_settings.capn_proto.use_autogenerated_schema = settings.format_capn_proto_use_autogenerated_schema;
format_settings.seekable_read = settings.input_format_allow_seeks;
format_settings.msgpack.number_of_columns = settings.input_format_msgpack_number_of_columns;
format_settings.msgpack.output_uuid_representation = settings.output_format_msgpack_uuid_representation;

View File

@ -1,6 +1,8 @@
#include <Formats/FormatSchemaInfo.h>
#include <Interpreters/Context.h>
#include <Common/Exception.h>
#include <Common/filesystemHelpers.h>
#include <Disks/IO/WriteBufferFromTemporaryFile.h>
#include <filesystem>
@ -105,4 +107,84 @@ FormatSchemaInfo::FormatSchemaInfo(const FormatSettings & settings, const String
{
}
template <typename SchemaGenerator>
MaybeAutogeneratedFormatSchemaInfo<SchemaGenerator>::MaybeAutogeneratedFormatSchemaInfo(
const FormatSettings & settings, const String & format, const Block & header, bool use_autogenerated_schema)
{
if (!use_autogenerated_schema || !settings.schema.format_schema.empty())
{
schema_info = std::make_unique<FormatSchemaInfo>(settings, format, true);
return;
}
String schema_path;
fs::path default_schema_directory_path(fs::canonical(settings.schema.format_schema_path) / "");
fs::path path;
if (!settings.schema.output_format_schema.empty())
{
schema_path = settings.schema.output_format_schema;
path = schema_path;
if (path.is_absolute())
{
if (settings.schema.is_server)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Absolute path in the 'output_format_schema' setting is prohibited: {}", path.string());
}
else if (path.has_parent_path() && !fs::weakly_canonical(default_schema_directory_path / path).string().starts_with(fs::weakly_canonical(default_schema_directory_path).string()))
{
if (settings.schema.is_server)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: {} ({} not in {})",
default_schema_directory_path.string(),
path.string(),
default_schema_directory_path.string());
path = default_schema_directory_path / path;
}
else
{
path = default_schema_directory_path / path;
}
}
else
{
if (settings.schema.is_server)
{
tmp_file_path = PocoTemporaryFile::tempName(default_schema_directory_path.string()) + '.' + getFormatSchemaDefaultFileExtension(format);
schema_path = fs::path(tmp_file_path).filename();
}
else
{
tmp_file_path = PocoTemporaryFile::tempName() + '.' + getFormatSchemaDefaultFileExtension(format);
schema_path = tmp_file_path;
}
path = tmp_file_path;
}
WriteBufferFromFile buf(path.string());
SchemaGenerator::writeSchema(buf, "Message", header.getNamesAndTypesList());
buf.finalize();
schema_info = std::make_unique<FormatSchemaInfo>(schema_path + ":Message", format, true, settings.schema.is_server, settings.schema.format_schema_path);
}
template <typename SchemaGenerator>
MaybeAutogeneratedFormatSchemaInfo<SchemaGenerator>::~MaybeAutogeneratedFormatSchemaInfo()
{
if (!tmp_file_path.empty())
{
try
{
fs::remove(tmp_file_path);
}
catch (...)
{
tryLogCurrentException("MaybeAutogeneratedFormatSchemaInfo", "Cannot delete temporary schema file");
}
}
}
template class MaybeAutogeneratedFormatSchemaInfo<StructureToCapnProtoSchema>;
template class MaybeAutogeneratedFormatSchemaInfo<StructureToProtobufSchema>;
}

View File

@ -2,6 +2,8 @@
#include <base/types.h>
#include <Formats/FormatSettings.h>
#include <Formats/StructureToCapnProtoSchema.h>
#include <Formats/StructureToProtobufSchema.h>
namespace DB
{
@ -30,4 +32,23 @@ private:
String message_name;
};
template <typename SchemaGenerator>
class MaybeAutogeneratedFormatSchemaInfo
{
public:
MaybeAutogeneratedFormatSchemaInfo(const FormatSettings & settings, const String & format, const Block & header, bool use_autogenerated_schema);
~MaybeAutogeneratedFormatSchemaInfo();
const FormatSchemaInfo & getSchemaInfo() const { return *schema_info; }
private:
std::unique_ptr<FormatSchemaInfo> schema_info;
String tmp_file_path;
};
using CapnProtoSchemaInfo = MaybeAutogeneratedFormatSchemaInfo<StructureToCapnProtoSchema>;
using ProtobufSchemaInfo = MaybeAutogeneratedFormatSchemaInfo<StructureToProtobufSchema>;
}

View File

@ -276,6 +276,7 @@ struct FormatSettings
*/
bool allow_multiple_rows_without_delimiter = false;
bool skip_fields_with_unsupported_types_in_schema_inference = false;
bool use_autogenerated_schema = true;
} protobuf;
struct
@ -297,6 +298,7 @@ struct FormatSettings
std::string format_schema;
std::string format_schema_path;
bool is_server = false;
std::string output_format_schema;
} schema;
struct
@ -359,6 +361,7 @@ struct FormatSettings
{
CapnProtoEnumComparingMode enum_comparing_mode = CapnProtoEnumComparingMode::BY_VALUES;
bool skip_fields_with_unsupported_types_in_schema_inference = false;
bool use_autogenerated_schema = true;
} capn_proto;
enum class MsgPackUUIDRepresentation

View File

@ -3029,7 +3029,7 @@ namespace
if (!message_serializer)
{
throw Exception(ErrorCodes::NO_COLUMNS_SERIALIZED_TO_PROTOBUF_FIELDS,
"Not found matches between the names of the columns {{}} and the fields {{}} of the message {} in the protobuf schema",
"Not found matches between the names of the columns ({}) and the fields ({}) of the message {} in the protobuf schema",
boost::algorithm::join(column_names, ", "), boost::algorithm::join(getFieldNames(message_descriptor), ", "),
quoteString(message_descriptor.full_name()));
}
@ -3647,7 +3647,7 @@ namespace
if (!message_serializer)
{
throw Exception(ErrorCodes::NO_COLUMNS_SERIALIZED_TO_PROTOBUF_FIELDS,
"Not found matches between the names of the tuple's elements {{}} and the fields {{}} "
"Not found matches between the names of the tuple's elements ({}) and the fields ({}) "
"of the message {} in the protobuf schema",
boost::algorithm::join(tuple_data_type.getElementNames(), ", "),
boost::algorithm::join(getFieldNames(*field_descriptor.message_type()), ", "),

View File

@ -0,0 +1,236 @@
#include <Formats/StructureToCapnProtoSchema.h>
#include <Formats/StructureToFormatSchemaUtils.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeEnum.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/randomSeed.h>
#include <pcg_random.hpp>
namespace DB
{
using namespace StructureToFormatSchemaUtils;
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
namespace
{
const std::unordered_map<TypeIndex, String> capn_proto_simple_type_names =
{
{TypeIndex::Int8, "Int8"},
{TypeIndex::UInt8, "UInt8"},
{TypeIndex::Int16, "Int16"},
{TypeIndex::UInt16, "UInt16"},
{TypeIndex::Int32, "Int32"},
{TypeIndex::UInt32, "UInt32"},
{TypeIndex::Int64, "Int64"},
{TypeIndex::UInt64, "UInt64"},
{TypeIndex::Int128, "Data"},
{TypeIndex::UInt128, "Data"},
{TypeIndex::Int256, "Data"},
{TypeIndex::UInt256, "Data"},
{TypeIndex::Float32, "Float32"},
{TypeIndex::Float64, "Float64"},
{TypeIndex::Decimal32, "Int32"},
{TypeIndex::Decimal64, "Int64"},
{TypeIndex::Decimal128, "Data"},
{TypeIndex::Decimal256, "Data"},
{TypeIndex::String, "Data"},
{TypeIndex::FixedString, "Data"},
{TypeIndex::UUID, "Data"},
{TypeIndex::Date, "UInt16"},
{TypeIndex::Date32, "Int32"},
{TypeIndex::DateTime, "UInt32"},
{TypeIndex::DateTime64, "Int64"},
{TypeIndex::IPv4, "UInt32"},
{TypeIndex::IPv6, "Data"},
};
void writeCapnProtoHeader(WriteBuffer & buf)
{
pcg64 rng(randomSeed());
size_t id = rng() | (1ull << 63); /// First bit should be 1
writeString(fmt::format("@0x{};\n\n", getHexUIntLowercase(id)), buf);
}
void writeFieldDefinition(WriteBuffer & buf, const String & type_name, const String & column_name, size_t & field_index, size_t indent)
{
writeIndent(buf, indent);
writeString(fmt::format("{} @{} : {};\n", getSchemaFieldName(column_name), field_index++, type_name), buf);
}
void startEnum(WriteBuffer & buf, const String & enum_name, size_t indent)
{
startNested(buf, enum_name, "enum", indent);
}
void startUnion(WriteBuffer & buf, size_t indent)
{
startNested(buf, "", "union", indent);
}
void startStruct(WriteBuffer & buf, const String & struct_name, size_t indent)
{
startNested(buf, struct_name, "struct", indent);
}
String prepareAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent);
void writeField(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t & field_index, size_t indent)
{
auto field_type_name = prepareAndGetCapnProtoTypeName(buf, data_type, column_name, indent);
writeFieldDefinition(buf, field_type_name, column_name, field_index, indent);
}
String prepareArrayAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
const auto & nested_type = assert_cast<const DataTypeArray &>(*data_type).getNestedType();
auto nested_type_name = prepareAndGetCapnProtoTypeName(buf, nested_type, column_name, indent);
return "List(" + nested_type_name + ")";
}
String prepareNullableAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
/// Nullable is represented as a struct with union with 2 fields:
///
/// struct Nullable
/// {
/// union
/// {
/// value @0 : Value;
/// null @1 : Void;
/// }
/// }
auto struct_name = getSchemaMessageName(column_name);
startStruct(buf, struct_name, indent);
auto nested_type_name = prepareAndGetCapnProtoTypeName(buf, assert_cast<const DataTypeNullable &>(*data_type).getNestedType(), column_name, indent);
startUnion(buf, indent + 1);
size_t field_index = 0;
writeFieldDefinition(buf, nested_type_name, "value", field_index, indent + 2);
writeFieldDefinition(buf, "Void", "null", field_index, indent + 2);
endNested(buf, indent + 1);
endNested(buf, indent);
return struct_name;
}
String prepareTupleAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
const auto & tuple_type = assert_cast<const DataTypeTuple &>(*data_type);
auto nested_names_and_types = getCollectedTupleElements(tuple_type);
String struct_name = getSchemaMessageName(column_name);
startStruct(buf, struct_name, indent);
size_t nested_field_index = 0;
for (const auto & [name, type] : nested_names_and_types)
writeField(buf, type, name, nested_field_index, indent + 1);
endNested(buf, indent);
return struct_name;
}
String prepareMapAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
/// We output/input Map type as follow CapnProto schema
///
/// struct Map
/// {
/// struct Entry
/// {
/// key @0: Key;
/// value @1: Value;
/// }
/// entries @0 :List(Entry);
/// }
const auto & map_type = assert_cast<const DataTypeMap &>(*data_type);
const auto & key_type = map_type.getKeyType();
const auto & value_type = map_type.getValueType();
String struct_name = getSchemaMessageName(column_name);
startStruct(buf, struct_name, indent);
startStruct(buf, "Entry", indent + 1);
auto key_type_name = prepareAndGetCapnProtoTypeName(buf, key_type, "key", indent + 2);
auto value_type_name = prepareAndGetCapnProtoTypeName(buf, value_type, "value", indent + 2);
size_t field_index = 0;
writeFieldDefinition(buf, key_type_name, "key", field_index, indent + 2);
writeFieldDefinition(buf, value_type_name, "value", field_index, indent + 2);
endNested(buf, indent + 1);
field_index = 0;
writeFieldDefinition(buf, "List(Entry)", "entries", field_index, indent + 1);
endNested(buf, indent);
return struct_name;
}
template <typename EnumType>
String prepareEnumAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
const auto & enum_type = assert_cast<const DataTypeEnum<EnumType> &>(*data_type);
String enum_name = getSchemaMessageName(column_name);
startEnum(buf, enum_name, indent);
const auto & names = enum_type.getAllRegisteredNames();
for (size_t i = 0; i != names.size(); ++i)
{
writeIndent(buf, indent + 1);
writeString(fmt::format("{} @{};\n", names[i], std::to_string(i)), buf);
}
endNested(buf, indent);
return enum_name;
}
String prepareAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
TypeIndex type_id = data_type->getTypeId();
switch (data_type->getTypeId())
{
case TypeIndex::Nullable:
return prepareNullableAndGetCapnProtoTypeName(buf, data_type, column_name, indent);
case TypeIndex::LowCardinality:
return prepareAndGetCapnProtoTypeName(buf, assert_cast<const DataTypeLowCardinality &>(*data_type).getDictionaryType(), column_name, indent);
case TypeIndex::Array:
return prepareArrayAndGetCapnProtoTypeName(buf, data_type, column_name, indent);
case TypeIndex::Tuple:
return prepareTupleAndGetCapnProtoTypeName(buf, data_type, column_name, indent);
case TypeIndex::Map:
return prepareMapAndGetCapnProtoTypeName(buf, data_type, column_name, indent);
case TypeIndex::Enum8:
return prepareEnumAndGetCapnProtoTypeName<Int8>(buf, data_type, column_name, indent);
case TypeIndex::Enum16:
return prepareEnumAndGetCapnProtoTypeName<Int16>(buf, data_type, column_name, indent);
default:
{
if (isBool(data_type))
return "Bool";
auto it = capn_proto_simple_type_names.find(type_id);
if (it == capn_proto_simple_type_names.end())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "CapnProto type name is not found for type {}", data_type->getName());
return it->second;
}
}
}
}
void StructureToCapnProtoSchema::writeSchema(WriteBuffer & buf, const String & message_name, const NamesAndTypesList & names_and_types_)
{
auto names_and_types = collectNested(names_and_types_);
writeCapnProtoHeader(buf);
startStruct(buf, getSchemaMessageName(message_name), 0);
size_t field_index = 0;
for (const auto & [column_name, data_type] : names_and_types)
writeField(buf, data_type, column_name, field_index, 1);
endNested(buf, 0);
}
}

View File

@ -0,0 +1,16 @@
#pragma once
#include <IO/WriteBuffer.h>
#include <Core/NamesAndTypes.h>
namespace DB
{
struct StructureToCapnProtoSchema
{
static constexpr auto name = "structureToCapnProtoSchema";
static void writeSchema(WriteBuffer & buf, const String & message_name, const NamesAndTypesList & names_and_types_);
};
}

View File

@ -0,0 +1,117 @@
#include <Formats/StructureToFormatSchemaUtils.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace StructureToFormatSchemaUtils
{
void writeIndent(WriteBuffer & buf, size_t indent)
{
writeChar(' ', indent * 4, buf);
}
void startNested(WriteBuffer & buf, const String & nested_name, const String & nested_type, size_t indent)
{
writeIndent(buf, indent);
writeString(nested_type, buf);
if (!nested_name.empty())
{
writeChar(' ', buf);
writeString(nested_name, buf);
}
writeChar('\n', buf);
writeIndent(buf, indent);
writeCString("{\n", buf);
}
void endNested(WriteBuffer & buf, size_t indent)
{
writeIndent(buf, indent);
writeCString("}\n", buf);
}
String getSchemaFieldName(const String & column_name)
{
String result = column_name;
/// Replace all first uppercase letters to lower-case,
/// because fields in CapnProto schema must begin with a lower-case letter.
/// Don't replace all letters to lower-case to remain camelCase field names.
for (auto & symbol : result)
{
if (islower(symbol))
break;
symbol = tolower(symbol);
}
return result;
}
String getSchemaMessageName(const String & column_name)
{
String result = column_name;
if (!column_name.empty() && isalpha(column_name[0]))
result[0] = toupper(column_name[0]);
return result;
}
namespace
{
std::pair<String, String> splitName(const String & name)
{
const auto * begin = name.data();
const auto * end = name.data() + name.size();
const auto * it = find_first_symbols<'_', '.'>(begin, end);
String first = String(begin, it);
String second = it == end ? "" : String(it + 1, end);
return {std::move(first), std::move(second)};
}
}
NamesAndTypesList collectNested(const NamesAndTypesList & names_and_types)
{
/// Find all columns with dots '.' or underscores '_' and move them into a tuple.
/// For example if we have columns 'a.b UInt32, a.c UInt32, x_y String' we will
/// change it to 'a Tuple(b UInt32, c UInt32), x Tuple(y String)'
NamesAndTypesList result;
std::unordered_map<String, NamesAndTypesList> nested;
for (const auto & [name, type] : names_and_types)
{
auto [field_name, nested_name] = splitName(name);
if (nested_name.empty())
result.emplace_back(name, type);
else
nested[field_name].emplace_back(nested_name, type);
}
for (const auto & [field_name, elements]: nested)
result.emplace_back(field_name, std::make_shared<DataTypeTuple>(elements.getTypes(), elements.getNames()));
return result;
}
NamesAndTypesList getCollectedTupleElements(const DataTypeTuple & tuple_type)
{
const auto & nested_types = tuple_type.getElements();
Names nested_names;
if (tuple_type.haveExplicitNames())
{
nested_names = tuple_type.getElementNames();
}
else
{
nested_names.reserve(nested_types.size());
for (size_t i = 0; i != nested_types.size(); ++i)
nested_names.push_back("e" + std::to_string(i + 1));
}
NamesAndTypesList result;
for (size_t i = 0; i != nested_names.size(); ++i)
result.emplace_back(nested_names[i], nested_types[i]);
return collectNested(result);
}
}
}

View File

@ -0,0 +1,27 @@
#pragma once
#include <Core/NamesAndTypes.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeTuple.h>
namespace DB
{
namespace StructureToFormatSchemaUtils
{
void writeIndent(WriteBuffer & buf, size_t indent);
void startNested(WriteBuffer & buf, const String & nested_name, const String & nested_type, size_t indent);
void endNested(WriteBuffer & buf, size_t indent);
String getSchemaFieldName(const String & column_name);
String getSchemaMessageName(const String & column_name);
NamesAndTypesList collectNested(const NamesAndTypesList & names_and_types);
NamesAndTypesList getCollectedTupleElements(const DataTypeTuple & tuple_type);
}
}

View File

@ -0,0 +1,214 @@
#include <Formats/StructureToProtobufSchema.h>
#include <Formats/StructureToFormatSchemaUtils.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeEnum.h>
#include <Common/StringUtils/StringUtils.h>
namespace DB
{
using namespace StructureToFormatSchemaUtils;
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
namespace
{
const std::unordered_map<TypeIndex, String> protobuf_simple_type_names =
{
{TypeIndex::Int8, "int32"},
{TypeIndex::UInt8, "uint32"},
{TypeIndex::Int16, "int32"},
{TypeIndex::UInt16, "uint32"},
{TypeIndex::Int32, "int32"},
{TypeIndex::UInt32, "uint32"},
{TypeIndex::Int64, "int64"},
{TypeIndex::UInt64, "uint64"},
{TypeIndex::Int128, "bytes"},
{TypeIndex::UInt128, "bytes"},
{TypeIndex::Int256, "bytes"},
{TypeIndex::UInt256, "bytes"},
{TypeIndex::Float32, "float"},
{TypeIndex::Float64, "double"},
{TypeIndex::Decimal32, "bytes"},
{TypeIndex::Decimal64, "bytes"},
{TypeIndex::Decimal128, "bytes"},
{TypeIndex::Decimal256, "bytes"},
{TypeIndex::String, "bytes"},
{TypeIndex::FixedString, "bytes"},
{TypeIndex::UUID, "bytes"},
{TypeIndex::Date, "uint32"},
{TypeIndex::Date32, "int32"},
{TypeIndex::DateTime, "uint32"},
{TypeIndex::DateTime64, "uint64"},
{TypeIndex::IPv4, "uint32"},
{TypeIndex::IPv6, "bytes"},
};
void writeProtobufHeader(WriteBuffer & buf)
{
writeCString("syntax = \"proto3\";\n\n", buf);
}
void startEnum(WriteBuffer & buf, const String & enum_name, size_t indent)
{
startNested(buf, enum_name, "enum", indent);
}
void startMessage(WriteBuffer & buf, const String & message_name, size_t indent)
{
startNested(buf, message_name, "message", indent);
}
void writeFieldDefinition(WriteBuffer & buf, const String & type_name, const String & column_name, size_t & field_index, size_t indent)
{
writeIndent(buf, indent);
writeString(fmt::format("{} {} = {};\n", type_name, getSchemaFieldName(column_name), field_index++), buf);
}
String prepareAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent);
void writeProtobufField(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t & field_index, size_t indent)
{
auto field_type_name = prepareAndGetProtobufTypeName(buf, data_type, column_name, indent);
writeFieldDefinition(buf, field_type_name, column_name, field_index, indent);
}
String prepareArrayAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
const auto & nested_type = assert_cast<const DataTypeArray &>(*data_type).getNestedType();
/// Simple case when we can just use 'repeated <nested_type>'.
if (!isArray(nested_type) && !isMap(nested_type))
{
auto nested_type_name = prepareAndGetProtobufTypeName(buf, nested_type, column_name, indent);
return "repeated " + nested_type_name;
}
/// Protobuf doesn't support multidimensional repeated fields and repeated maps.
/// When we have Array(Array(...)) or Array(Map(...)) we should place nested type into a nested Message with one field.
String message_name = getSchemaMessageName(column_name);
startMessage(buf, message_name, indent);
size_t nested_field_index = 1;
writeProtobufField(buf, nested_type, column_name, nested_field_index, indent + 1);
endNested(buf, indent);
return "repeated " + message_name;
}
String prepareTupleAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
const auto & tuple_type = assert_cast<const DataTypeTuple &>(*data_type);
auto nested_names_and_types = getCollectedTupleElements(tuple_type);
String message_name = getSchemaMessageName(column_name);
startMessage(buf, message_name, indent);
size_t nested_field_index = 1;
for (const auto & [name, type] : nested_names_and_types)
writeProtobufField(buf, type, name, nested_field_index, indent + 1);
endNested(buf, indent);
return message_name;
}
String prepareMapAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
const auto & map_type = assert_cast<const DataTypeMap &>(*data_type);
const auto & key_type = map_type.getKeyType();
const auto & value_type = map_type.getValueType();
auto it = protobuf_simple_type_names.find(key_type->getTypeId());
if (it == protobuf_simple_type_names.end())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type {} is not supported for conversion into Map key in Protobuf schema", data_type->getName());
auto key_type_name = it->second;
/// Protobuf map type doesn't support "bytes" type as a key. Change it to "string"
if (key_type_name == "bytes")
key_type_name = "string";
/// Special cases when value type is Array or Map, because Protobuf
/// doesn't support syntax "map<Key, repeated Value>" and "map<Key, map<..., ...>>"
/// In this case we should place it into a nested Message with one field.
String value_type_name;
if (isArray(value_type) || isMap(value_type))
{
value_type_name = getSchemaMessageName(column_name) + "Value";
startMessage(buf, value_type_name, indent);
size_t nested_field_index = 1;
writeProtobufField(buf, value_type, column_name + "Value", nested_field_index, indent + 1);
endNested(buf, indent);
}
else
{
value_type_name = prepareAndGetProtobufTypeName(buf, value_type, column_name + "Value", indent);
}
return fmt::format("map<{}, {}>", key_type_name, value_type_name);
}
template <typename EnumType>
String prepareEnumAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
const auto & enum_type = assert_cast<const DataTypeEnum<EnumType> &>(*data_type);
String enum_name = getSchemaMessageName(column_name);
startEnum(buf, enum_name, indent);
const auto & names = enum_type.getAllRegisteredNames();
for (size_t i = 0; i != names.size(); ++i)
{
writeIndent(buf, indent + 1);
writeString(fmt::format("{} = {};\n", names[i], std::to_string(i)), buf);
}
endNested(buf, indent);
return enum_name;
}
String prepareAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
TypeIndex type_id = data_type->getTypeId();
switch (data_type->getTypeId())
{
case TypeIndex::Nullable:
return prepareAndGetProtobufTypeName(buf, assert_cast<const DataTypeNullable &>(*data_type).getNestedType(), column_name, indent);
case TypeIndex::LowCardinality:
return prepareAndGetProtobufTypeName(buf, assert_cast<const DataTypeLowCardinality &>(*data_type).getDictionaryType(), column_name, indent);
case TypeIndex::Array:
return prepareArrayAndGetProtobufTypeName(buf, data_type, column_name, indent);
case TypeIndex::Tuple:
return prepareTupleAndGetProtobufTypeName(buf, data_type, column_name, indent);
case TypeIndex::Map:
return prepareMapAndGetProtobufTypeName(buf, data_type, column_name, indent);
case TypeIndex::Enum8:
return prepareEnumAndGetProtobufTypeName<Int8>(buf, data_type, column_name, indent);
case TypeIndex::Enum16:
return prepareEnumAndGetProtobufTypeName<Int16>(buf, data_type, column_name, indent);
default:
{
if (isBool(data_type))
return "bool";
auto it = protobuf_simple_type_names.find(type_id);
if (it == protobuf_simple_type_names.end())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type {} is not supported for conversion into Protobuf schema", data_type->getName());
return it->second;
}
}
}
}
void StructureToProtobufSchema::writeSchema(WriteBuffer & buf, const String & message_name, const NamesAndTypesList & names_and_types_)
{
auto names_and_types = collectNested(names_and_types_);
writeProtobufHeader(buf);
startMessage(buf, getSchemaMessageName(message_name), 0);
size_t field_index = 1;
for (const auto & [column_name, data_type] : names_and_types)
writeProtobufField(buf, data_type, column_name, field_index, 1);
endNested(buf, 0);
}
}

View File

@ -0,0 +1,16 @@
#pragma once
#include <IO/WriteBuffer.h>
#include <Core/NamesAndTypes.h>
namespace DB
{
struct StructureToProtobufSchema
{
static constexpr auto name = "structureToProtobufSchema";
static void writeSchema(WriteBuffer & buf, const String & message_name, const NamesAndTypesList & names_and_types_);
};
}

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

@ -0,0 +1,145 @@
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeEnum.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Interpreters/parseColumnsListForTableFunction.h>
#include <Interpreters/Context.h>
#include <IO/WriteBufferFromVector.h>
#include <Formats/StructureToCapnProtoSchema.h>
#include <Formats/StructureToProtobufSchema.h>
#include <Common/randomSeed.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
template <class Impl>
class FunctionStructureToFormatSchema : public IFunction
{
public:
static constexpr auto name = Impl::name;
explicit FunctionStructureToFormatSchema(ContextPtr context_) : context(std::move(context_))
{
}
static FunctionPtr create(ContextPtr ctx)
{
return std::make_shared<FunctionStructureToFormatSchema>(std::move(ctx));
}
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 0; }
bool isVariadic() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; }
bool useDefaultImplementationForConstants() const override { return false; }
bool useDefaultImplementationForNulls() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.empty() || arguments.size() > 2)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, expected 1 or 2",
getName(), arguments.size());
if (!isString(arguments[0]))
{
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of the first argument of function {}, expected constant string",
arguments[0]->getName(),
getName());
}
if (arguments.size() > 1 && !isString(arguments[1]))
{
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of the second argument of function {}, expected constant string",
arguments[1]->getName(),
getName());
}
return std::make_shared<DataTypeString>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
if (arguments.empty() || arguments.size() > 2)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, expected 1 or 2",
getName(), arguments.size());
String structure = arguments[0].column->getDataAt(0).toString();
String message_name = arguments.size() == 2 ? arguments[1].column->getDataAt(0).toString() : "Message";
auto columns_list = parseColumnsListFromString(structure, context);
auto col_res = ColumnString::create();
auto & data = assert_cast<ColumnString &>(*col_res).getChars();
WriteBufferFromVector buf(data);
Impl::writeSchema(buf, message_name, columns_list.getAll());
buf.finalize();
auto & offsets = assert_cast<ColumnString &>(*col_res).getOffsets();
offsets.push_back(data.size());
return ColumnConst::create(std::move(col_res), input_rows_count);
}
private:
ContextPtr context;
};
REGISTER_FUNCTION(StructureToCapnProtoSchema)
{
factory.registerFunction<FunctionStructureToFormatSchema<StructureToCapnProtoSchema>>(FunctionDocumentation
{
.description=R"(
Function that converts ClickHouse table structure to CapnProto format schema
)",
.examples{
{"random", "SELECT structureToCapnProtoSchema('s String, x UInt32', 'MessageName') format TSVRaw", "struct MessageName\n"
"{\n"
" s @0 : Data;\n"
" x @1 : UInt32;\n"
"}"},
},
.categories{"Other"}
},
FunctionFactory::CaseSensitive);
}
REGISTER_FUNCTION(StructureToProtobufSchema)
{
factory.registerFunction<FunctionStructureToFormatSchema<StructureToProtobufSchema>>(FunctionDocumentation
{
.description=R"(
Function that converts ClickHouse table structure to Protobuf format schema
)",
.examples{
{"random", "SELECT structureToCapnProtoSchema('s String, x UInt32', 'MessageName') format TSVRaw", "syntax = \"proto3\";\n"
"\n"
"message MessageName\n"
"{\n"
" bytes s = 1;\n"
" uint32 x = 2;\n"
"}"},
},
.categories{"Other"}
},
FunctionFactory::CaseSensitive);
}
}

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

@ -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

@ -764,7 +764,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
/// Table function without columns list.
auto table_function_ast = create.as_table_function->ptr();
auto table_function = TableFunctionFactory::instance().get(table_function_ast, getContext());
properties.columns = table_function->getActualTableStructure(getContext());
properties.columns = table_function->getActualTableStructure(getContext(), /*is_insert_query*/ true);
}
else if (create.is_dictionary)
{

View File

@ -96,7 +96,7 @@ BlockIO InterpreterDescribeQuery::execute()
else if (table_expression.table_function)
{
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, getContext());
auto table_function_column_descriptions = table_function_ptr->getActualTableStructure(getContext());
auto table_function_column_descriptions = table_function_ptr->getActualTableStructure(getContext(), /*is_insert_query*/ true);
for (const auto & table_function_column_description : table_function_column_descriptions)
columns.emplace_back(table_function_column_description);
}

View File

@ -1034,7 +1034,7 @@ std::shared_ptr<Block> MergeJoin::loadRightBlock(size_t pos) const
{
auto load_func = [&]() -> std::shared_ptr<Block>
{
TemporaryFileStreamLegacy input(flushed_right_blocks[pos]->getPath(), materializeBlock(right_sample_block));
TemporaryFileStreamLegacy input(flushed_right_blocks[pos]->getAbsolutePath(), materializeBlock(right_sample_block));
return std::make_shared<Block>(input.block_in->read());
};

View File

@ -39,7 +39,7 @@ namespace
TemporaryFileOnDiskHolder flushToFile(const DiskPtr & disk, const Block & header, QueryPipelineBuilder pipeline, const String & codec)
{
auto tmp_file = std::make_unique<TemporaryFileOnDisk>(disk, CurrentMetrics::TemporaryFilesForJoin);
auto write_stat = TemporaryFileStreamLegacy::write(tmp_file->getPath(), header, std::move(pipeline), codec);
auto write_stat = TemporaryFileStreamLegacy::write(tmp_file->getAbsolutePath(), header, std::move(pipeline), codec);
ProfileEvents::increment(ProfileEvents::ExternalProcessingCompressedBytesTotal, write_stat.compressed_bytes);
ProfileEvents::increment(ProfileEvents::ExternalProcessingUncompressedBytesTotal, write_stat.uncompressed_bytes);
@ -267,7 +267,7 @@ SortedBlocksWriter::SortedFiles SortedBlocksWriter::finishMerge(std::function<vo
Pipe SortedBlocksWriter::streamFromFile(const TmpFilePtr & file) const
{
return Pipe(std::make_shared<TemporaryFileLazySource>(file->getPath(), materializeBlock(sample_block)));
return Pipe(std::make_shared<TemporaryFileLazySource>(file->getAbsolutePath(), materializeBlock(sample_block)));
}

View File

@ -235,9 +235,9 @@ TemporaryFileStream::TemporaryFileStream(TemporaryFileOnDiskHolder file_, const
: parent(parent_)
, header(header_)
, file(std::move(file_))
, out_writer(std::make_unique<OutputWriter>(std::make_unique<WriteBufferFromFile>(file->getPath()), header))
, out_writer(std::make_unique<OutputWriter>(std::make_unique<WriteBufferFromFile>(file->getAbsolutePath()), header))
{
LOG_TEST(&Poco::Logger::get("TemporaryFileStream"), "Writing to temporary file {}", file->getPath());
LOG_TEST(&Poco::Logger::get("TemporaryFileStream"), "Writing to temporary file {}", file->getAbsolutePath());
}
TemporaryFileStream::TemporaryFileStream(FileSegmentsHolderPtr segments_, const Block & header_, TemporaryDataOnDisk * parent_)
@ -365,7 +365,7 @@ void TemporaryFileStream::release()
String TemporaryFileStream::getPath() const
{
if (file)
return file->getPath();
return file->getAbsolutePath();
if (segment_holder && !segment_holder->empty())
return segment_holder->front().getPathInLocalCache();

View File

@ -17,12 +17,12 @@ namespace ErrorCodes
extern const int INCORRECT_DATA;
}
CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings)
CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header_, Params params_, const CapnProtoSchemaInfo & info, const FormatSettings & format_settings)
: IRowInputFormat(std::move(header_), in_, std::move(params_))
, parser(std::make_shared<CapnProtoSchemaParser>())
{
// Parse the schema and fetch the root object
schema = parser->getMessageSchema(info);
schema = parser->getMessageSchema(info.getSchemaInfo());
const auto & header = getPort().getHeader();
serializer = std::make_unique<CapnProtoSerializer>(header.getDataTypes(), header.getNames(), schema, format_settings.capn_proto);
}
@ -106,8 +106,12 @@ void registerInputFormatCapnProto(FormatFactory & factory)
"CapnProto",
[](ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings)
{
return std::make_shared<CapnProtoRowInputFormat>(buf, sample, std::move(params),
FormatSchemaInfo(settings, "CapnProto", true), settings);
return std::make_shared<CapnProtoRowInputFormat>(
buf,
sample,
std::move(params),
CapnProtoSchemaInfo(settings, "CapnProto", sample, settings.capn_proto.use_autogenerated_schema),
settings);
});
factory.markFormatSupportsSubsetOfColumns("CapnProto");
factory.registerFileExtension("capnp", "CapnProto");

View File

@ -24,7 +24,7 @@ class ReadBuffer;
class CapnProtoRowInputFormat final : public IRowInputFormat
{
public:
CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_);
CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const CapnProtoSchemaInfo & info, const FormatSettings & format_settings);
String getName() const override { return "CapnProtoRowInputFormat"; }

View File

@ -23,14 +23,14 @@ void CapnProtoOutputStream::write(const void * buffer, size_t size)
CapnProtoRowOutputFormat::CapnProtoRowOutputFormat(
WriteBuffer & out_,
const Block & header_,
const FormatSchemaInfo & info,
const CapnProtoSchemaInfo & info,
const FormatSettings & format_settings)
: IRowOutputFormat(header_, out_)
, column_names(header_.getNames())
, column_types(header_.getDataTypes())
, output_stream(std::make_unique<CapnProtoOutputStream>(out_))
{
schema = schema_parser.getMessageSchema(info);
schema = schema_parser.getMessageSchema(info.getSchemaInfo());
const auto & header = getPort(PortKind::Main).getHeader();
serializer = std::make_unique<CapnProtoSerializer>(header.getDataTypes(), header.getNames(), schema, format_settings.capn_proto);
capnp::MallocMessageBuilder message;
@ -52,7 +52,11 @@ void registerOutputFormatCapnProto(FormatFactory & factory)
const Block & sample,
const FormatSettings & format_settings)
{
return std::make_shared<CapnProtoRowOutputFormat>(buf, sample, FormatSchemaInfo(format_settings, "CapnProto", true), format_settings);
return std::make_shared<CapnProtoRowOutputFormat>(
buf,
sample,
CapnProtoSchemaInfo(format_settings, "CapnProto", sample, format_settings.capn_proto.use_autogenerated_schema),
format_settings);
});
}

View File

@ -31,8 +31,8 @@ public:
CapnProtoRowOutputFormat(
WriteBuffer & out_,
const Block & header_,
const FormatSchemaInfo & info,
const FormatSettings & format_settings_);
const CapnProtoSchemaInfo & info,
const FormatSettings & format_settings);
String getName() const override { return "CapnProtoRowOutputFormat"; }

View File

@ -14,7 +14,7 @@ ProtobufListInputFormat::ProtobufListInputFormat(
ReadBuffer & in_,
const Block & header_,
const Params & params_,
const FormatSchemaInfo & schema_info_,
const ProtobufSchemaInfo & schema_info_,
bool flatten_google_wrappers_)
: IRowInputFormat(header_, in_, params_)
, reader(std::make_unique<ProtobufReader>(in_))
@ -22,7 +22,7 @@ ProtobufListInputFormat::ProtobufListInputFormat(
header_.getNames(),
header_.getDataTypes(),
missing_column_indices,
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::Yes),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::Yes),
/* with_length_delimiter = */ true,
/* with_envelope = */ true,
flatten_google_wrappers_,
@ -84,7 +84,7 @@ void registerInputFormatProtobufList(FormatFactory & factory)
const FormatSettings & settings)
{
return std::make_shared<ProtobufListInputFormat>(buf, sample, std::move(params),
FormatSchemaInfo(settings, "Protobuf", true), settings.protobuf.input_flatten_google_wrappers);
ProtobufSchemaInfo(settings, "Protobuf", sample, settings.protobuf.use_autogenerated_schema), settings.protobuf.input_flatten_google_wrappers);
});
factory.markFormatSupportsSubsetOfColumns("ProtobufList");
factory.registerAdditionalInfoForSchemaCacheGetter(

View File

@ -28,7 +28,7 @@ public:
ReadBuffer & in_,
const Block & header_,
const Params & params_,
const FormatSchemaInfo & schema_info_,
const ProtobufSchemaInfo & schema_info_,
bool flatten_google_wrappers_);
String getName() const override { return "ProtobufListInputFormat"; }

View File

@ -2,7 +2,6 @@
#if USE_PROTOBUF
# include <Formats/FormatFactory.h>
# include <Formats/FormatSchemaInfo.h>
# include <Formats/ProtobufWriter.h>
# include <Formats/ProtobufSerializer.h>
# include <Formats/ProtobufSchemas.h>
@ -13,14 +12,14 @@ namespace DB
ProtobufListOutputFormat::ProtobufListOutputFormat(
WriteBuffer & out_,
const Block & header_,
const FormatSchemaInfo & schema_info_,
const ProtobufSchemaInfo & schema_info_,
bool defaults_for_nullable_google_wrappers_)
: IRowOutputFormat(header_, out_)
, writer(std::make_unique<ProtobufWriter>(out))
, serializer(ProtobufSerializer::create(
header_.getNames(),
header_.getDataTypes(),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::Yes),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::Yes),
/* with_length_delimiter = */ true,
/* with_envelope = */ true,
defaults_for_nullable_google_wrappers_,
@ -55,7 +54,7 @@ void registerOutputFormatProtobufList(FormatFactory & factory)
const FormatSettings & settings)
{
return std::make_shared<ProtobufListOutputFormat>(
buf, header, FormatSchemaInfo(settings, "Protobuf", true),
buf, header, ProtobufSchemaInfo(settings, "Protobuf", header, settings.protobuf.use_autogenerated_schema),
settings.protobuf.output_nullables_with_google_wrappers);
});
}

View File

@ -4,10 +4,10 @@
#if USE_PROTOBUF
# include <Processors/Formats/IRowOutputFormat.h>
# include <Formats/FormatSchemaInfo.h>
namespace DB
{
class FormatSchemaInfo;
class ProtobufWriter;
class ProtobufSerializer;
@ -26,7 +26,7 @@ public:
ProtobufListOutputFormat(
WriteBuffer & out_,
const Block & header_,
const FormatSchemaInfo & schema_info_,
const ProtobufSchemaInfo & schema_info_,
bool defaults_for_nullable_google_wrappers_);
String getName() const override { return "ProtobufListOutputFormat"; }

View File

@ -11,9 +11,9 @@ namespace DB
{
ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_,
const FormatSchemaInfo & schema_info_, bool with_length_delimiter_, bool flatten_google_wrappers_)
const ProtobufSchemaInfo & schema_info_, bool with_length_delimiter_, bool flatten_google_wrappers_)
: IRowInputFormat(header_, in_, params_)
, message_descriptor(ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::No))
, message_descriptor(ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::No))
, with_length_delimiter(with_length_delimiter_)
, flatten_google_wrappers(flatten_google_wrappers_)
{
@ -89,7 +89,7 @@ void registerInputFormatProtobuf(FormatFactory & factory)
const FormatSettings & settings)
{
return std::make_shared<ProtobufRowInputFormat>(buf, sample, std::move(params),
FormatSchemaInfo(settings, "Protobuf", true),
ProtobufSchemaInfo(settings, "Protobuf", sample, settings.protobuf.use_autogenerated_schema),
with_length_delimiter,
settings.protobuf.input_flatten_google_wrappers);
});

View File

@ -33,7 +33,7 @@ public:
ReadBuffer & in_,
const Block & header_,
const Params & params_,
const FormatSchemaInfo & schema_info_,
const ProtobufSchemaInfo & schema_info_,
bool with_length_delimiter_,
bool flatten_google_wrappers_);

View File

@ -3,7 +3,6 @@
#if USE_PROTOBUF
# include <Formats/FormatFactory.h>
# include <Core/Block.h>
# include <Formats/FormatSchemaInfo.h>
# include <Formats/FormatSettings.h>
# include <Formats/ProtobufSchemas.h>
# include <Formats/ProtobufSerializer.h>
@ -20,7 +19,7 @@ namespace ErrorCodes
ProtobufRowOutputFormat::ProtobufRowOutputFormat(
WriteBuffer & out_,
const Block & header_,
const FormatSchemaInfo & schema_info_,
const ProtobufSchemaInfo & schema_info_,
const FormatSettings & settings_,
bool with_length_delimiter_)
: IRowOutputFormat(header_, out_)
@ -28,7 +27,7 @@ ProtobufRowOutputFormat::ProtobufRowOutputFormat(
, serializer(ProtobufSerializer::create(
header_.getNames(),
header_.getDataTypes(),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::No),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::No),
with_length_delimiter_,
/* with_envelope = */ false,
settings_.protobuf.output_nullables_with_google_wrappers,
@ -61,7 +60,7 @@ void registerOutputFormatProtobuf(FormatFactory & factory)
const FormatSettings & settings)
{
return std::make_shared<ProtobufRowOutputFormat>(
buf, header, FormatSchemaInfo(settings, "Protobuf", true),
buf, header, ProtobufSchemaInfo(settings, "Protobuf", header, settings.protobuf.use_autogenerated_schema),
settings, with_length_delimiter);
});
}

View File

@ -4,11 +4,11 @@
#if USE_PROTOBUF
# include <Processors/Formats/IRowOutputFormat.h>
# include <Formats/FormatSchemaInfo.h>
namespace DB
{
class DB;
class FormatSchemaInfo;
class ProtobufSerializer;
class ProtobufWriter;
class WriteBuffer;
@ -30,7 +30,7 @@ public:
ProtobufRowOutputFormat(
WriteBuffer & out_,
const Block & header_,
const FormatSchemaInfo & schema_info_,
const ProtobufSchemaInfo & schema_info_,
const FormatSettings & settings_,
bool with_length_delimiter_);

View File

@ -5,6 +5,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int SET_SIZE_LIMIT_EXCEEDED;
}
@ -126,9 +127,20 @@ bool DistinctSortedChunkTransform::isKey(const size_t key_pos, const size_t row_
bool DistinctSortedChunkTransform::isLatestKeyFromPrevChunk(const size_t row_pos) const
{
for (size_t i = 0; i < sorted_columns.size(); ++i)
for (size_t i = 0, s = sorted_columns.size(); i < s; ++i)
{
const int res = prev_chunk_latest_key[i]->compareAt(0, row_pos, *sorted_columns[i], sorted_columns_descr[i].nulls_direction);
const auto & sorted_column = *sorted_columns[i];
/// temporary hardening due to suspious crashes in sqlancer tests
if (unlikely(sorted_column.size() <= row_pos))
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Unexpected size of a sorted column: size {}, row_pos {}, column position {}, type {}",
sorted_column.size(),
row_pos,
i,
sorted_column.getFamilyName());
const int res = prev_chunk_latest_key[i]->compareAt(0, row_pos, sorted_column, sorted_columns_descr[i].nulls_direction);
if (res != 0)
return false;
}

View File

@ -159,7 +159,7 @@ void PartialSortingTransform::transform(Chunk & chunk)
{
MutableColumnPtr sort_description_threshold_column_updated = raw_block_columns[i]->cloneEmpty();
sort_description_threshold_column_updated->insertFrom(*raw_block_columns[i], min_row_to_compare);
sort_description_threshold_columns_updated[i] = std::move(sort_description_threshold_column_updated);
sort_description_threshold_columns_updated[i] = sort_description_threshold_column_updated->convertToFullColumnIfSparse();
}
sort_description_threshold_columns = std::move(sort_description_threshold_columns_updated);

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

@ -250,15 +250,16 @@ StorageKafka::StorageKafka(
: IStorage(table_id_)
, WithContext(context_->getGlobalContext())
, kafka_settings(std::move(kafka_settings_))
, topics(parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value)))
, brokers(getContext()->getMacros()->expand(kafka_settings->kafka_broker_list.value))
, group(getContext()->getMacros()->expand(kafka_settings->kafka_group_name.value))
, macros_info{.table_id = table_id_}
, topics(parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info)))
, brokers(getContext()->getMacros()->expand(kafka_settings->kafka_broker_list.value, macros_info))
, group(getContext()->getMacros()->expand(kafka_settings->kafka_group_name.value, macros_info))
, client_id(
kafka_settings->kafka_client_id.value.empty() ? getDefaultClientId(table_id_)
: getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value))
: getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value, macros_info))
, format_name(getContext()->getMacros()->expand(kafka_settings->kafka_format.value))
, max_rows_per_message(kafka_settings->kafka_max_rows_per_message.value)
, schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value))
, schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value, macros_info))
, num_consumers(kafka_settings->kafka_num_consumers.value)
, log(&Poco::Logger::get("StorageKafka (" + table_id_.table_name + ")"))
, semaphore(0, static_cast<int>(num_consumers))

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Macros.h>
#include <Core/BackgroundSchedulePool.h>
#include <Storages/IStorage.h>
#include <Storages/Kafka/KafkaConsumer.h>
@ -79,6 +80,7 @@ public:
private:
// Configuration and state
std::unique_ptr<KafkaSettings> kafka_settings;
Macros::MacroExpansionInfo macros_info;
const Names topics;
const String brokers;
const String group;

View File

@ -350,7 +350,7 @@ void DataPartStorageOnDiskBase::backup(
temp_dir_it = temp_dirs->emplace(disk, std::make_shared<TemporaryFileOnDisk>(disk, "tmp/")).first;
temp_dir_owner = temp_dir_it->second;
fs::path temp_dir = temp_dir_owner->getPath();
fs::path temp_dir = temp_dir_owner->getRelativePath();
temp_part_dir = temp_dir / part_path_in_backup.relative_path();
disk->createDirectories(temp_part_dir);
}

View File

@ -5266,7 +5266,7 @@ public:
auto it = temp_dirs.find(disk);
if (it == temp_dirs.end())
it = temp_dirs.emplace(disk, std::make_shared<TemporaryFileOnDisk>(disk, "tmp/")).first;
return it->second->getPath();
return it->second->getRelativePath();
}
private:

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

@ -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

@ -86,7 +86,7 @@ const std::unordered_set<std::string_view> optional_configuration_keys = {
bool isConnectionString(const std::string & candidate)
{
return candidate.starts_with("DefaultEndpointsProtocol");
return !candidate.starts_with("http");
}
}
@ -257,7 +257,7 @@ void registerStorageAzureBlob(StorageFactory & factory)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments");
auto configuration = StorageAzureBlob::getConfiguration(engine_args, args.getLocalContext());
auto client = StorageAzureBlob::createClient(configuration);
auto client = StorageAzureBlob::createClient(configuration, /* is_read_only */ false);
// Use format settings from global server context + settings from
// the SETTINGS clause of the create query. Settings from current
// session and user are ignored.
@ -309,58 +309,113 @@ void registerStorageAzureBlob(StorageFactory & factory)
});
}
AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration configuration)
static bool containerExists(std::unique_ptr<BlobServiceClient> &blob_service_client, std::string container_name)
{
Azure::Storage::Blobs::ListBlobContainersOptions options;
options.Prefix = container_name;
options.PageSizeHint = 1;
auto containers_list_response = blob_service_client->ListBlobContainers(options);
auto containers_list = containers_list_response.BlobContainers;
for (const auto & container : containers_list)
{
if (container_name == container.Name)
return true;
}
return false;
}
AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration configuration, bool is_read_only)
{
AzureClientPtr result;
if (configuration.is_connection_string)
{
std::unique_ptr<BlobServiceClient> blob_service_client = std::make_unique<BlobServiceClient>(BlobServiceClient::CreateFromConnectionString(configuration.connection_url));
result = std::make_unique<BlobContainerClient>(BlobContainerClient::CreateFromConnectionString(configuration.connection_url, configuration.container));
result->CreateIfNotExists();
}
else
{
if (configuration.account_name.has_value() && configuration.account_key.has_value())
bool container_exists = containerExists(blob_service_client,configuration.container);
if (!container_exists)
{
auto storage_shared_key_credential = std::make_shared<Azure::Storage::StorageSharedKeyCredential>(*configuration.account_name, *configuration.account_key);
auto blob_service_client = std::make_unique<BlobServiceClient>(configuration.connection_url, storage_shared_key_credential);
if (is_read_only)
throw Exception(
ErrorCodes::DATABASE_ACCESS_DENIED,
"AzureBlobStorage container does not exist '{}'",
configuration.container);
try
{
result = std::make_unique<BlobContainerClient>(blob_service_client->CreateBlobContainer(configuration.container).Value);
}
catch (const Azure::Storage::StorageException & e)
result->CreateIfNotExists();
} catch (const Azure::Storage::StorageException & e)
{
if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict)
{
auto final_url = configuration.connection_url
+ (configuration.connection_url.back() == '/' ? "" : "/")
+ configuration.container;
result = std::make_unique<BlobContainerClient>(final_url, storage_shared_key_credential);
}
else
if (!(e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict
&& e.ReasonPhrase == "The specified container already exists."))
{
throw;
}
}
}
}
else
{
std::shared_ptr<Azure::Storage::StorageSharedKeyCredential> storage_shared_key_credential;
if (configuration.account_name.has_value() && configuration.account_key.has_value())
{
storage_shared_key_credential
= std::make_shared<Azure::Storage::StorageSharedKeyCredential>(*configuration.account_name, *configuration.account_key);
}
std::unique_ptr<BlobServiceClient> blob_service_client;
if (storage_shared_key_credential)
{
blob_service_client = std::make_unique<BlobServiceClient>(configuration.connection_url, storage_shared_key_credential);
}
else
{
auto managed_identity_credential = std::make_shared<Azure::Identity::ManagedIdentityCredential>();
auto blob_service_client = std::make_unique<BlobServiceClient>(configuration.connection_url, managed_identity_credential);
blob_service_client = std::make_unique<BlobServiceClient>(configuration.connection_url);
}
bool container_exists = containerExists(blob_service_client,configuration.container);
std::string final_url;
size_t pos = configuration.connection_url.find('?');
if (pos != std::string::npos)
{
auto url_without_sas = configuration.connection_url.substr(0, pos);
final_url = url_without_sas + (url_without_sas.back() == '/' ? "" : "/") + configuration.container
+ configuration.connection_url.substr(pos);
}
else
final_url
= configuration.connection_url + (configuration.connection_url.back() == '/' ? "" : "/") + configuration.container;
if (container_exists)
{
if (storage_shared_key_credential)
result = std::make_unique<BlobContainerClient>(final_url, storage_shared_key_credential);
else
result = std::make_unique<BlobContainerClient>(final_url);
}
else
{
if (is_read_only)
throw Exception(
ErrorCodes::DATABASE_ACCESS_DENIED,
"AzureBlobStorage container does not exist '{}'",
configuration.container);
try
{
result = std::make_unique<BlobContainerClient>(blob_service_client->CreateBlobContainer(configuration.container).Value);
}
catch (const Azure::Storage::StorageException & e)
} catch (const Azure::Storage::StorageException & e)
{
if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict)
if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict
&& e.ReasonPhrase == "The specified container already exists.")
{
auto final_url = configuration.connection_url
+ (configuration.connection_url.back() == '/' ? "" : "/")
+ configuration.container;
result = std::make_unique<BlobContainerClient>(final_url, managed_identity_credential);
if (storage_shared_key_credential)
result = std::make_unique<BlobContainerClient>(final_url, storage_shared_key_credential);
else
result = std::make_unique<BlobContainerClient>(final_url);
}
else
{
@ -438,7 +493,7 @@ void StorageAzureBlob::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont
{
throw Exception(
ErrorCodes::DATABASE_ACCESS_DENIED,
"S3 key '{}' contains globs, so the table is in readonly mode",
"AzureBlobStorage key '{}' contains globs, so the table is in readonly mode",
configuration.blob_path);
}
@ -1203,7 +1258,7 @@ ColumnsDescription StorageAzureBlob::getTableStructureFromData(
return nullptr;
}
/// S3 file iterator could get new keys after new iteration, check them in schema cache.
///AzureBlobStorage file iterator could get new keys after new iteration, check them in schema cache.
if (ctx->getSettingsRef().schema_inference_use_cache_for_azure && read_keys.size() > prev_read_keys_size)
{
columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), configuration, format_settings, ctx);

View File

@ -65,7 +65,7 @@ public:
ASTPtr partition_by_);
static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context);
static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration);
static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration, bool is_read_only);
static AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context);

View File

@ -946,7 +946,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c
fs::path data_path_in_backup_fs = data_path_in_backup;
auto temp_dir_owner = std::make_shared<TemporaryFileOnDisk>(disk, "tmp/");
fs::path temp_dir = temp_dir_owner->getPath();
fs::path temp_dir = temp_dir_owner->getRelativePath();
disk->createDirectories(temp_dir);
bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks;

View File

@ -314,7 +314,7 @@ namespace
backup_entries.resize(file_paths.size());
temp_dir_owner.emplace(temp_disk);
fs::path temp_dir = temp_dir_owner->getPath();
fs::path temp_dir = temp_dir_owner->getRelativePath();
temp_disk->createDirectories(temp_dir);
/// Writing data.bin
@ -453,10 +453,10 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat
if (!dynamic_cast<ReadBufferFromFileBase *>(in.get()))
{
temp_data_file.emplace(temporary_disk);
auto out = std::make_unique<WriteBufferFromFile>(temp_data_file->getPath());
auto out = std::make_unique<WriteBufferFromFile>(temp_data_file->getAbsolutePath());
copyData(*in, *out);
out.reset();
in = createReadBufferFromFileBase(temp_data_file->getPath(), {});
in = createReadBufferFromFileBase(temp_data_file->getAbsolutePath(), {});
}
std::unique_ptr<ReadBufferFromFileBase> in_from_file{static_cast<ReadBufferFromFileBase *>(in.release())};
CompressedReadBufferFromFile compressed_in{std::move(in_from_file)};

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

@ -544,7 +544,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec
fs::path data_path_in_backup_fs = data_path_in_backup;
auto temp_dir_owner = std::make_shared<TemporaryFileOnDisk>(disk, "tmp/");
fs::path temp_dir = temp_dir_owner->getPath();
fs::path temp_dir = temp_dir_owner->getRelativePath();
disk->createDirectories(temp_dir);
bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks;

View File

@ -371,7 +371,7 @@ std::pair<Poco::URI, std::unique_ptr<ReadWriteBufferFromHTTP>> StorageURLSource:
for (; option != end; ++option)
{
bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end);
auto request_uri = Poco::URI(*option, context->getSettingsRef().disable_url_encoding);
auto request_uri = Poco::URI(*option, context->getSettingsRef().enable_url_encoding);
for (const auto & [param, value] : params)
request_uri.addQueryParameter(param, value);

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