mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-30 19:42:00 +00:00
Merge branch 'master' into fix-misleading-error
This commit is contained in:
commit
6eebbab90f
@ -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));
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -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,
|
||||
|
@ -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 \
|
||||
|
@ -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.
|
||||
|
@ -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}
|
||||
|
||||
|
224
docs/en/engines/table-engines/integrations/s3queue.md
Normal file
224
docs/en/engines/table-engines/integrations/s3queue.md
Normal 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 `?`.
|
||||
:::
|
@ -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.
|
||||
|
@ -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.
|
||||
|
@ -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.
|
@ -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 [Cap’n 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 [Cap’n 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)
|
||||
|
@ -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 │
|
||||
└─────────────────────┴──────────────────────────┘
|
||||
```
|
||||
|
@ -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 it’s `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 it’s `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.
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
}
|
||||
```
|
||||
|
@ -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**
|
||||
|
||||
|
@ -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 │
|
||||
└─────────────────────┴──────────────────────────┘
|
||||
```
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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"; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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>(),
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
|
||||
};
|
||||
|
||||
}
|
@ -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>());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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,
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
@ -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>");
|
||||
}
|
||||
|
@ -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.
|
||||
|
@ -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}})
|
||||
}
|
||||
|
@ -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)
|
||||
|
||||
}
|
||||
|
@ -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_))
|
||||
{
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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>;
|
||||
|
||||
}
|
||||
|
@ -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>;
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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()), ", "),
|
||||
|
236
src/Formats/StructureToCapnProtoSchema.cpp
Normal file
236
src/Formats/StructureToCapnProtoSchema.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
16
src/Formats/StructureToCapnProtoSchema.h
Normal file
16
src/Formats/StructureToCapnProtoSchema.h
Normal 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_);
|
||||
};
|
||||
|
||||
}
|
117
src/Formats/StructureToFormatSchemaUtils.cpp
Normal file
117
src/Formats/StructureToFormatSchemaUtils.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
27
src/Formats/StructureToFormatSchemaUtils.h
Normal file
27
src/Formats/StructureToFormatSchemaUtils.h
Normal 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);
|
||||
}
|
||||
|
||||
}
|
214
src/Formats/StructureToProtobufSchema.cpp
Normal file
214
src/Formats/StructureToProtobufSchema.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
16
src/Formats/StructureToProtobufSchema.h
Normal file
16
src/Formats/StructureToProtobufSchema.h
Normal 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_);
|
||||
};
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
145
src/Functions/structureToFormatSchema.cpp
Normal file
145
src/Functions/structureToFormatSchema.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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());
|
||||
};
|
||||
|
||||
|
@ -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)));
|
||||
}
|
||||
|
||||
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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");
|
||||
|
@ -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"; }
|
||||
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -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"; }
|
||||
|
||||
|
@ -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(
|
||||
|
@ -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"; }
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
@ -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"; }
|
||||
|
@ -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);
|
||||
});
|
||||
|
@ -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_);
|
||||
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
@ -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_);
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
}
|
||||
|
||||
|
@ -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))
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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:
|
||||
|
@ -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 =
|
||||
{
|
||||
|
@ -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,
|
||||
|
351
src/Storages/S3Queue/S3QueueFilesMetadata.cpp
Normal file
351
src/Storages/S3Queue/S3QueueFilesMetadata.cpp
Normal 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
|
124
src/Storages/S3Queue/S3QueueFilesMetadata.h
Normal file
124
src/Storages/S3Queue/S3QueueFilesMetadata.h
Normal 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
|
41
src/Storages/S3Queue/S3QueueSettings.cpp
Normal file
41
src/Storages/S3Queue/S3QueueSettings.cpp
Normal 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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
46
src/Storages/S3Queue/S3QueueSettings.h
Normal file
46
src/Storages/S3Queue/S3QueueSettings.h
Normal 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);
|
||||
};
|
||||
|
||||
}
|
321
src/Storages/S3Queue/S3QueueSource.cpp
Normal file
321
src/Storages/S3Queue/S3QueueSource.cpp
Normal 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
|
124
src/Storages/S3Queue/S3QueueSource.h
Normal file
124
src/Storages/S3Queue/S3QueueSource.h
Normal 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
|
115
src/Storages/S3Queue/S3QueueTableMetadata.cpp
Normal file
115
src/Storages/S3Queue/S3QueueTableMetadata.cpp
Normal 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
|
43
src/Storages/S3Queue/S3QueueTableMetadata.h
Normal file
43
src/Storages/S3Queue/S3QueueTableMetadata.h
Normal 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
|
711
src/Storages/S3Queue/StorageS3Queue.cpp
Normal file
711
src/Storages/S3Queue/StorageS3Queue.cpp
Normal 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
|
146
src/Storages/S3Queue/StorageS3Queue.h
Normal file
146
src/Storages/S3Queue/StorageS3Queue.h
Normal 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
|
@ -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);
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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)};
|
||||
|
@ -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()
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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
Loading…
Reference in New Issue
Block a user