Merge branch 'master' into fix-30975

This commit is contained in:
Alexey Milovidov 2022-12-21 01:41:29 +03:00 committed by GitHub
commit ae7a7bc850
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
165 changed files with 5176 additions and 1712 deletions

View File

@ -16,6 +16,7 @@ on: # yamllint disable-line rule:truthy
- 'docker/docs/**'
- 'docs/**'
- 'website/**'
- 'utils/check-style/aspell-ignore/**'
jobs:
CheckLabels:
runs-on: [self-hosted, style-checker]

View File

@ -17,6 +17,7 @@ concurrency:
- 'docs/**'
- 'utils/list-versions/version_date.tsv'
- 'website/**'
- 'utils/check-style/aspell-ignore/**'
workflow_dispatch:
jobs:
DockerHubPushAarch64:

View File

@ -842,7 +842,7 @@ jobs:
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
BuilderBinAmd64SSE2:
BuilderBinAmd64Compat:
needs: [DockerHubPush]
runs-on: [self-hosted, builder]
steps:
@ -853,7 +853,7 @@ jobs:
IMAGES_PATH=${{runner.temp}}/images_path
REPO_COPY=${{runner.temp}}/build_check/ClickHouse
CACHES_PATH=${{runner.temp}}/../ccaches
BUILD_NAME=binary_amd64sse2
BUILD_NAME=binary_amd64_compat
EOF
- name: Download changed images
uses: actions/download-artifact@v2
@ -1017,7 +1017,7 @@ jobs:
- BuilderBinFreeBSD
# - BuilderBinGCC
- BuilderBinPPC64
- BuilderBinAmd64SSE2
- BuilderBinAmd64Compat
- BuilderBinAarch64V80Compat
- BuilderBinClangTidy
- BuilderDebShared

View File

@ -16,6 +16,7 @@ on: # yamllint disable-line rule:truthy
- 'docker/docs/**'
- 'docs/**'
- 'website/**'
- 'utils/check-style/aspell-ignore/**'
##########################################################################################
##################################### SMALL CHECKS #######################################
##########################################################################################
@ -900,7 +901,7 @@ jobs:
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
BuilderBinAmd64SSE2:
BuilderBinAmd64Compat:
needs: [DockerHubPush, FastTest, StyleCheck]
runs-on: [self-hosted, builder]
steps:
@ -911,7 +912,7 @@ jobs:
IMAGES_PATH=${{runner.temp}}/images_path
REPO_COPY=${{runner.temp}}/build_check/ClickHouse
CACHES_PATH=${{runner.temp}}/../ccaches
BUILD_NAME=binary_amd64sse2
BUILD_NAME=binary_amd64_compat
EOF
- name: Download changed images
uses: actions/download-artifact@v2
@ -1070,7 +1071,7 @@ jobs:
- BuilderBinFreeBSD
# - BuilderBinGCC
- BuilderBinPPC64
- BuilderBinAmd64SSE2
- BuilderBinAmd64Compat
- BuilderBinAarch64V80Compat
- BuilderBinClangTidy
- BuilderDebShared

3
.gitmodules vendored
View File

@ -269,9 +269,6 @@
[submodule "contrib/vectorscan"]
path = contrib/vectorscan
url = https://github.com/VectorCamp/vectorscan.git
[submodule "contrib/liburing"]
path = contrib/liburing
url = https://github.com/axboe/liburing.git
[submodule "contrib/c-ares"]
path = contrib/c-ares
url = https://github.com/ClickHouse/c-ares

View File

@ -17,6 +17,9 @@
### <a id="2212"></a> ClickHouse release 22.12, 2022-12-15
#### Backward Incompatible Change
* Add `GROUP BY ALL` syntax: [#37631](https://github.com/ClickHouse/ClickHouse/issues/37631). [#42265](https://github.com/ClickHouse/ClickHouse/pull/42265) ([刘陶峰](https://github.com/taofengliu)). If you have a column or an alias named `all` and doing `GROUP BY all` without the intention to group by all the columns, the query will have a different semantic. To keep the old semantic, put `all` into backticks or double quotes `"all"` to make it an identifier instead of a keyword.
#### Upgrade Notes
* Fixed backward incompatibility in (de)serialization of states of `min`, `max`, `any*`, `argMin`, `argMax` aggregate functions with `String` argument. The incompatibility affects 22.9, 22.10 and 22.11 branches (fixed since 22.9.6, 22.10.4 and 22.11.2 correspondingly). Some minor releases of 22.3, 22.7 and 22.8 branches are also affected: 22.3.13...22.3.14 (fixed since 22.3.15), 22.8.6...22.8.9 (fixed since 22.8.10), 22.7.6 and newer (will not be fixed in 22.7, we recommend upgrading from 22.7.* to 22.8.10 or newer). This release note does not concern users that have never used affected versions. Incompatible versions append an extra `'\0'` to strings when reading states of the aggregate functions mentioned above. For example, if an older version saved state of `anyState('foobar')` to `state_column` then the incompatible version will print `'foobar\0'` on `anyMerge(state_column)`. Also incompatible versions write states of the aggregate functions without trailing `'\0'`. Newer versions (that have the fix) can correctly read data written by all versions including incompatible versions, except one corner case. If an incompatible version saved a state with a string that actually ends with null character, then newer version will trim trailing `'\0'` when reading state of affected aggregate function. For example, if an incompatible version saved state of `anyState('abrac\0dabra\0')` to `state_column` then newer versions will print `'abrac\0dabra'` on `anyMerge(state_column)`. The issue also affects distributed queries when an incompatible version works in a cluster together with older or newer versions. [#43038](https://github.com/ClickHouse/ClickHouse/pull/43038) ([Alexander Tokmakov](https://github.com/tavplubix), [Raúl Marín](https://github.com/Algunenano)). Note: all the official ClickHouse builds already include the patches. This is not necessarily true for unofficial third-party builds that should be avoided.

View File

@ -131,7 +131,7 @@ def parse_env_variables(
ARM_V80COMPAT_SUFFIX = "-aarch64-v80compat"
FREEBSD_SUFFIX = "-freebsd"
PPC_SUFFIX = "-ppc64le"
AMD64_SSE2_SUFFIX = "-amd64sse2"
AMD64_COMPAT_SUFFIX = "-amd64-compat"
result = []
result.append("OUTPUT_DIR=/output")
@ -144,7 +144,7 @@ def parse_env_variables(
is_cross_arm_v80compat = compiler.endswith(ARM_V80COMPAT_SUFFIX)
is_cross_ppc = compiler.endswith(PPC_SUFFIX)
is_cross_freebsd = compiler.endswith(FREEBSD_SUFFIX)
is_amd64_sse2 = compiler.endswith(AMD64_SSE2_SUFFIX)
is_amd64_compat = compiler.endswith(AMD64_COMPAT_SUFFIX)
if is_cross_darwin:
cc = compiler[: -len(DARWIN_SUFFIX)]
@ -197,8 +197,8 @@ def parse_env_variables(
cmake_flags.append(
"-DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-ppc64le.cmake"
)
elif is_amd64_sse2:
cc = compiler[: -len(AMD64_SSE2_SUFFIX)]
elif is_amd64_compat:
cc = compiler[: -len(AMD64_COMPAT_SUFFIX)]
result.append("DEB_ARCH=amd64")
cmake_flags.append("-DNO_SSE3_OR_HIGHER=1")
else:
@ -358,7 +358,7 @@ if __name__ == "__main__":
"clang-15-aarch64",
"clang-15-aarch64-v80compat",
"clang-15-ppc64le",
"clang-15-amd64sse2",
"clang-15-amd64-compat",
"clang-15-freebsd",
"gcc-11",
),

View File

@ -17,6 +17,7 @@ ENV S3_URL="https://clickhouse-datasets.s3.amazonaws.com"
ENV DATASETS="hits visits"
RUN npm install -g azurite
RUN npm install tslib
COPY run.sh /
CMD ["/bin/bash", "/run.sh"]

View File

@ -80,6 +80,7 @@ ENV MINIO_ROOT_PASSWORD="clickhouse"
ENV EXPORT_S3_STORAGE_POLICIES=1
RUN npm install -g azurite
RUN npm install tslib
COPY run.sh /
COPY setup_minio.sh /

View File

@ -19,6 +19,7 @@ def process_result(result_folder):
"typos",
"whitespaces",
"workflows",
"submodules",
"docs spelling",
)

View File

@ -10,7 +10,7 @@ echo "Check style" | ts
echo "Check python formatting with black" | ts
./check-black -n |& tee /test_output/black_output.txt
echo "Check python type hinting with mypy" | ts
./check-mypy -n |& tee /test_output/mypy_output.txt
./check-mypy -n |& tee /test_output/mypy_output.txt
echo "Check typos" | ts
./check-typos |& tee /test_output/typos_output.txt
echo "Check docs spelling" | ts
@ -19,6 +19,8 @@ echo "Check whitespaces" | ts
./check-whitespaces -n |& tee /test_output/whitespaces_output.txt
echo "Check workflows" | ts
./check-workflows |& tee /test_output/workflows_output.txt
echo "Check submodules" | ts
./check-submodules |& tee /test_output/submodules_output.txt
echo "Check shell scripts with shellcheck" | ts
./shellcheck-run.sh |& tee /test_output/shellcheck_output.txt
/process_style_check_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv

View File

@ -9,14 +9,22 @@ if [ "${OS}" = "Linux" ]
then
if [ "${ARCH}" = "x86_64" -o "${ARCH}" = "amd64" ]
then
DIR="amd64"
# Require at least x86-64 + SSE4.2 (introduced in 2006). On older hardware fall back to plain x86-64 (introduced in 1999) which
# guarantees at least SSE2. The caveat is that plain x86-64 builds are much less tested than SSE 4.2 builds.
HAS_SSE42=$(grep sse4_2 /proc/cpuinfo)
if [ "${HAS_SSE42}" ]
then
DIR="amd64"
else
DIR="amd64compat"
fi
elif [ "${ARCH}" = "aarch64" -o "${ARCH}" = "arm64" ]
then
# If the system has >=ARMv8.2 (https://en.wikipedia.org/wiki/AArch64), choose the corresponding build, else fall back to a v8.0
# compat build. Unfortunately, the ARM ISA level cannot be read directly, we need to guess from the "features" in /proc/cpuinfo.
# Also, the flags in /proc/cpuinfo are named differently than the flags passed to the compiler (cmake/cpu_features.cmake).
ARMV82=$(grep -m 1 'Features' /proc/cpuinfo | awk '/asimd/ && /sha1/ && /aes/ && /atomics/ && /lrcpc/')
if [ "${ARMV82}" ]
HAS_ARMV82=$(grep -m 1 'Features' /proc/cpuinfo | awk '/asimd/ && /sha1/ && /aes/ && /atomics/ && /lrcpc/')
if [ "${HAS_ARMV82}" ]
then
DIR="aarch64"
else

View File

@ -6,10 +6,11 @@ slug: /en/install
# Installing ClickHouse
You have two options for getting up and running with ClickHouse:
You have three options for getting up and running with ClickHouse:
- **[ClickHouse Cloud](https://clickhouse.com/cloud/):** the official ClickHouse as a service, - built by, maintained, and supported by the creators of ClickHouse
- **[Self-managed ClickHouse](#self-managed-install):** ClickHouse can run on any Linux, FreeBSD, or Mac OS X with x86_64, AArch64, or PowerPC64LE CPU architecture
- **[ClickHouse Cloud](https://clickhouse.com/cloud/):** The official ClickHouse as a service, - built by, maintained and supported by the creators of ClickHouse
- **[Self-managed ClickHouse](#self-managed-install):** ClickHouse can run on any Linux, FreeBSD, or Mac OS X with x86-64, ARM, or PowerPC64LE CPU architecture
- **[Docker Image](https://hub.docker.com/r/clickhouse/clickhouse-server/):** Read the guide with the official image in Docker Hub
## ClickHouse Cloud
@ -22,73 +23,49 @@ The quickest and easiest way to get up and running with ClickHouse is to create
Once your Cloud service is provisioned, you will be able to [connect to it](/docs/en/integrations/connect-a-client.md) and start [inserting data](/docs/en/integrations/data-ingestion.md).
:::note
The [Quick Start](/docs/en/quick-start.mdx) walks through the steps to get a ClickHouse Cloud service up and running, connecting to it, and inserting data.
:::
## Self-Managed Requirements
### CPU Architecture
ClickHouse can run on any Linux, FreeBSD, or Mac OS X with x86_64, AArch64, or PowerPC64LE CPU architecture.
Official pre-built binaries are typically compiled for x86_64 and leverage SSE 4.2 instruction set, so unless otherwise stated usage of CPU that supports it becomes an additional system requirement. Heres the command to check if current CPU has support for SSE 4.2:
``` bash
$ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported"
```
To run ClickHouse on processors that do not support SSE 4.2 or have AArch64 or PowerPC64LE architecture, you should [build ClickHouse from sources](#from-sources) with proper configuration adjustments.
ClickHouse implements parallel data processing and uses all the hardware resources available. When choosing a processor, take into account that ClickHouse works more efficiently at configurations with a large number of cores but a lower clock rate than at configurations with fewer cores and a higher clock rate. For example, 16 cores with 2600 MHz is preferable to 8 cores with 3600 MHz.
It is recommended to use **Turbo Boost** and **hyper-threading** technologies. It significantly improves performance with a typical workload.
### RAM {#ram}
We recommend using a minimum of 4GB of RAM to perform non-trivial queries. The ClickHouse server can run with a much smaller amount of RAM, but it requires memory for processing queries.
The required volume of RAM depends on:
- The complexity of queries.
- The amount of data that is processed in queries.
To calculate the required volume of RAM, you should estimate the size of temporary data for [GROUP BY](/docs/en/sql-reference/statements/select/group-by.md#select-group-by-clause), [DISTINCT](/docs/en/sql-reference/statements/select/distinct.md#select-distinct), [JOIN](/docs/en/sql-reference/statements/select/join.md#select-join) and other operations you use.
ClickHouse can use external memory for temporary data. See [GROUP BY in External Memory](/docs/en/sql-reference/statements/select/group-by.md#select-group-by-in-external-memory) for details.
### Swap File {#swap-file}
Disable the swap file for production environments.
### Storage Subsystem {#storage-subsystem}
You need to have 2GB of free disk space to install ClickHouse.
The volume of storage required for your data should be calculated separately. Assessment should include:
- Estimation of the data volume.
You can take a sample of the data and get the average size of a row from it. Then multiply the value by the number of rows you plan to store.
- The data compression coefficient.
To estimate the data compression coefficient, load a sample of your data into ClickHouse, and compare the actual size of the data with the size of the table stored. For example, clickstream data is usually compressed by 6-10 times.
To calculate the final volume of data to be stored, apply the compression coefficient to the estimated data volume. If you plan to store data in several replicas, then multiply the estimated volume by the number of replicas.
### Network {#network}
If possible, use networks of 10G or higher class.
The network bandwidth is critical for processing distributed queries with a large amount of intermediate data. Besides, network speed affects replication processes.
### Software {#software}
ClickHouse is developed primarily for the Linux family of operating systems. The recommended Linux distribution is Ubuntu. The `tzdata` package should be installed in the system.
## Self-Managed Install
1. The simplest way to download ClickHouse locally is to run the following command. If your operating system is supported, an appropriate ClickHouse binary will be downloaded and made runnable:
```bash
curl https://clickhouse.com/ | sh
```
1. Run the `install` command, which defines a collection of useful symlinks along with the files and folders used by ClickHouse - all of which you can see in the output of the install script:
```bash
sudo ./clickhouse install
```
1. At the end of the install script, you are prompted for a password for the `default` user. Feel free to enter a password, or you can optionally leave it blank:
```response
Creating log directory /var/log/clickhouse-server.
Creating data directory /var/lib/clickhouse.
Creating pid directory /var/run/clickhouse-server.
chown -R clickhouse:clickhouse '/var/log/clickhouse-server'
chown -R clickhouse:clickhouse '/var/run/clickhouse-server'
chown clickhouse:clickhouse '/var/lib/clickhouse'
Enter password for default user:
```
You should see the following output:
```response
ClickHouse has been successfully installed.
Start clickhouse-server with:
sudo clickhouse start
Start clickhouse-client with:
clickhouse-client
```
1. Run the following command to start the ClickHouse server:
```bash
sudo clickhouse start
```
:::tip
The [Quick Start](/docs/en/quick-start.mdx/#step-1-get-clickhouse) walks through the steps to download and run ClickHouse, connect to it, and insert data.
:::
## Available Installation Options {#available-installation-options}
### From DEB Packages {#install-from-deb-packages}
@ -278,50 +255,16 @@ For production environments, its recommended to use the latest `stable`-versi
To run ClickHouse inside Docker follow the guide on [Docker Hub](https://hub.docker.com/r/clickhouse/clickhouse-server/). Those images use official `deb` packages inside.
### Single Binary {#from-single-binary}
You can install ClickHouse on Linux using a single portable binary from the latest commit of the `master` branch: [https://builds.clickhouse.com/master/amd64/clickhouse].
``` bash
curl -O 'https://builds.clickhouse.com/master/amd64/clickhouse' && chmod a+x clickhouse
sudo ./clickhouse install
```
### From Precompiled Binaries for Non-Standard Environments {#from-binaries-non-linux}
For non-Linux operating systems and for AArch64 CPU architecture, ClickHouse builds are provided as a cross-compiled binary from the latest commit of the `master` branch (with a few hours delay).
- [MacOS x86_64](https://builds.clickhouse.com/master/macos/clickhouse)
```bash
curl -O 'https://builds.clickhouse.com/master/macos/clickhouse' && chmod a+x ./clickhouse
```
- [MacOS Aarch64 (Apple Silicon)](https://builds.clickhouse.com/master/macos-aarch64/clickhouse)
```bash
curl -O 'https://builds.clickhouse.com/master/macos-aarch64/clickhouse' && chmod a+x ./clickhouse
```
- [FreeBSD x86_64](https://builds.clickhouse.com/master/freebsd/clickhouse)
```bash
curl -O 'https://builds.clickhouse.com/master/freebsd/clickhouse' && chmod a+x ./clickhouse
```
- [Linux AArch64](https://builds.clickhouse.com/master/aarch64/clickhouse)
```bash
curl -O 'https://builds.clickhouse.com/master/aarch64/clickhouse' && chmod a+x ./clickhouse
```
Run `sudo ./clickhouse install` to install ClickHouse system-wide (also with needed configuration files, configuring users etc.). Then run `sudo clickhouse start` commands to start the clickhouse-server and `clickhouse-client` to connect to it.
Use the `clickhouse client` to connect to the server, or `clickhouse local` to process local data.
### From Sources {#from-sources}
To manually compile ClickHouse, follow the instructions for [Linux](/docs/en/development/build.md) or [Mac OS X](/docs/en/development/build-osx.md).
You can compile packages and install them or use programs without installing packages. Also by building manually you can disable SSE 4.2 requirement or build for AArch64 CPUs.
You can compile packages and install them or use programs without installing packages.
Client: programs/clickhouse-client
Server: programs/clickhouse-server
Client: <build_directory>/programs/clickhouse-client
Server: <build_directory>/programs/clickhouse-server
Youll need to create a data and metadata folders and `chown` them for the desired user. Their paths can be changed in server config (src/programs/server/config.xml), by default they are:
Youll need to create data and metadata folders manually and `chown` them for the desired user. Their paths can be changed in server config (src/programs/server/config.xml), by default they are:
/var/lib/clickhouse/data/default/
/var/lib/clickhouse/metadata/default/
@ -406,3 +349,42 @@ SELECT 1
**Congratulations, the system works!**
To continue experimenting, you can download one of the test data sets or go through [tutorial](/docs/en/tutorial.md).
## Recommendations for Self-Managed ClickHouse
ClickHouse can run on any Linux, FreeBSD, or Mac OS X with x86-64, ARM, or PowerPC64LE CPU architecture.
ClickHouse uses all hardware resources available to process data.
ClickHouse tends to work more efficiently with a large number of cores at a lower clock rate than with fewer cores at a higher clock rate.
We recommend using a minimum of 4GB of RAM to perform non-trivial queries. The ClickHouse server can run with a much smaller amount of RAM, but queries will then frequently abort.
The required volume of RAM generally depends on:
- The complexity of queries.
- The amount of data that is processed in queries.
To calculate the required volume of RAM, you may estimate the size of temporary data for [GROUP BY](/docs/en/sql-reference/statements/select/group-by.md#select-group-by-clause), [DISTINCT](/docs/en/sql-reference/statements/select/distinct.md#select-distinct), [JOIN](/docs/en/sql-reference/statements/select/join.md#select-join) and other operations you use.
To reduce memory consumption, ClickHouse can swap temporary data to external storage. See [GROUP BY in External Memory](/docs/en/sql-reference/statements/select/group-by.md#select-group-by-in-external-memory) for details.
We recommend to disable the operating system's swap file in production environments.
The ClickHouse binary requires at least 2.5 GB of disk space for installation.
The volume of storage required for your data may be calculated separately based on
- an estimation of the data volume.
You can take a sample of the data and get the average size of a row from it. Then multiply the value by the number of rows you plan to store.
- The data compression coefficient.
To estimate the data compression coefficient, load a sample of your data into ClickHouse, and compare the actual size of the data with the size of the table stored. For example, clickstream data is usually compressed by 6-10 times.
To calculate the final volume of data to be stored, apply the compression coefficient to the estimated data volume. If you plan to store data in several replicas, then multiply the estimated volume by the number of replicas.
For distributed ClickHouse deployments (clustering), we recommend at least 10G class network connectivity.
Network bandwidth is critical for processing distributed queries with a large amount of intermediate data. Besides, network speed affects replication processes.

View File

@ -3588,6 +3588,13 @@ y Nullable(String)
z IPv4
```
## schema_inference_make_columns_nullable {#schema_inference_make_columns_nullable}
Controls making inferred types `Nullable` in schema inference for formats without information about nullability.
If the setting is enabled, the inferred type will be `Nullable` only if column contains `NULL` in a sample that is parsed during schema inference.
Default value: `false`.
## input_format_try_infer_integers {#input_format_try_infer_integers}
If enabled, ClickHouse will try to infer integers instead of floats in schema inference for text formats. If all numbers in the column from input data are integers, the result type will be `Int64`, if at least one number is float, the result type will be `Float64`.

View File

@ -70,6 +70,8 @@
#include <QueryPipeline/ConnectionCollector.h>
#include <Dictionaries/registerDictionaries.h>
#include <Disks/registerDisks.h>
#include <IO/Resource/registerSchedulerNodes.h>
#include <IO/Resource/registerResourceManagers.h>
#include <Common/Config/ConfigReloader.h>
#include <Server/HTTPHandlerFactory.h>
#include "MetricsTransmitter.h"
@ -715,6 +717,8 @@ try
registerDisks(/* global_skip_access_check= */ false);
registerFormats();
registerRemoteFileMetadatas();
registerSchedulerNodes();
registerResourceManagers();
CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision());
CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger());
@ -1289,6 +1293,11 @@ try
global_context->getDistributedSchedulePool().increaseThreadsCount(new_pool_size);
}
if (config->has("resources"))
{
global_context->getResourceManager()->updateConfiguration(*config);
}
if (!initial_loading)
{
/// We do not load ZooKeeper configuration on the first config loading

View File

@ -75,6 +75,10 @@ void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const A
}
}
bool SettingsProfileElement::isConstraint() const
{
return this->writability || !this->min_value.isNull() || !this->max_value.isNull();
}
std::shared_ptr<ASTSettingsProfileElement> SettingsProfileElement::toAST() const
{
@ -213,7 +217,7 @@ SettingsConstraints SettingsProfileElements::toSettingsConstraints(const AccessC
{
SettingsConstraints res{access_control};
for (const auto & elem : *this)
if (!elem.setting_name.empty() && elem.setting_name != ALLOW_BACKUP_SETTING_NAME)
if (!elem.setting_name.empty() && elem.isConstraint() && elem.setting_name != ALLOW_BACKUP_SETTING_NAME)
res.set(
elem.setting_name,
elem.min_value,

View File

@ -44,6 +44,8 @@ struct SettingsProfileElement
std::shared_ptr<ASTSettingsProfileElement> toAST() const;
std::shared_ptr<ASTSettingsProfileElement> toASTWithNames(const AccessControl & access_control) const;
bool isConstraint() const;
private:
void init(const ASTSettingsProfileElement & ast, const AccessControl * access_control);
};

View File

@ -129,7 +129,7 @@ BackupWriterS3::BackupWriterS3(
, request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings)
, log(&Poco::Logger::get("BackupWriterS3"))
{
request_settings.updateFromSettingsIfEmpty(context_->getSettingsRef());
request_settings.updateFromSettings(context_->getSettingsRef());
request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint
}
@ -210,20 +210,21 @@ void BackupWriterS3::copyObjectMultipartImpl(
std::vector<String> part_tags;
size_t position = 0;
size_t upload_part_size = request_settings.min_upload_part_size;
const auto & settings = request_settings.getUploadSettings();
size_t upload_part_size = settings.min_upload_part_size;
for (size_t part_number = 1; position < size; ++part_number)
{
/// Check that part number is not too big.
if (part_number > request_settings.max_part_number)
if (part_number > settings.max_part_number)
{
throw Exception(
ErrorCodes::INVALID_CONFIG_PARAMETER,
"Part number exceeded {} while writing {} bytes to S3. Check min_upload_part_size = {}, max_upload_part_size = {}, "
"upload_part_size_multiply_factor = {}, upload_part_size_multiply_parts_count_threshold = {}, max_single_operation_copy_size = {}",
request_settings.max_part_number, size, request_settings.min_upload_part_size, request_settings.max_upload_part_size,
request_settings.upload_part_size_multiply_factor, request_settings.upload_part_size_multiply_parts_count_threshold,
request_settings.max_single_operation_copy_size);
settings.max_part_number, size, settings.min_upload_part_size, settings.max_upload_part_size,
settings.upload_part_size_multiply_factor, settings.upload_part_size_multiply_parts_count_threshold,
settings.max_single_operation_copy_size);
}
size_t next_position = std::min(position + upload_part_size, size);
@ -256,10 +257,10 @@ void BackupWriterS3::copyObjectMultipartImpl(
position = next_position;
/// Maybe increase `upload_part_size` (we need to increase it sometimes to keep `part_number` less or equal than `max_part_number`).
if (part_number % request_settings.upload_part_size_multiply_parts_count_threshold == 0)
if (part_number % settings.upload_part_size_multiply_parts_count_threshold == 0)
{
upload_part_size *= request_settings.upload_part_size_multiply_factor;
upload_part_size = std::min(upload_part_size, request_settings.max_upload_part_size);
upload_part_size *= settings.upload_part_size_multiply_factor;
upload_part_size = std::min(upload_part_size, settings.max_upload_part_size);
}
}
@ -302,7 +303,7 @@ void BackupWriterS3::copyFileNative(DiskPtr from_disk, const String & file_name_
auto file_path = fs::path(s3_uri.key) / file_name_to;
auto head = S3::headObject(*client, source_bucket, objects[0].absolute_path).GetResult();
if (static_cast<size_t>(head.GetContentLength()) < request_settings.max_single_operation_copy_size)
if (static_cast<size_t>(head.GetContentLength()) < request_settings.getUploadSettings().max_single_operation_copy_size)
{
copyObjectImpl(
source_bucket, objects[0].absolute_path, s3_uri.bucket, file_path, head);

View File

@ -88,6 +88,7 @@ add_headers_and_sources(clickhouse_common_io Common)
add_headers_and_sources(clickhouse_common_io Common/HashTable)
add_headers_and_sources(clickhouse_common_io IO)
add_headers_and_sources(clickhouse_common_io IO/Archives)
add_headers_and_sources(clickhouse_common_io IO/Resource)
add_headers_and_sources(clickhouse_common_io IO/S3)
list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp)

View File

@ -640,6 +640,9 @@
M(669, NAMED_COLLECTION_DOESNT_EXIST) \
M(670, NAMED_COLLECTION_ALREADY_EXISTS) \
M(671, NAMED_COLLECTION_IS_IMMUTABLE) \
M(672, INVALID_SCHEDULER_NODE) \
M(673, RESOURCE_ACCESS_DENIED) \
M(674, RESOURCE_NOT_FOUND) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -136,7 +136,6 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa
return;
S3Settings::RequestSettings request_settings_1;
request_settings_1.setEmptyFieldsByDefault();
const auto create_writer = [&](const auto & key)
{

View File

@ -83,6 +83,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(UInt64, distributed_connections_pool_size, 1024, "Maximum number of connections with one remote server in the pool.", 0) \
M(UInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \
M(UInt64, s3_min_upload_part_size, 16*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \
M(UInt64, s3_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to S3.", 0) \
M(UInt64, s3_upload_part_size_multiply_factor, 2, "Multiply s3_min_upload_part_size by this factor each time s3_multiply_parts_count_threshold parts were uploaded from a single write to S3.", 0) \
M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3 s3_min_upload_part_size multiplied by s3_upload_part_size_multiply_factor.", 0) \
M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \
@ -651,6 +652,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Map, additional_table_filters, "", "Additional filter expression which would be applied after reading from specified table. Syntax: {'table1': 'expression', 'database.table2': 'expression'}", 0) \
M(String, additional_result_filter, "", "Additional filter expression which would be applied to query result", 0) \
\
M(String, workload, "default", "Name of workload to be used to access resources", 0) \
\
/** Experimental functions */ \
M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \
M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \
@ -761,6 +764,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Bool, input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Arrow", 0) \
M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \
M(String, schema_inference_hints, "", "The list of column names and types to use in schema inference for formats without column names. The format: 'column_name1 column_type1, column_name2 column_type2, ...'", 0) \
M(Bool, schema_inference_make_columns_nullable, true, "If set to true, all inferred types will be Nullable in schema inference for formats without information about nullability.", 0) \
M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \
M(Bool, input_format_json_try_infer_numbers_from_strings, true, "Try to infer numbers from string fields while schema inference", 0) \
M(Bool, input_format_json_validate_types_from_metadata, true, "For JSON/JSONCompact/JSONColumnsWithMetadata input formats this controls whether format parser should check if data types from input metadata match data types of the corresponding columns from the table", 0) \

View File

@ -3,6 +3,7 @@
#include <cstdint>
#include <string>
#include <vector>
#include <unordered_set>
#include <base/strong_typedef.h>
#include <base/Decimal.h>
#include <base/defines.h>
@ -93,4 +94,5 @@ using Int256 = ::Int256;
/// Not a data type in database, defined just for convenience.
using Strings = std::vector<String>;
using TypeIndexesSet = std::unordered_set<TypeIndex>;
}

View File

@ -41,6 +41,8 @@ public:
SerializationPtr doGetDefaultSerialization() const override;
bool hasNullableSubcolumns() const { return is_nullable; }
const String & getSchemaFormat() const { return schema_format; }
};
}

View File

@ -8,74 +8,62 @@
namespace DB
{
void transformTypesRecursively(DataTypes & types, std::function<void(DataTypes &)> transform_simple_types, std::function<void(DataTypes &)> transform_complex_types)
void transformTypesRecursively(DataTypes & types, std::function<void(DataTypes &, const TypeIndexesSet &)> transform_simple_types, std::function<void(DataTypes &, const TypeIndexesSet &)> transform_complex_types)
{
TypeIndexesSet type_indexes;
for (const auto & type : types)
type_indexes.insert(type->getTypeId());
/// Arrays
if (type_indexes.contains(TypeIndex::Array))
{
/// Arrays
bool have_array = false;
bool all_arrays = true;
DataTypes nested_types;
for (const auto & type : types)
/// All types are Array
if (type_indexes.size() == 1)
{
if (const DataTypeArray * type_array = typeid_cast<const DataTypeArray *>(type.get()))
{
have_array = true;
nested_types.push_back(type_array->getNestedType());
}
else
all_arrays = false;
DataTypes nested_types;
for (const auto & type : types)
nested_types.push_back(typeid_cast<const DataTypeArray *>(type.get())->getNestedType());
transformTypesRecursively(nested_types, transform_simple_types, transform_complex_types);
for (size_t i = 0; i != types.size(); ++i)
types[i] = std::make_shared<DataTypeArray>(nested_types[i]);
}
if (have_array)
{
if (all_arrays)
{
transformTypesRecursively(nested_types, transform_simple_types, transform_complex_types);
for (size_t i = 0; i != types.size(); ++i)
types[i] = std::make_shared<DataTypeArray>(nested_types[i]);
}
if (transform_complex_types)
transform_complex_types(types, type_indexes);
if (transform_complex_types)
transform_complex_types(types);
return;
}
return;
}
/// Tuples
if (type_indexes.contains(TypeIndex::Tuple))
{
/// Tuples
bool have_tuple = false;
bool all_tuples = true;
size_t tuple_size = 0;
std::vector<DataTypes> nested_types;
for (const auto & type : types)
/// All types are Tuple
if (type_indexes.size() == 1)
{
if (const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(type.get()))
{
if (!have_tuple)
{
tuple_size = type_tuple->getElements().size();
nested_types.resize(tuple_size);
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
nested_types[elem_idx].reserve(types.size());
}
else if (tuple_size != type_tuple->getElements().size())
return;
std::vector<DataTypes> nested_types;
const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(types[0].get());
size_t tuple_size = type_tuple->getElements().size();
nested_types.resize(tuple_size);
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
nested_types[elem_idx].reserve(types.size());
have_tuple = true;
bool sizes_are_equal = true;
for (const auto & type : types)
{
type_tuple = typeid_cast<const DataTypeTuple *>(type.get());
if (type_tuple->getElements().size() != tuple_size)
{
sizes_are_equal = false;
break;
}
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
nested_types[elem_idx].emplace_back(type_tuple->getElements()[elem_idx]);
}
else
all_tuples = false;
}
if (have_tuple)
{
if (all_tuples)
if (sizes_are_equal)
{
std::vector<DataTypes> transposed_nested_types(types.size());
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
@ -88,56 +76,47 @@ void transformTypesRecursively(DataTypes & types, std::function<void(DataTypes &
for (size_t i = 0; i != types.size(); ++i)
types[i] = std::make_shared<DataTypeTuple>(transposed_nested_types[i]);
}
if (transform_complex_types)
transform_complex_types(types);
return;
}
if (transform_complex_types)
transform_complex_types(types, type_indexes);
return;
}
/// Maps
if (type_indexes.contains(TypeIndex::Map))
{
/// Maps
bool have_maps = false;
bool all_maps = true;
DataTypes key_types;
DataTypes value_types;
key_types.reserve(types.size());
value_types.reserve(types.size());
for (const auto & type : types)
/// All types are Map
if (type_indexes.size() == 1)
{
if (const DataTypeMap * type_map = typeid_cast<const DataTypeMap *>(type.get()))
DataTypes key_types;
DataTypes value_types;
key_types.reserve(types.size());
value_types.reserve(types.size());
for (const auto & type : types)
{
have_maps = true;
const DataTypeMap * type_map = typeid_cast<const DataTypeMap *>(type.get());
key_types.emplace_back(type_map->getKeyType());
value_types.emplace_back(type_map->getValueType());
}
else
all_maps = false;
transformTypesRecursively(key_types, transform_simple_types, transform_complex_types);
transformTypesRecursively(value_types, transform_simple_types, transform_complex_types);
for (size_t i = 0; i != types.size(); ++i)
types[i] = std::make_shared<DataTypeMap>(key_types[i], value_types[i]);
}
if (have_maps)
{
if (all_maps)
{
transformTypesRecursively(key_types, transform_simple_types, transform_complex_types);
transformTypesRecursively(value_types, transform_simple_types, transform_complex_types);
if (transform_complex_types)
transform_complex_types(types, type_indexes);
for (size_t i = 0; i != types.size(); ++i)
types[i] = std::make_shared<DataTypeMap>(key_types[i], value_types[i]);
}
if (transform_complex_types)
transform_complex_types(types);
return;
}
return;
}
/// Nullable
if (type_indexes.contains(TypeIndex::Nullable))
{
/// Nullable
bool have_nullable = false;
std::vector<UInt8> is_nullable;
is_nullable.reserve(types.size());
DataTypes nested_types;
@ -146,7 +125,6 @@ void transformTypesRecursively(DataTypes & types, std::function<void(DataTypes &
{
if (const DataTypeNullable * type_nullable = typeid_cast<const DataTypeNullable *>(type.get()))
{
have_nullable = true;
is_nullable.push_back(1);
nested_types.push_back(type_nullable->getNestedType());
}
@ -157,28 +135,28 @@ void transformTypesRecursively(DataTypes & types, std::function<void(DataTypes &
}
}
if (have_nullable)
transformTypesRecursively(nested_types, transform_simple_types, transform_complex_types);
for (size_t i = 0; i != types.size(); ++i)
{
transformTypesRecursively(nested_types, transform_simple_types, transform_complex_types);
for (size_t i = 0; i != types.size(); ++i)
{
if (is_nullable[i])
types[i] = makeNullable(nested_types[i]);
else
types[i] = nested_types[i];
}
return;
if (is_nullable[i])
types[i] = makeNullable(nested_types[i]);
else
types[i] = nested_types[i];
}
if (transform_complex_types)
transform_complex_types(types, type_indexes);
return;
}
transform_simple_types(types);
transform_simple_types(types, type_indexes);
}
void callOnNestedSimpleTypes(DataTypePtr & type, std::function<void(DataTypePtr &)> callback)
{
DataTypes types = {type};
transformTypesRecursively(types, [callback](auto & data_types){ callback(data_types[0]); }, {});
transformTypesRecursively(types, [callback](auto & data_types, const TypeIndexesSet &){ callback(data_types[0]); }, {});
}
}

View File

@ -12,7 +12,7 @@ namespace DB
/// If not all types are the same complex type (Array/Map/Tuple), this function won't be called to nested types.
/// Function transform_simple_types will be applied to resulting simple types after all recursive calls.
/// Function transform_complex_types will be applied to complex types (Array/Map/Tuple) after recursive call to their nested types.
void transformTypesRecursively(DataTypes & types, std::function<void(DataTypes &)> transform_simple_types, std::function<void(DataTypes &)> transform_complex_types);
void transformTypesRecursively(DataTypes & types, std::function<void(DataTypes &, const TypeIndexesSet &)> transform_simple_types, std::function<void(DataTypes &, const TypeIndexesSet &)> transform_complex_types);
void callOnNestedSimpleTypes(DataTypePtr & type, std::function<void(DataTypePtr &)> callback);

View File

@ -512,7 +512,7 @@ void S3ObjectStorage::copyObjectMultipartImpl(
std::vector<String> part_tags;
size_t upload_part_size = settings_ptr->request_settings.min_upload_part_size;
size_t upload_part_size = settings_ptr->request_settings.getUploadSettings().min_upload_part_size;
for (size_t position = 0, part_number = 1; position < size; ++part_number, position += upload_part_size)
{
ProfileEvents::increment(ProfileEvents::S3UploadPartCopy);

View File

@ -34,24 +34,7 @@ namespace ErrorCodes
std::unique_ptr<S3ObjectStorageSettings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context)
{
const Settings & settings = context->getSettingsRef();
S3Settings::RequestSettings request_settings;
request_settings.max_single_read_retries = config.getUInt64(config_prefix + ".s3_max_single_read_retries", settings.s3_max_single_read_retries);
request_settings.min_upload_part_size = config.getUInt64(config_prefix + ".s3_min_upload_part_size", settings.s3_min_upload_part_size);
request_settings.max_upload_part_size = config.getUInt64(config_prefix + ".s3_max_upload_part_size", S3Settings::RequestSettings::DEFAULT_MAX_UPLOAD_PART_SIZE);
request_settings.upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_factor", settings.s3_upload_part_size_multiply_factor);
request_settings.upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_parts_count_threshold", settings.s3_upload_part_size_multiply_parts_count_threshold);
request_settings.max_part_number = config.getUInt64(config_prefix + ".s3_max_part_number", S3Settings::RequestSettings::DEFAULT_MAX_PART_NUMBER);
request_settings.max_single_part_upload_size = config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", settings.s3_max_single_part_upload_size);
request_settings.check_objects_after_upload = config.getUInt64(config_prefix + ".s3_check_objects_after_upload", settings.s3_check_objects_after_upload);
request_settings.max_unexpected_write_error_retries = config.getUInt64(config_prefix + ".s3_max_unexpected_write_error_retries", settings.s3_max_unexpected_write_error_retries);
// NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used.
if (UInt64 max_get_rps = config.getUInt64(config_prefix + ".s3_max_get_rps", settings.s3_max_get_rps))
request_settings.get_request_throttler = std::make_shared<Throttler>(
max_get_rps, config.getUInt64(config_prefix + ".s3_max_get_burst", settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * max_get_rps));
if (UInt64 max_put_rps = config.getUInt64(config_prefix + ".s3_max_put_rps", settings.s3_max_put_rps))
request_settings.put_request_throttler = std::make_shared<Throttler>(
max_put_rps, config.getUInt64(config_prefix + ".s3_max_put_burst", settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * max_put_rps));
S3Settings::RequestSettings request_settings(config, config_prefix, settings, "s3_");
return std::make_unique<S3ObjectStorageSettings>(
request_settings,

View File

@ -1,21 +1,11 @@
#include <Formats/EscapingRuleUtils.h>
#include <Formats/JSONUtils.h>
#include <Formats/ReadSchemaUtils.h>
#include <Formats/SchemaInferenceUtils.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNothing.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeObject.h>
#include <DataTypes/getLeastSupertype.h>
#include <DataTypes/transformTypesRecursively.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromString.h>
@ -261,556 +251,76 @@ String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule e
return readByEscapingRule<true>(buf, escaping_rule, format_settings);
}
void transformInferredTypesIfNeededImpl(DataTypes & types, const FormatSettings & settings, bool is_json, const std::unordered_set<const IDataType *> * numbers_parsed_from_json_strings = nullptr)
{
/// Do nothing if we didn't try to infer something special.
if (!settings.try_infer_integers && !settings.try_infer_dates && !settings.try_infer_datetimes && !is_json)
return;
auto transform_simple_types = [&](DataTypes & data_types)
{
/// If we have floats and integers convert them all to float.
if (settings.try_infer_integers)
{
bool have_floats = false;
bool have_integers = false;
for (const auto & type : data_types)
{
have_floats |= isFloat(type);
have_integers |= isInteger(type) && !isBool(type);
}
if (have_floats && have_integers)
{
for (auto & type : data_types)
{
if (isInteger(type))
type = std::make_shared<DataTypeFloat64>();
}
}
}
/// If we have only dates and datetimes, convert dates to datetime.
/// If we have date/datetimes and smth else, convert them to string, because
/// There is a special case when we inferred both Date/DateTime and Int64 from Strings,
/// for example: "arr: ["2020-01-01", "2000"]" -> Tuple(Date, Int64),
/// so if we have Date/DateTime and smth else (not only String) we should
/// convert Date/DateTime back to String, so then we will be able to
/// convert Int64 back to String as well.
if (settings.try_infer_dates || settings.try_infer_datetimes)
{
bool have_dates = false;
bool have_datetimes = false;
bool all_dates_or_datetimes = true;
for (const auto & type : data_types)
{
have_dates |= isDate(type);
have_datetimes |= isDateTime64(type);
all_dates_or_datetimes &= isDate(type) || isDateTime64(type);
}
if (!all_dates_or_datetimes && (have_dates || have_datetimes))
{
for (auto & type : data_types)
{
if (isDate(type) || isDateTime64(type))
type = std::make_shared<DataTypeString>();
}
}
else if (have_dates && have_datetimes)
{
for (auto & type : data_types)
{
if (isDate(type))
type = std::make_shared<DataTypeDateTime64>(9);
}
}
}
if (!is_json)
return;
/// Check settings specific for JSON formats.
/// If we have numbers and strings, convert numbers to strings.
if (settings.json.try_infer_numbers_from_strings || settings.json.read_numbers_as_strings)
{
bool have_strings = false;
bool have_numbers = false;
for (const auto & type : data_types)
{
have_strings |= isString(type);
have_numbers |= isNumber(type);
}
if (have_strings && have_numbers)
{
for (auto & type : data_types)
{
if (isNumber(type)
&& (settings.json.read_numbers_as_strings || !numbers_parsed_from_json_strings
|| numbers_parsed_from_json_strings->contains(type.get())))
type = std::make_shared<DataTypeString>();
}
}
}
if (settings.json.read_bools_as_numbers)
{
/// Note that have_floats and have_integers both cannot be
/// equal to true as in one of previous checks we convert
/// integers to floats if we have both.
bool have_floats = false;
bool have_integers = false;
bool have_bools = false;
for (const auto & type : data_types)
{
have_floats |= isFloat(type);
have_integers |= isInteger(type) && !isBool(type);
have_bools |= isBool(type);
}
if (have_bools && (have_integers || have_floats))
{
for (auto & type : data_types)
{
if (isBool(type))
{
if (have_integers)
type = std::make_shared<DataTypeInt64>();
else
type = std::make_shared<DataTypeFloat64>();
}
}
}
}
};
auto transform_complex_types = [&](DataTypes & data_types)
{
if (!is_json)
return;
bool have_maps = false;
bool have_objects = false;
bool have_strings = false;
bool are_maps_equal = true;
DataTypePtr first_map_type;
for (const auto & type : data_types)
{
if (isMap(type))
{
if (!have_maps)
{
first_map_type = type;
have_maps = true;
}
else
{
are_maps_equal &= type->equals(*first_map_type);
}
}
else if (isObject(type))
{
have_objects = true;
}
else if (isString(type))
{
have_strings = false;
}
}
if (have_maps && (have_objects || !are_maps_equal))
{
for (auto & type : data_types)
{
if (isMap(type))
type = std::make_shared<DataTypeObject>("json", true);
}
}
if (settings.json.read_objects_as_strings && have_strings && (have_maps || have_objects))
{
for (auto & type : data_types)
{
if (isMap(type) || isObject(type))
type = std::make_shared<DataTypeString>();
}
}
};
transformTypesRecursively(types, transform_simple_types, transform_complex_types);
}
void transformInferredTypesIfNeeded(DataTypes & types, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule)
{
transformInferredTypesIfNeededImpl(types, settings, escaping_rule == FormatSettings::EscapingRule::JSON);
}
void transformInferredTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule)
{
DataTypes types = {first, second};
transformInferredTypesIfNeeded(types, settings, escaping_rule);
first = std::move(types[0]);
second = std::move(types[1]);
}
void transformInferredJSONTypesIfNeeded(DataTypes & types, const FormatSettings & settings, const std::unordered_set<const IDataType *> * numbers_parsed_from_json_strings)
{
transformInferredTypesIfNeededImpl(types, settings, true, numbers_parsed_from_json_strings);
}
void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings)
{
DataTypes types = {first, second};
transformInferredJSONTypesIfNeeded(types, settings);
first = std::move(types[0]);
second = std::move(types[1]);
}
bool tryInferDate(const std::string_view & field)
{
ReadBufferFromString buf(field);
DayNum tmp;
return tryReadDateText(tmp, buf) && buf.eof();
}
bool tryInferDateTime(const std::string_view & field, const FormatSettings & settings)
{
if (field.empty())
return false;
ReadBufferFromString buf(field);
Float64 tmp_float;
/// Check if it's just a number, and if so, don't try to infer DateTime from it,
/// because we can interpret this number as a timestamp and it will lead to
/// inferring DateTime instead of simple Int64/Float64 in some cases.
if (tryReadFloatText(tmp_float, buf) && buf.eof())
return false;
buf.seek(0, SEEK_SET); /// Return position to the beginning
DateTime64 tmp;
switch (settings.date_time_input_format)
{
case FormatSettings::DateTimeInputFormat::Basic:
if (tryReadDateTime64Text(tmp, 9, buf) && buf.eof())
return true;
break;
case FormatSettings::DateTimeInputFormat::BestEffort:
if (tryParseDateTime64BestEffort(tmp, 9, buf, DateLUT::instance(), DateLUT::instance("UTC")) && buf.eof())
return true;
break;
case FormatSettings::DateTimeInputFormat::BestEffortUS:
if (tryParseDateTime64BestEffortUS(tmp, 9, buf, DateLUT::instance(), DateLUT::instance("UTC")) && buf.eof())
return true;
break;
}
return false;
}
DataTypePtr tryInferDateOrDateTime(const std::string_view & field, const FormatSettings & settings)
{
if (settings.try_infer_dates && tryInferDate(field))
return makeNullable(std::make_shared<DataTypeDate>());
if (settings.try_infer_datetimes && tryInferDateTime(field, settings))
return makeNullable(std::make_shared<DataTypeDateTime64>(9));
return nullptr;
}
static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBufferFromString & buf, const FormatSettings & settings)
{
if (buf.eof())
return nullptr;
/// Array
if (checkChar('[', buf))
{
skipWhitespaceIfAny(buf);
DataTypes nested_types;
bool first = true;
while (!buf.eof() && *buf.position() != ']')
{
if (!first)
{
skipWhitespaceIfAny(buf);
if (!checkChar(',', buf))
return nullptr;
skipWhitespaceIfAny(buf);
}
else
first = false;
auto nested_type = determineDataTypeForSingleFieldImpl(buf, settings);
if (!nested_type)
return nullptr;
nested_types.push_back(nested_type);
}
if (buf.eof())
return nullptr;
++buf.position();
if (nested_types.empty())
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeNothing>());
transformInferredTypesIfNeeded(nested_types, settings);
auto least_supertype = tryGetLeastSupertype(nested_types);
if (!least_supertype)
return nullptr;
return std::make_shared<DataTypeArray>(least_supertype);
}
/// Tuple
if (checkChar('(', buf))
{
skipWhitespaceIfAny(buf);
DataTypes nested_types;
bool first = true;
while (!buf.eof() && *buf.position() != ')')
{
if (!first)
{
skipWhitespaceIfAny(buf);
if (!checkChar(',', buf))
return nullptr;
skipWhitespaceIfAny(buf);
}
else
first = false;
auto nested_type = determineDataTypeForSingleFieldImpl(buf, settings);
if (!nested_type)
return nullptr;
nested_types.push_back(nested_type);
}
if (buf.eof() || nested_types.empty())
return nullptr;
++buf.position();
return std::make_shared<DataTypeTuple>(nested_types);
}
/// Map
if (checkChar('{', buf))
{
skipWhitespaceIfAny(buf);
DataTypes key_types;
DataTypes value_types;
bool first = true;
while (!buf.eof() && *buf.position() != '}')
{
if (!first)
{
skipWhitespaceIfAny(buf);
if (!checkChar(',', buf))
return nullptr;
skipWhitespaceIfAny(buf);
}
else
first = false;
auto key_type = determineDataTypeForSingleFieldImpl(buf, settings);
if (!key_type)
return nullptr;
key_types.push_back(key_type);
skipWhitespaceIfAny(buf);
if (!checkChar(':', buf))
return nullptr;
skipWhitespaceIfAny(buf);
auto value_type = determineDataTypeForSingleFieldImpl(buf, settings);
if (!value_type)
return nullptr;
value_types.push_back(value_type);
}
if (buf.eof())
return nullptr;
++buf.position();
skipWhitespaceIfAny(buf);
if (key_types.empty())
return std::make_shared<DataTypeMap>(std::make_shared<DataTypeNothing>(), std::make_shared<DataTypeNothing>());
transformInferredTypesIfNeeded(key_types, settings);
transformInferredTypesIfNeeded(value_types, settings);
auto key_least_supertype = tryGetLeastSupertype(key_types);
auto value_least_supertype = tryGetLeastSupertype(value_types);
if (!key_least_supertype || !value_least_supertype)
return nullptr;
if (!DataTypeMap::checkKeyType(key_least_supertype))
return nullptr;
return std::make_shared<DataTypeMap>(key_least_supertype, value_least_supertype);
}
/// String
if (*buf.position() == '\'')
{
++buf.position();
String field;
while (!buf.eof())
{
char * next_pos = find_first_symbols<'\\', '\''>(buf.position(), buf.buffer().end());
field.append(buf.position(), next_pos);
buf.position() = next_pos;
if (!buf.hasPendingData())
continue;
if (*buf.position() == '\'')
break;
field.push_back(*buf.position());
if (*buf.position() == '\\')
++buf.position();
}
if (buf.eof())
return nullptr;
++buf.position();
if (auto type = tryInferDateOrDateTime(field, settings))
return type;
return std::make_shared<DataTypeString>();
}
/// Bool
if (checkStringCaseInsensitive("true", buf) || checkStringCaseInsensitive("false", buf))
return DataTypeFactory::instance().get("Bool");
/// Null
if (checkStringCaseInsensitive("NULL", buf))
return std::make_shared<DataTypeNothing>();
/// Number
Float64 tmp;
auto * pos_before_float = buf.position();
if (tryReadFloatText(tmp, buf))
{
if (settings.try_infer_integers)
{
auto * float_end_pos = buf.position();
buf.position() = pos_before_float;
Int64 tmp_int;
if (tryReadIntText(tmp_int, buf) && buf.position() == float_end_pos)
return std::make_shared<DataTypeInt64>();
buf.position() = float_end_pos;
}
return std::make_shared<DataTypeFloat64>();
}
return nullptr;
}
static DataTypePtr determineDataTypeForSingleField(ReadBufferFromString & buf, const FormatSettings & settings)
{
return makeNullableRecursivelyAndCheckForNothing(determineDataTypeForSingleFieldImpl(buf, settings));
}
DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule)
DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info)
{
switch (escaping_rule)
{
case FormatSettings::EscapingRule::Quoted:
{
ReadBufferFromString buf(field);
auto type = determineDataTypeForSingleField(buf, format_settings);
return buf.eof() ? type : nullptr;
}
return tryInferDataTypeForSingleField(field, format_settings);
case FormatSettings::EscapingRule::JSON:
return JSONUtils::getDataTypeFromField(field, format_settings);
return tryInferDataTypeForSingleJSONField(field, format_settings, json_info);
case FormatSettings::EscapingRule::CSV:
{
if (!format_settings.csv.use_best_effort_in_schema_inference)
return makeNullable(std::make_shared<DataTypeString>());
return std::make_shared<DataTypeString>();
if (field.empty() || field == format_settings.csv.null_representation)
if (field.empty())
return nullptr;
if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation)
return DataTypeFactory::instance().get("Nullable(Bool)");
if (field == format_settings.csv.null_representation)
return makeNullable(std::make_shared<DataTypeNothing>());
if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation)
return DataTypeFactory::instance().get("Bool");
/// In CSV complex types are serialized in quotes. If we have quotes, we should try to infer type
/// from data inside quotes.
if (field.size() > 1 && ((field.front() == '\'' && field.back() == '\'') || (field.front() == '"' && field.back() == '"')))
{
auto data = std::string_view(field.data() + 1, field.size() - 2);
if (auto date_type = tryInferDateOrDateTime(data, format_settings))
/// First, try to infer dates and datetimes.
if (auto date_type = tryInferDateOrDateTimeFromString(data, format_settings))
return date_type;
ReadBufferFromString buf(data);
/// Try to determine the type of value inside quotes
auto type = determineDataTypeForSingleField(buf, format_settings);
auto type = tryInferDataTypeForSingleField(data, format_settings);
if (!type)
return nullptr;
/// If it's a number or tuple in quotes or there is some unread data in buffer, we determine it as a string.
if (isNumber(removeNullable(type)) || isTuple(type) || !buf.eof())
return makeNullable(std::make_shared<DataTypeString>());
/// If we couldn't infer any type or it's a number or tuple in quotes, we determine it as a string.
if (!type || isNumber(removeNullable(type)) || isTuple(type))
return std::make_shared<DataTypeString>();
return type;
}
/// Case when CSV value is not in quotes. Check if it's a number, and if not, determine it's as a string.
if (format_settings.try_infer_integers)
{
ReadBufferFromString buf(field);
Int64 tmp_int;
if (tryReadIntText(tmp_int, buf) && buf.eof())
return makeNullable(std::make_shared<DataTypeInt64>());
}
auto type = tryInferNumberFromString(field, format_settings);
ReadBufferFromString buf(field);
Float64 tmp;
if (tryReadFloatText(tmp, buf) && buf.eof())
return makeNullable(std::make_shared<DataTypeFloat64>());
if (!type)
return std::make_shared<DataTypeString>();
return makeNullable(std::make_shared<DataTypeString>());
return type;
}
case FormatSettings::EscapingRule::Raw: [[fallthrough]];
case FormatSettings::EscapingRule::Escaped:
{
if (!format_settings.tsv.use_best_effort_in_schema_inference)
return makeNullable(std::make_shared<DataTypeString>());
return std::make_shared<DataTypeString>();
if (field.empty() || field == format_settings.tsv.null_representation)
if (field.empty())
return nullptr;
if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation)
return DataTypeFactory::instance().get("Nullable(Bool)");
if (field == format_settings.tsv.null_representation)
return makeNullable(std::make_shared<DataTypeNothing>());
if (auto date_type = tryInferDateOrDateTime(field, format_settings))
if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation)
return DataTypeFactory::instance().get("Bool");
if (auto date_type = tryInferDateOrDateTimeFromString(field, format_settings))
return date_type;
ReadBufferFromString buf(field);
auto type = determineDataTypeForSingleField(buf, format_settings);
if (!buf.eof())
return makeNullable(std::make_shared<DataTypeString>());
auto type = tryInferDataTypeForSingleField(field, format_settings);
if (!type)
return std::make_shared<DataTypeString>();
return type;
}
default:
@ -818,15 +328,34 @@ DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSe
}
}
DataTypes determineDataTypesByEscapingRule(const std::vector<String> & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule)
DataTypes tryInferDataTypesByEscapingRule(const std::vector<String> & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info)
{
DataTypes data_types;
data_types.reserve(fields.size());
for (const auto & field : fields)
data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, escaping_rule));
data_types.push_back(tryInferDataTypeByEscapingRule(field, format_settings, escaping_rule, json_info));
return data_types;
}
void transformInferredTypesByEscapingRuleIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info)
{
switch (escaping_rule)
{
case FormatSettings::EscapingRule::JSON:
transformInferredJSONTypesIfNeeded(first, second, settings, json_info);
break;
case FormatSettings::EscapingRule::Escaped: [[fallthrough]];
case FormatSettings::EscapingRule::Raw: [[fallthrough]];
case FormatSettings::EscapingRule::Quoted: [[fallthrough]];
case FormatSettings::EscapingRule::CSV:
transformInferredTypesIfNeeded(first, second, settings);
break;
default:
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot transform inferred types for value with {} escaping rule", escapingRuleToString(escaping_rule));
}
}
DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escaping_rule)
{
switch (escaping_rule)
@ -834,7 +363,7 @@ DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escap
case FormatSettings::EscapingRule::CSV:
case FormatSettings::EscapingRule::Escaped:
case FormatSettings::EscapingRule::Raw:
return makeNullable(std::make_shared<DataTypeString>());
return std::make_shared<DataTypeString>();
default:
return nullptr;
}
@ -851,9 +380,10 @@ DataTypes getDefaultDataTypeForEscapingRules(const std::vector<FormatSettings::E
String getAdditionalFormatInfoForAllRowBasedFormats(const FormatSettings & settings)
{
return fmt::format(
"schema_inference_hints={}, max_rows_to_read_for_schema_inference={}",
"schema_inference_hints={}, max_rows_to_read_for_schema_inference={}, schema_inference_make_columns_nullable={}",
settings.schema_inference_hints,
settings.max_rows_to_read_for_schema_inference);
settings.max_rows_to_read_for_schema_inference,
settings.schema_inference_make_columns_nullable);
}
String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule)
@ -890,7 +420,11 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo
settings.csv.tuple_delimiter);
break;
case FormatSettings::EscapingRule::JSON:
result += fmt::format(", try_infer_numbers_from_strings={}, read_bools_as_numbers={}", settings.json.try_infer_numbers_from_strings, settings.json.read_bools_as_numbers);
result += fmt::format(
", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, try_infer_objects={}",
settings.json.try_infer_numbers_from_strings,
settings.json.read_bools_as_numbers,
settings.json.try_infer_objects);
break;
default:
break;

View File

@ -1,6 +1,7 @@
#pragma once
#include <Formats/FormatSettings.h>
#include <Formats/SchemaInferenceUtils.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/Serializations/ISerialization.h>
#include <IO/ReadBuffer.h>
@ -38,45 +39,17 @@ String readFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule es
/// Try to determine the type of the field written by a specific escaping rule.
/// If cannot, return nullptr.
/// - For Quoted escaping rule we can interpret a single field as a constant
/// expression and get it's type by evaluation this expression.
/// - For JSON escaping rule we can use JSON parser to parse a single field
/// and then convert JSON type of this field to ClickHouse type.
/// - For CSV escaping rule we can do the next:
/// - If the field is an unquoted string, then we try to parse it as a number,
/// and if we cannot, treat it as a String.
/// - If the field is a string in quotes, then we try to use some
/// tweaks and heuristics to determine the type inside quotes, and if we can't or
/// the result is a number or tuple (we don't parse numbers in quotes and don't
/// support tuples in CSV) we treat it as a String.
/// - If input_format_csv_use_best_effort_in_schema_inference is disabled, we
/// treat everything as a string.
/// - For TSV and TSVRaw we try to use some tweaks and heuristics to determine the type
/// of value if setting input_format_tsv_use_best_effort_in_schema_inference is enabled,
/// otherwise we treat everything as a string.
DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule);
DataTypes determineDataTypesByEscapingRule(const std::vector<String> & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule);
/// See tryInferDataTypeForSingle(JSON)Field in SchemaInferenceUtils.h
DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info = nullptr);
DataTypes tryInferDataTypesByEscapingRule(const std::vector<String> & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info = nullptr);
/// Check if we need to transform types inferred from data and transform it if necessary.
/// See transformInferred(JSON)TypesIfNeeded in SchemaInferenceUtils.h
void transformInferredTypesByEscapingRuleIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info = nullptr);
DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escaping_rule);
DataTypes getDefaultDataTypeForEscapingRules(const std::vector<FormatSettings::EscapingRule> & escaping_rules);
/// Try to infer Date or Datetime from string if corresponding settings are enabled.
DataTypePtr tryInferDateOrDateTime(const std::string_view & field, const FormatSettings & settings);
/// Check if we need to transform types inferred from data and transform it if necessary.
/// It's used when we try to infer some not ordinary types from another types.
/// For example dates from strings, we should check if dates were inferred from all strings
/// in the same way and if not, transform inferred dates back to strings.
/// For example, if we have array of strings and we tried to infer dates from them,
/// to make the result type Array(Date) we should ensure that all strings were
/// successfully parsed as dated and if not, convert all dates back to strings and make result type Array(String).
void transformInferredTypesIfNeeded(DataTypes & types, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule = FormatSettings::EscapingRule::Escaped);
void transformInferredTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule = FormatSettings::EscapingRule::Escaped);
/// Same as transformInferredTypesIfNeeded but takes into account settings that are special for JSON formats.
void transformInferredJSONTypesIfNeeded(DataTypes & types, const FormatSettings & settings, const std::unordered_set<const IDataType *> * numbers_parsed_from_json_strings = nullptr);
void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings);
String getAdditionalFormatInfoForAllRowBasedFormats(const FormatSettings & settings);
String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule);

View File

@ -169,6 +169,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.max_rows_to_read_for_schema_inference = settings.input_format_max_rows_to_read_for_schema_inference;
format_settings.column_names_for_schema_inference = settings.column_names_for_schema_inference;
format_settings.schema_inference_hints = settings.schema_inference_hints;
format_settings.schema_inference_make_columns_nullable = settings.schema_inference_make_columns_nullable;
format_settings.mysql_dump.table_name = settings.input_format_mysql_dump_table_name;
format_settings.mysql_dump.map_column_names = settings.input_format_mysql_dump_map_column_names;
format_settings.sql_insert.max_batch_size = settings.output_format_sql_insert_max_batch_size;
@ -182,6 +183,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string;
format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference;
format_settings.max_binary_string_size = settings.format_binary_max_string_size;
format_settings.max_parser_depth = context->getSettingsRef().max_parser_depth;
/// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context
if (format_settings.schema.is_server)

View File

@ -71,6 +71,8 @@ struct FormatSettings
Raw
};
bool schema_inference_make_columns_nullable = true;
DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple;
bool input_format_ipv4_default_on_conversion_error = false;
@ -81,6 +83,8 @@ struct FormatSettings
UInt64 max_binary_string_size = 0;
UInt64 max_parser_depth = DBMS_DEFAULT_MAX_PARSER_DEPTH;
struct
{
UInt64 row_group_size = 1000000;

View File

@ -6,19 +6,13 @@
#include <IO/WriteBufferValidUTF8.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeObject.h>
#include <DataTypes/DataTypeFactory.h>
#include <Common/JSONParsers/SimdJSONParser.h>
#include <Common/JSONParsers/RapidJSONParser.h>
#include <Common/JSONParsers/DummyJSONParser.h>
#include <base/find_symbols.h>
#include <Common/logger_useful.h>
namespace DB
{
@ -122,206 +116,6 @@ namespace JSONUtils
return {loadAtPosition(in, memory, pos), number_of_rows};
}
template <const char opening_bracket, const char closing_bracket>
static String readJSONEachRowLineIntoStringImpl(ReadBuffer & in)
{
Memory memory;
fileSegmentationEngineJSONEachRowImpl<opening_bracket, closing_bracket>(in, memory, 0, 1, 1);
return String(memory.data(), memory.size());
}
template <class Element>
DataTypePtr getDataTypeFromFieldImpl(const Element & field, const FormatSettings & settings, std::unordered_set<const IDataType *> & numbers_parsed_from_json_strings)
{
if (field.isNull())
return nullptr;
if (field.isBool())
return DataTypeFactory::instance().get("Nullable(Bool)");
if (field.isInt64() || field.isUInt64())
{
if (settings.try_infer_integers)
return makeNullable(std::make_shared<DataTypeInt64>());
return makeNullable(std::make_shared<DataTypeFloat64>());
}
if (field.isDouble())
return makeNullable(std::make_shared<DataTypeFloat64>());
if (field.isString())
{
if (auto date_type = tryInferDateOrDateTime(field.getString(), settings))
return date_type;
if (!settings.json.try_infer_numbers_from_strings)
return makeNullable(std::make_shared<DataTypeString>());
ReadBufferFromString buf(field.getString());
if (settings.try_infer_integers)
{
Int64 tmp_int;
if (tryReadIntText(tmp_int, buf) && buf.eof())
{
auto type = std::make_shared<DataTypeInt64>();
numbers_parsed_from_json_strings.insert(type.get());
return makeNullable(type);
}
}
Float64 tmp;
if (tryReadFloatText(tmp, buf) && buf.eof())
{
auto type = std::make_shared<DataTypeFloat64>();
numbers_parsed_from_json_strings.insert(type.get());
return makeNullable(type);
}
return makeNullable(std::make_shared<DataTypeString>());
}
if (field.isArray())
{
auto array = field.getArray();
/// Return nullptr in case of empty array because we cannot determine nested type.
if (array.size() == 0)
return nullptr;
DataTypes nested_data_types;
/// If this array contains fields with different types we will treat it as Tuple.
bool are_types_the_same = true;
for (const auto element : array)
{
auto type = getDataTypeFromFieldImpl(element, settings, numbers_parsed_from_json_strings);
if (!type)
return nullptr;
if (!nested_data_types.empty() && !type->equals(*nested_data_types.back()))
are_types_the_same = false;
nested_data_types.push_back(std::move(type));
}
if (!are_types_the_same)
{
auto nested_types_copy = nested_data_types;
transformInferredJSONTypesIfNeeded(nested_types_copy, settings, &numbers_parsed_from_json_strings);
are_types_the_same = true;
for (size_t i = 1; i < nested_types_copy.size(); ++i)
are_types_the_same &= nested_types_copy[i]->equals(*nested_types_copy[i - 1]);
if (are_types_the_same)
nested_data_types = std::move(nested_types_copy);
}
if (!are_types_the_same)
return std::make_shared<DataTypeTuple>(nested_data_types);
return std::make_shared<DataTypeArray>(nested_data_types.back());
}
if (field.isObject())
{
auto object = field.getObject();
DataTypes value_types;
for (const auto key_value_pair : object)
{
auto type = getDataTypeFromFieldImpl(key_value_pair.second, settings, numbers_parsed_from_json_strings);
if (!type)
{
/// If we couldn't infer nested type and Object type is not enabled,
/// we can't determine the type of this JSON field.
if (!settings.json.try_infer_objects)
{
/// If read_objects_as_strings is enabled, we can read objects into strings.
if (settings.json.read_objects_as_strings)
return makeNullable(std::make_shared<DataTypeString>());
return nullptr;
}
continue;
}
if (settings.json.try_infer_objects && isObject(type))
return std::make_shared<DataTypeObject>("json", true);
value_types.push_back(type);
}
if (value_types.empty())
return nullptr;
transformInferredJSONTypesIfNeeded(value_types, settings, &numbers_parsed_from_json_strings);
bool are_types_equal = true;
for (size_t i = 1; i < value_types.size(); ++i)
are_types_equal &= value_types[i]->equals(*value_types[0]);
if (!are_types_equal)
{
if (!settings.json.try_infer_objects)
{
/// If read_objects_as_strings is enabled, we can read objects into strings.
if (settings.json.read_objects_as_strings)
return makeNullable(std::make_shared<DataTypeString>());
return nullptr;
}
return std::make_shared<DataTypeObject>("json", true);
}
return std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), value_types[0]);
}
throw Exception{ErrorCodes::INCORRECT_DATA, "Unexpected JSON type"};
}
auto getJSONParserAndElement()
{
#if USE_SIMDJSON
return std::pair<SimdJSONParser, SimdJSONParser::Element>();
#elif USE_RAPIDJSON
return std::pair<RapidJSONParser, RapidJSONParser::Element>();
#else
return std::pair<DummyJSONParser, DummyJSONParser::Element>();
#endif
}
DataTypePtr getDataTypeFromField(const String & field, const FormatSettings & settings)
{
auto [parser, element] = getJSONParserAndElement();
bool parsed = parser.parse(field, element);
if (!parsed)
throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse JSON object here: {}", field);
std::unordered_set<const IDataType *> numbers_parsed_from_json_strings;
return getDataTypeFromFieldImpl(element, settings, numbers_parsed_from_json_strings);
}
template <class Extractor, const char opening_bracket, const char closing_bracket>
static DataTypes determineColumnDataTypesFromJSONEachRowDataImpl(ReadBuffer & in, const FormatSettings & settings, bool /*json_strings*/, Extractor & extractor)
{
String line = readJSONEachRowLineIntoStringImpl<opening_bracket, closing_bracket>(in);
auto [parser, element] = getJSONParserAndElement();
bool parsed = parser.parse(line, element);
if (!parsed)
throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse JSON object here: {}", line);
auto fields = extractor.extract(element);
DataTypes data_types;
data_types.reserve(fields.size());
std::unordered_set<const IDataType *> numbers_parsed_from_json_strings;
for (const auto & field : fields)
data_types.push_back(getDataTypeFromFieldImpl(field, settings, numbers_parsed_from_json_strings));
/// TODO: For JSONStringsEachRow/JSONCompactStringsEach all types will be strings.
/// Should we try to parse data inside strings somehow in this case?
return data_types;
}
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t max_rows)
{
return fileSegmentationEngineJSONEachRowImpl<'{', '}'>(in, memory, min_bytes, 1, max_rows);
@ -333,68 +127,56 @@ namespace JSONUtils
return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_bytes, min_rows, max_rows);
}
struct JSONEachRowFieldsExtractor
NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, const FormatSettings & settings, JSONInferenceInfo * inference_info)
{
template <class Element>
std::vector<Element> extract(const Element & element)
skipWhitespaceIfAny(in);
assertChar('{', in);
bool first = true;
NamesAndTypesList names_and_types;
String field;
while (!in.eof() && *in.position() != '}')
{
/// {..., "<column_name>" : <value>, ...}
if (!first)
skipComma(in);
else
first = false;
if (!element.isObject())
throw Exception(ErrorCodes::INCORRECT_DATA, "Root JSON value is not an object");
auto object = element.getObject();
std::vector<Element> fields;
fields.reserve(object.size());
column_names.reserve(object.size());
for (const auto & key_value_pair : object)
{
column_names.emplace_back(key_value_pair.first);
fields.push_back(key_value_pair.second);
}
return fields;
auto name = readFieldName(in);
auto type = tryInferDataTypeForSingleJSONField(in, settings, inference_info);
names_and_types.emplace_back(name, type);
}
std::vector<String> column_names;
};
if (in.eof())
throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF while reading JSON object");
NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings)
{
JSONEachRowFieldsExtractor extractor;
auto data_types
= determineColumnDataTypesFromJSONEachRowDataImpl<JSONEachRowFieldsExtractor, '{', '}'>(in, settings, json_strings, extractor);
NamesAndTypesList result;
for (size_t i = 0; i != extractor.column_names.size(); ++i)
result.emplace_back(extractor.column_names[i], data_types[i]);
return result;
assertChar('}', in);
return names_and_types;
}
struct JSONCompactEachRowFieldsExtractor
DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, const FormatSettings & settings, JSONInferenceInfo * inference_info)
{
template <class Element>
std::vector<Element> extract(const Element & element)
skipWhitespaceIfAny(in);
assertChar('[', in);
bool first = true;
DataTypes types;
String field;
while (!in.eof() && *in.position() != ']')
{
/// [..., <value>, ...]
if (!element.isArray())
throw Exception(ErrorCodes::INCORRECT_DATA, "Root JSON value is not an array");
auto array = element.getArray();
std::vector<Element> fields;
fields.reserve(array.size());
for (size_t i = 0; i != array.size(); ++i)
fields.push_back(array[i]);
return fields;
if (!first)
skipComma(in);
else
first = false;
auto type = tryInferDataTypeForSingleJSONField(in, settings, inference_info);
types.push_back(std::move(type));
}
};
DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings)
{
JSONCompactEachRowFieldsExtractor extractor;
return determineColumnDataTypesFromJSONEachRowDataImpl<JSONCompactEachRowFieldsExtractor, '[', ']'>(in, settings, json_strings, extractor);
if (in.eof())
throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF while reading JSON array");
assertChar(']', in);
return types;
}
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf)
{
/// For JSONEachRow we can safely skip whitespace characters

View File

@ -13,24 +13,21 @@
namespace DB
{
struct JSONInferenceInfo;
namespace JSONUtils
{
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t max_rows);
std::pair<bool, size_t> fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t min_rows, size_t max_rows);
/// Parse JSON from string and convert it's type to ClickHouse type. Make the result type always Nullable.
/// JSON array with different nested types is treated as Tuple.
/// If cannot convert (for example when field contains null), return nullptr.
DataTypePtr getDataTypeFromField(const String & field, const FormatSettings & settings);
/// Read row in JSONEachRow format and try to determine type for each field.
/// Return list of names and types.
/// If cannot determine the type of some field, return nullptr for it.
NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings);
NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, const FormatSettings & settings, JSONInferenceInfo * inference_info);
/// Read row in JSONCompactEachRow format and try to determine type for each field.
/// If cannot determine the type of some field, return nullptr for it.
DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings);
DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, const FormatSettings & settings, JSONInferenceInfo * inference_info);
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf);

View File

@ -197,69 +197,6 @@ ColumnsDescription readSchemaFromFormat(const String & format_name, const std::o
return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out);
}
DataTypePtr makeNullableRecursivelyAndCheckForNothing(DataTypePtr type)
{
if (!type)
return nullptr;
WhichDataType which(type);
if (which.isNothing())
return nullptr;
if (which.isNullable())
{
const auto * nullable_type = assert_cast<const DataTypeNullable *>(type.get());
return makeNullableRecursivelyAndCheckForNothing(nullable_type->getNestedType());
}
if (which.isArray())
{
const auto * array_type = assert_cast<const DataTypeArray *>(type.get());
auto nested_type = makeNullableRecursivelyAndCheckForNothing(array_type->getNestedType());
return nested_type ? std::make_shared<DataTypeArray>(nested_type) : nullptr;
}
if (which.isTuple())
{
const auto * tuple_type = assert_cast<const DataTypeTuple *>(type.get());
DataTypes nested_types;
for (const auto & element : tuple_type->getElements())
{
auto nested_type = makeNullableRecursivelyAndCheckForNothing(element);
if (!nested_type)
return nullptr;
nested_types.push_back(nested_type);
}
return std::make_shared<DataTypeTuple>(std::move(nested_types));
}
if (which.isMap())
{
const auto * map_type = assert_cast<const DataTypeMap *>(type.get());
auto key_type = makeNullableRecursivelyAndCheckForNothing(map_type->getKeyType());
auto value_type = makeNullableRecursivelyAndCheckForNothing(map_type->getValueType());
return key_type && value_type ? std::make_shared<DataTypeMap>(removeNullable(key_type), value_type) : nullptr;
}
if (which.isLowCarnality())
{
const auto * lc_type = assert_cast<const DataTypeLowCardinality *>(type.get());
auto nested_type = makeNullableRecursivelyAndCheckForNothing(lc_type->getDictionaryType());
return nested_type ? std::make_shared<DataTypeLowCardinality>(nested_type) : nullptr;
}
return makeNullable(type);
}
NamesAndTypesList getNamesAndRecursivelyNullableTypes(const Block & header)
{
NamesAndTypesList result;
for (auto & [name, type] : header.getNamesAndTypesList())
result.emplace_back(name, makeNullableRecursivelyAndCheckForNothing(type));
return result;
}
SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional<FormatSettings> & format_settings, const ContextPtr & context)
{
return getKeysForSchemaCache({source}, format, format_settings, context).front();

View File

@ -35,21 +35,7 @@ ColumnsDescription readSchemaFromFormat(
ContextPtr & context,
std::unique_ptr<ReadBuffer> & buf_out);
/// Make type Nullable recursively:
/// - Type -> Nullable(type)
/// - Array(Type) -> Array(Nullable(Type))
/// - Tuple(Type1, ..., TypeN) -> Tuple(Nullable(Type1), ..., Nullable(TypeN))
/// - Map(KeyType, ValueType) -> Map(KeyType, Nullable(ValueType))
/// - LowCardinality(Type) -> LowCardinality(Nullable(Type))
/// If type is Nothing or one of the nested types is Nothing, return nullptr.
DataTypePtr makeNullableRecursivelyAndCheckForNothing(DataTypePtr type);
/// Call makeNullableRecursivelyAndCheckForNothing for all types
/// in the block and return names and types.
NamesAndTypesList getNamesAndRecursivelyNullableTypes(const Block & header);
SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional<FormatSettings> & format_settings, const ContextPtr & context);
SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String & format, const std::optional<FormatSettings> & format_settings, const ContextPtr & context);
void splitSchemaCacheKey(const String & key, String & source, String & format, String & additional_format_info);
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,93 @@
#pragma once
#include <DataTypes/IDataType.h>
#include <IO/ReadBuffer.h>
namespace DB
{
/// Struct with some additional information about inferred types for JSON formats.
struct JSONInferenceInfo
{
/// We store numbers that were parsed from strings.
/// It's used in types transformation to change such numbers back to string if needed.
std::unordered_set<const IDataType *> numbers_parsed_from_json_strings;
/// Indicates if currently we are inferring type for Map/Object key.
bool is_object_key = false;
};
/// Try to determine datatype of the value in buffer/string. If the type cannot be inferred, return nullptr.
/// In general, it tries to parse a type using the following logic:
/// If we see '[', we try to parse an array of values and recursively determine datatype for each element.
/// If we see '(', we try to parse a tuple of values and recursively determine datatype for each element.
/// If we see '{', we try to parse a Map of keys and values and recursively determine datatype for each key/value.
/// If we see a quote '\'', we treat it as a string and read until next quote.
/// If we see NULL it returns Nullable(Nothing)
/// Otherwise we try to read a number.
DataTypePtr tryInferDataTypeForSingleField(ReadBuffer & buf, const FormatSettings & settings);
DataTypePtr tryInferDataTypeForSingleField(std::string_view field, const FormatSettings & settings);
/// The same as tryInferDataTypeForSingleField, but for JSON values.
DataTypePtr tryInferDataTypeForSingleJSONField(ReadBuffer & buf, const FormatSettings & settings, JSONInferenceInfo * json_info);
DataTypePtr tryInferDataTypeForSingleJSONField(std::string_view field, const FormatSettings & settings, JSONInferenceInfo * json_info);
/// Try to parse Date or DateTime value from a string.
DataTypePtr tryInferDateOrDateTimeFromString(std::string_view field, const FormatSettings & settings);
/// Try to parse a number value from a string. By default, it tries to parse Float64,
/// but if setting try_infer_integers is enabled, it also tries to parse Int64.
DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSettings & settings);
/// It takes two types inferred for the same column and tries to transform them to a common type if possible.
/// It's also used when we try to infer some not ordinary types from another types.
/// Example 1:
/// Dates inferred from strings. In this case we should check if dates were inferred from all strings
/// in the same way and if not, transform inferred dates back to strings.
/// For example, when we have Array(Date) (like `['2020-01-01', '2020-02-02']`) and Array(String) (like `['string', 'abc']`
/// we will convert the first type to Array(String).
/// Example 2:
/// When we have integers and floats for the same value, we should convert all integers to floats.
/// For example, when we have Array(Int64) (like `[123, 456]`) and Array(Float64) (like `[42.42, 4.42]`)
/// we will convert the first type to Array(Float64)
/// Example 3:
/// When we have not complete types like Nullable(Nothing), Array(Nullable(Nothing)) or Tuple(UInt64, Nullable(Nothing)),
/// we try to complete them using the other type.
/// For example, if we have Tuple(UInt64, Nullable(Nothing)) and Tuple(Nullable(Nothing), String) we will convert both
/// types to common type Tuple(Nullable(UInt64), Nullable(String))
void transformInferredTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings);
/// The same as transformInferredTypesIfNeeded but uses some specific transformations for JSON.
/// Example 1:
/// When we have numbers inferred from strings and strings, we convert all such numbers back to string.
/// For example, if we have Array(Int64) (like `['123', '456']`) and Array(String) (like `['str', 'abc']`)
/// we will convert the first type to Array(String). Note that we collect information about numbers inferred
/// from strings in json_info while inference and use it here, so we will know that Array(Int64) contains
/// integer inferred from a string.
/// Example 2:
/// When we have maps with different value types, we convert all types to JSON object type.
/// For example, if we have Map(String, UInt64) (like `{"a" : 123}`) and Map(String, String) (like `{"b" : 'abc'}`)
/// we will convert both types to Object('JSON').
void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, JSONInferenceInfo * json_info);
/// Check if type is Tuple(...), try to transform nested types to find a common type for them and if all nested types
/// are the same after transform, we convert this tuple to an Array with common nested type.
/// For example, if we have Tuple(String, Nullable(Nothing)) we will convert it to Array(String).
/// It's used when all rows were read and we have Tuple in the result type that can be actually an Array.
void transformJSONTupleToArrayIfPossible(DataTypePtr & data_type, const FormatSettings & settings, JSONInferenceInfo * json_info);
/// Make type Nullable recursively:
/// - Type -> Nullable(type)
/// - Array(Type) -> Array(Nullable(Type))
/// - Tuple(Type1, ..., TypeN) -> Tuple(Nullable(Type1), ..., Nullable(TypeN))
/// - Map(KeyType, ValueType) -> Map(KeyType, Nullable(ValueType))
/// - LowCardinality(Type) -> LowCardinality(Nullable(Type))
DataTypePtr makeNullableRecursively(DataTypePtr type);
/// Call makeNullableRecursively for all types
/// in the block and return names and types.
NamesAndTypesList getNamesAndRecursivelyNullableTypes(const Block & header);
/// Check if type contains Nothing, like Array(Tuple(Nullable(Nothing), String))
bool checkIfTypeIsComplete(const DataTypePtr & type);
}

View File

@ -9,6 +9,7 @@
#include <Interpreters/castColumn.h>
#include <Interpreters/Context.h>
#include <numeric>
#include <vector>
namespace DB
@ -56,7 +57,7 @@ private:
for (const auto & arg : arguments)
{
if (!isUnsignedInteger(arg))
if (!isInteger(arg))
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
@ -72,8 +73,12 @@ private:
{
const auto & in_data = in->getData();
const auto total_values = std::accumulate(std::begin(in_data), std::end(in_data), size_t{},
[this] (const size_t lhs, const size_t rhs)
[this] (const size_t lhs, const T rhs)
{
if (rhs < 0)
throw Exception{"A call to function " + getName() + " overflows, only support positive values when only end is provided",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
const auto sum = lhs + rhs;
if (sum < lhs)
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
@ -96,7 +101,7 @@ private:
IColumn::Offset offset{};
for (size_t row_idx = 0, rows = in->size(); row_idx < rows; ++row_idx)
{
for (size_t elem_idx = 0, elems = in_data[row_idx]; elem_idx < elems; ++elem_idx)
for (T elem_idx = 0, elems = in_data[row_idx]; elem_idx < elems; ++elem_idx)
out_data[offset + elem_idx] = static_cast<T>(elem_idx);
offset += in_data[row_idx];
@ -121,15 +126,20 @@ private:
size_t total_values = 0;
size_t pre_values = 0;
std::vector<size_t> row_length(input_rows_count);
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
{
if (start < end_data[row_idx] && step == 0)
if (step == 0)
throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
pre_values += start >= end_data[row_idx] ? 0
: (end_data[row_idx] - start - 1) / step + 1;
if (start < end_data[row_idx] && step > 0)
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start) - 1) / static_cast<__int128_t>(step) + 1;
else if (start > end_data[row_idx] && step < 0)
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start) + 1) / static_cast<__int128_t>(step) + 1;
pre_values += row_length[row_idx];
if (pre_values < total_values)
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
@ -151,15 +161,8 @@ private:
IColumn::Offset offset{};
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
{
for (size_t st = start, ed = end_data[row_idx]; st < ed; st += step)
{
out_data[offset++] = static_cast<T>(st);
if (st > st + step)
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
}
for (size_t idx = 0; idx < row_length[row_idx]; idx++)
out_data[offset++] = static_cast<T>(start + offset * step);
out_offsets[row_idx] = offset;
}
@ -180,19 +183,25 @@ private:
size_t total_values = 0;
size_t pre_values = 0;
std::vector<size_t> row_length(input_rows_count);
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
{
if (start_data[row_idx] < end_data[row_idx] && step == 0)
if (step == 0)
throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
pre_values += start_data[row_idx] >= end_data[row_idx] ? 0
: (end_data[row_idx] - start_data[row_idx] - 1) / step + 1;
if (start_data[row_idx] < end_data[row_idx] && step > 0)
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start_data[row_idx]) - 1) / static_cast<__int128_t>(step) + 1;
else if (start_data[row_idx] > end_data[row_idx] && step < 0)
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start_data[row_idx]) + 1) / static_cast<__int128_t>(step) + 1;
pre_values += row_length[row_idx];
if (pre_values < total_values)
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
total_values = pre_values;
if (total_values > max_elements)
@ -210,15 +219,8 @@ private:
IColumn::Offset offset{};
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
{
for (size_t st = start_data[row_idx], ed = end_data[row_idx]; st < ed; st += step)
{
out_data[offset++] = static_cast<T>(st);
if (st > st + step)
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
}
for (size_t idx = 0; idx < row_length[row_idx]; idx++)
out_data[offset++] = static_cast<T>(start_data[row_idx] + idx * step);
out_offsets[row_idx] = offset;
}
@ -239,15 +241,20 @@ private:
size_t total_values = 0;
size_t pre_values = 0;
std::vector<size_t> row_length(input_rows_count);
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
{
if (start < end_data[row_idx] && step_data[row_idx] == 0)
if (step_data[row_idx] == 0)
throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
pre_values += start >= end_data[row_idx] ? 0
: (end_data[row_idx] - start - 1) / step_data[row_idx] + 1;
if (start < end_data[row_idx] && step_data[row_idx] > 0)
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start) - 1) / static_cast<__int128_t>(step_data[row_idx]) + 1;
else if (start > end_data[row_idx] && step_data[row_idx] < 0)
row_length[row_idx] = (static_cast<__int128_t>(end_data[row_idx]) - static_cast<__int128_t>(start) + 1) / static_cast<__int128_t>(step_data[row_idx]) + 1;
pre_values += row_length[row_idx];
if (pre_values < total_values)
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
@ -269,15 +276,8 @@ private:
IColumn::Offset offset{};
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
{
for (size_t st = start, ed = end_data[row_idx]; st < ed; st += step_data[row_idx])
{
out_data[offset++] = static_cast<T>(st);
if (st > st + step_data[row_idx])
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
}
for (size_t idx = 0; idx < row_length[row_idx]; idx++)
out_data[offset++] = static_cast<T>(start + offset * step_data[row_idx]);
out_offsets[row_idx] = offset;
}
@ -301,15 +301,19 @@ private:
size_t total_values = 0;
size_t pre_values = 0;
std::vector<size_t> row_length(input_rows_count);
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
{
if (start_data[row_idx] < end_start[row_idx] && step_data[row_idx] == 0)
throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero",
if (step_data[row_idx] == 0)
throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't less or equal to zero",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
if (start_data[row_idx] < end_start[row_idx] && step_data[row_idx] > 0)
row_length[row_idx] = (static_cast<__int128_t>(end_start[row_idx]) - static_cast<__int128_t>(start_data[row_idx]) - 1) / static_cast<__int128_t>(step_data[row_idx]) + 1;
else if (start_data[row_idx] > end_start[row_idx] && step_data[row_idx] < 0)
row_length[row_idx] = (static_cast<__int128_t>(end_start[row_idx]) - static_cast<__int128_t>(start_data[row_idx]) + 1) / static_cast<__int128_t>(step_data[row_idx]) + 1;
pre_values += start_data[row_idx] >= end_start[row_idx] ? 0
: (end_start[row_idx] -start_data[row_idx] - 1) / (step_data[row_idx]) + 1;
pre_values += row_length[row_idx];
if (pre_values < total_values)
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
@ -331,15 +335,8 @@ private:
IColumn::Offset offset{};
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
{
for (size_t st = start_data[row_idx], ed = end_start[row_idx]; st < ed; st += step_data[row_idx])
{
out_data[offset++] = static_cast<T>(st);
if (st > st + step_data[row_idx])
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
}
for (size_t idx = 0; idx < row_length[row_idx]; idx++)
out_data[offset++] = static_cast<T>(start_data[row_idx] + idx * step_data[row_idx]);
out_offsets[row_idx] = offset;
}
@ -351,23 +348,20 @@ private:
DataTypePtr elem_type = checkAndGetDataType<DataTypeArray>(result_type.get())->getNestedType();
WhichDataType which(elem_type);
if (!which.isUInt8()
&& !which.isUInt16()
&& !which.isUInt32()
&& !which.isUInt64())
if (!which.isNativeUInt() && !which.isNativeInt())
{
throw Exception{"Illegal columns of arguments of function " + getName()
+ ", the function only implemented for unsigned integers up to 64 bit", ErrorCodes::ILLEGAL_COLUMN};
+ ", the function only implemented for unsigned/signed integers up to 64 bit",
ErrorCodes::ILLEGAL_COLUMN};
}
ColumnPtr res;
if (arguments.size() == 1)
{
const auto * col = arguments[0].column.get();
if (!((res = executeInternal<UInt8>(col))
|| (res = executeInternal<UInt16>(col))
|| (res = executeInternal<UInt32>(col))
|| (res = executeInternal<UInt64>(col))))
if (!((res = executeInternal<UInt8>(col)) || (res = executeInternal<UInt16>(col)) || (res = executeInternal<UInt32>(col))
|| (res = executeInternal<UInt64>(col)) || (res = executeInternal<Int8>(col)) || (res = executeInternal<Int16>(col))
|| (res = executeInternal<Int32>(col)) || (res = executeInternal<Int64>(col))))
{
throw Exception{"Illegal column " + col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
}
@ -402,44 +396,93 @@ private:
bool is_step_const = isColumnConst(*column_ptrs[2]);
if (is_start_const && is_step_const)
{
UInt64 start = assert_cast<const ColumnConst &>(*column_ptrs[0]).getUInt(0);
UInt64 step = assert_cast<const ColumnConst &>(*column_ptrs[2]).getUInt(0);
if ((res = executeConstStartStep<UInt8>(column_ptrs[1], start, step, input_rows_count)) ||
(res = executeConstStartStep<UInt16>(column_ptrs[1], start, step, input_rows_count)) ||
(res = executeConstStartStep<UInt32>(column_ptrs[1], static_cast<UInt32>(start), static_cast<UInt32>(step), input_rows_count)) ||
(res = executeConstStartStep<UInt64>(column_ptrs[1], start, step, input_rows_count)))
if (which.isNativeUInt())
{
UInt64 start = assert_cast<const ColumnConst &>(*column_ptrs[0]).getUInt(0);
UInt64 step = assert_cast<const ColumnConst &>(*column_ptrs[2]).getUInt(0);
if ((res = executeConstStartStep<UInt8>(column_ptrs[1], start, step, input_rows_count))
|| (res = executeConstStartStep<UInt16>(column_ptrs[1], start, step, input_rows_count))
|| (res = executeConstStartStep<UInt32>(
column_ptrs[1], static_cast<UInt32>(start), static_cast<UInt32>(step), input_rows_count))
|| (res = executeConstStartStep<UInt64>(column_ptrs[1], start, step, input_rows_count)))
{
}
}
else if (which.isNativeInt())
{
Int64 start = assert_cast<const ColumnConst &>(*column_ptrs[0]).getInt(0);
Int64 step = assert_cast<const ColumnConst &>(*column_ptrs[2]).getInt(0);
if ((res = executeConstStartStep<Int8>(column_ptrs[1], start, step, input_rows_count))
|| (res = executeConstStartStep<Int16>(column_ptrs[1], start, step, input_rows_count))
|| (res = executeConstStartStep<Int32>(
column_ptrs[1], static_cast<Int32>(start), static_cast<Int32>(step), input_rows_count))
|| (res = executeConstStartStep<Int64>(column_ptrs[1], start, step, input_rows_count)))
{
}
}
}
else if (is_start_const && !is_step_const)
{
UInt64 start = assert_cast<const ColumnConst &>(*column_ptrs[0]).getUInt(0);
if ((res = executeConstStart<UInt8>(column_ptrs[1], column_ptrs[2], start, input_rows_count)) ||
(res = executeConstStart<UInt16>(column_ptrs[1], column_ptrs[2], start, input_rows_count)) ||
(res = executeConstStart<UInt32>(column_ptrs[1], column_ptrs[2], static_cast<UInt32>(start), input_rows_count)) ||
(res = executeConstStart<UInt64>(column_ptrs[1], column_ptrs[2], start, input_rows_count)))
if (which.isNativeUInt())
{
UInt64 start = assert_cast<const ColumnConst &>(*column_ptrs[0]).getUInt(0);
if ((res = executeConstStart<UInt8>(column_ptrs[1], column_ptrs[2], start, input_rows_count))
|| (res = executeConstStart<UInt16>(column_ptrs[1], column_ptrs[2], start, input_rows_count))
|| (res = executeConstStart<UInt32>(column_ptrs[1], column_ptrs[2], static_cast<UInt32>(start), input_rows_count))
|| (res = executeConstStart<UInt64>(column_ptrs[1], column_ptrs[2], start, input_rows_count)))
{
}
}
else if (which.isNativeInt())
{
Int64 start = assert_cast<const ColumnConst &>(*column_ptrs[0]).getInt(0);
if ((res = executeConstStart<Int8>(column_ptrs[1], column_ptrs[2], start, input_rows_count))
|| (res = executeConstStart<Int16>(column_ptrs[1], column_ptrs[2], start, input_rows_count))
|| (res = executeConstStart<Int32>(column_ptrs[1], column_ptrs[2], static_cast<Int32>(start), input_rows_count))
|| (res = executeConstStart<Int64>(column_ptrs[1], column_ptrs[2], start, input_rows_count)))
{
}
}
}
else if (!is_start_const && is_step_const)
{
UInt64 step = assert_cast<const ColumnConst &>(*column_ptrs[2]).getUInt(0);
if ((res = executeConstStep<UInt8>(column_ptrs[0], column_ptrs[1], step, input_rows_count)) ||
(res = executeConstStep<UInt16>(column_ptrs[0], column_ptrs[1], step, input_rows_count)) ||
(res = executeConstStep<UInt32>(column_ptrs[0], column_ptrs[1], static_cast<UInt32>(step), input_rows_count)) ||
(res = executeConstStep<UInt64>(column_ptrs[0], column_ptrs[1], step, input_rows_count)))
if (which.isNativeUInt())
{
UInt64 step = assert_cast<const ColumnConst &>(*column_ptrs[2]).getUInt(0);
if ((res = executeConstStep<UInt8>(column_ptrs[0], column_ptrs[1], step, input_rows_count))
|| (res = executeConstStep<UInt16>(column_ptrs[0], column_ptrs[1], step, input_rows_count))
|| (res = executeConstStep<UInt32>(column_ptrs[0], column_ptrs[1], static_cast<UInt32>(step), input_rows_count))
|| (res = executeConstStep<UInt64>(column_ptrs[0], column_ptrs[1], step, input_rows_count)))
{
}
}
else if (which.isNativeInt())
{
Int64 step = assert_cast<const ColumnConst &>(*column_ptrs[2]).getInt(0);
if ((res = executeConstStep<Int8>(column_ptrs[0], column_ptrs[1], step, input_rows_count))
|| (res = executeConstStep<Int16>(column_ptrs[0], column_ptrs[1], step, input_rows_count))
|| (res = executeConstStep<Int32>(column_ptrs[0], column_ptrs[1], static_cast<Int32>(step), input_rows_count))
|| (res = executeConstStep<Int64>(column_ptrs[0], column_ptrs[1], step, input_rows_count)))
{
}
}
}
else
{
if ((res = executeGeneric<UInt8>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count)) ||
(res = executeGeneric<UInt16>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count)) ||
(res = executeGeneric<UInt32>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count)) ||
(res = executeGeneric<UInt64>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count)))
if ((res = executeGeneric<UInt8>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|| (res = executeGeneric<UInt16>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|| (res = executeGeneric<UInt32>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|| (res = executeGeneric<UInt64>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|| (res = executeGeneric<Int8>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|| (res = executeGeneric<Int16>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|| (res = executeGeneric<Int32>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))
|| (res = executeGeneric<Int64>(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count)))
{
}
}

53
src/IO/IResourceManager.h Normal file
View File

@ -0,0 +1,53 @@
#pragma once
#include <IO/ResourceRequest.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <boost/noncopyable.hpp>
#include <memory>
#include <unordered_map>
namespace DB
{
/*
* Instance of derived class holds everything required for resource consumption,
* including resources currently registered at `SchedulerRoot`. This is required to avoid
* problems during configuration update. Do not hold instances longer than required.
* Should be created on query start and destructed when query is done.
*/
class IClassifier : private boost::noncopyable
{
public:
virtual ~IClassifier() {}
/// Returns ResouceLink that should be used to access resource.
/// Returned link is valid until classifier destruction.
virtual ResourceLink get(const String & resource_name) = 0;
};
using ClassifierPtr = std::shared_ptr<IClassifier>;
/*
* Represents control plane of resource scheduling. Derived class is responsible for reading
* configuration, creating all required `ISchedulerNode` objects and
* managing their lifespan.
*/
class IResourceManager : private boost::noncopyable
{
public:
virtual ~IResourceManager() {}
/// Initialize or reconfigure manager.
virtual void updateConfiguration(const Poco::Util::AbstractConfiguration & config) = 0;
/// Obtain a classifier instance required to get access to resources.
/// Note that it holds resource configuration, so should be destructed when query is done.
virtual ClassifierPtr acquire(const String & classifier_name) = 0;
};
using ResourceManagerPtr = std::shared_ptr<IResourceManager>;
}

View File

@ -0,0 +1,55 @@
#pragma once
#include <IO/ISchedulerNode.h>
namespace DB
{
/*
* Constraint defined on the set of requests in consumption state.
* It allows to track two events:
* - dequeueRequest(): resource consumption begins
* - finishRequest(): resource consumption finishes
* This allows to keep track of in-flight requests and implement different constraints (e.g. in-flight limit).
* When constraint is violated, node must be deactivated by dequeueRequest() returning `false`.
* When constraint is again satisfied, scheduleActivation() is called from finishRequest().
*
* Derived class behaviour requirements:
* - dequeueRequest() must fill `request->constraint` iff it is nullptr;
* - finishRequest() must be recursive: call to `parent_constraint->finishRequest()`.
*/
class ISchedulerConstraint : public ISchedulerNode
{
public:
ISchedulerConstraint(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
: ISchedulerNode(event_queue_, config, config_prefix)
{}
/// Resource consumption by `request` is finished.
/// Should be called outside of scheduling subsystem, implementation must be thread-safe.
virtual void finishRequest(ResourceRequest * request) = 0;
void setParent(ISchedulerNode * parent_) override
{
ISchedulerNode::setParent(parent_);
// Assign `parent_constraint` to the nearest parent derived from ISchedulerConstraint
for (ISchedulerNode * node = parent_; node != nullptr; node = node->parent)
{
if (auto * constraint = dynamic_cast<ISchedulerConstraint *>(node))
{
parent_constraint = constraint;
break;
}
}
}
protected:
// Reference to nearest parent that is also derived from ISchedulerConstraint.
// Request can traverse through multiple constraints while being dequeue from hierarchy,
// while finishing request should traverse the same chain in reverse order.
// NOTE: it must be immutable after initialization, because it is accessed in not thread-safe way from finishRequest()
ISchedulerConstraint * parent_constraint = nullptr;
};
}

221
src/IO/ISchedulerNode.h Normal file
View File

@ -0,0 +1,221 @@
#pragma once
#include <Common/ErrorCodes.h>
#include <Common/Exception.h>
#include <IO/ResourceRequest.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/XMLConfiguration.h>
#include <boost/noncopyable.hpp>
#include <deque>
#include <functional>
#include <memory>
#include <mutex>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_SCHEDULER_NODE;
}
class ISchedulerNode;
inline const Poco::Util::AbstractConfiguration & emptyConfig()
{
static Poco::AutoPtr<Poco::Util::XMLConfiguration> config = new Poco::Util::XMLConfiguration();
return *config;
}
/*
* Info read and write for scheduling purposes by parent
*/
struct SchedulerNodeInfo
{
double weight = 1.0; /// Weight of this node among it's siblings
Int64 priority = 0; /// Priority of this node among it's siblings (higher value means higher priority)
/// Arbitrary data accessed/stored by parent
union {
size_t idx;
void * ptr;
} parent;
SchedulerNodeInfo() = default;
explicit SchedulerNodeInfo(const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
{
setWeight(config.getDouble(config_prefix + ".weight", weight));
setPriority(config.getInt64(config_prefix + ".priority", priority));
}
void setWeight(double value)
{
if (value <= 0 || !isfinite(value))
throw Exception(
ErrorCodes::INVALID_SCHEDULER_NODE,
"Negative and non-finite node weights are not allowed: {}",
value);
weight = value;
}
void setPriority(Int64 value)
{
priority = value;
}
};
/*
* Simple waitable thread-safe FIFO task queue.
* Intended to hold postponed events for later handling (usually by scheduler thread).
*/
class EventQueue
{
public:
using Event = std::function<void()>;
void enqueue(Event&& event)
{
std::unique_lock lock{mutex};
bool was_empty = queue.empty();
queue.emplace_back(event);
if (was_empty)
pending.notify_one();
}
/// Process single event if it exists
/// Returns `true` iff event has been processed
bool tryProcess()
{
std::unique_lock lock{mutex};
if (queue.empty())
return false;
Event event = std::move(queue.front());
queue.pop_front();
lock.unlock(); // do not hold queue mutext while processing events
event();
return true;
}
/// Wait for single event (if not available) and process it
void process()
{
std::unique_lock lock{mutex};
pending.wait(lock, [&] { return !queue.empty(); });
Event event = std::move(queue.front());
queue.pop_front();
lock.unlock(); // do not hold queue mutext while processing events
event();
}
private:
std::mutex mutex;
std::condition_variable pending;
std::deque<Event> queue;
};
/*
* Node of hierarchy for scheduling requests for resource. Base class for all
* kinds of scheduling elements (queues, policies, constraints and schedulers).
*
* Root node is a scheduler, which has it's thread to dequeue requests,
* execute requests (see ResourceRequest) and process events in a thread-safe manner.
* Immediate children of the scheduler represent independent resources.
* Each resource has it's own hierarchy to achieve required scheduling policies.
* Non-leaf nodes do not hold requests, but keep scheduling state
* (e.g. consumption history, amount of in-flight requests, etc).
* Leafs of hierarchy are queues capable of holding pending requests.
*
* scheduler (SchedulerRoot)
* / \
* constraint constraint (SemaphoreConstraint)
* | |
* policy policy (PriorityPolicy)
* / \ / \
* q1 q2 q3 q4 (FifoQueue)
*
* Dequeueing request from an inner node will dequeue request from one of active leaf-queues in its subtree.
* Node is considered to be active iff:
* - it has at least one pending request in one of leaves of it's subtree;
* - and enforced constraints, if any, are satisfied
* (e.g. amount of concurrent requests is not greater than some number).
*
* All methods must be called only from scheduler thread for thread-safety.
*/
class ISchedulerNode : private boost::noncopyable
{
public:
ISchedulerNode(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
: event_queue(event_queue_)
, info(config, config_prefix)
{}
virtual ~ISchedulerNode() {}
// Checks if two nodes configuration is equal
virtual bool equals(ISchedulerNode * other) = 0;
/// Attach new child
virtual void attachChild(const std::shared_ptr<ISchedulerNode> & child) = 0;
/// Detach and destroy child
virtual void removeChild(ISchedulerNode * child) = 0;
/// Get attached child by name
virtual ISchedulerNode * getChild(const String & child_name) = 0;
/// Activation of child due to the first pending request
/// Should be called on leaf node (i.e. queue) to propagate activation signal through chain to the root
virtual void activateChild(ISchedulerNode * child) = 0;
/// Returns true iff node is active
virtual bool isActive() = 0;
/// Returns the first request to be executed as the first component of resuting pair.
/// The second pair component is `true` iff node is still active after dequeueing.
virtual std::pair<ResourceRequest *, bool> dequeueRequest() = 0;
/// Returns full path string using names of every parent
String getPath()
{
String result;
ISchedulerNode * ptr = this;
while (ptr->parent)
{
result = "/" + ptr->basename + result;
ptr = ptr->parent;
}
return result.empty() ? "/" : result;
}
/// Attach to a parent (used by attachChild)
virtual void setParent(ISchedulerNode * parent_)
{
parent = parent_;
}
protected:
/// Notify parents about the first pending request or constraint becoming satisfied.
/// Postponed to be handled in scheduler thread, so it is intended to be called from outside.
void scheduleActivation()
{
if (likely(parent))
{
event_queue->enqueue([this] { parent->activateChild(this); });
}
}
public:
EventQueue * const event_queue;
String basename;
SchedulerNodeInfo info;
ISchedulerNode * parent = nullptr;
};
using SchedulerNodePtr = std::shared_ptr<ISchedulerNode>;
}

26
src/IO/ISchedulerQueue.h Normal file
View File

@ -0,0 +1,26 @@
#pragma once
#include <IO/ISchedulerNode.h>
#include <memory>
namespace DB
{
/*
* Queue for pending requests for specific resource, leaf of hierarchy.
*/
class ISchedulerQueue : public ISchedulerNode
{
public:
ISchedulerQueue(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
: ISchedulerNode(event_queue_, config, config_prefix)
{}
/// Enqueue new request to be executed using underlying resource.
/// Should be called outside of scheduling subsystem, implementation must be thread-safe.
virtual void enqueueRequest(ResourceRequest * request) = 0;
};
}

View File

@ -319,12 +319,17 @@ template void readStringUntilEOFInto<PaddedPODArray<UInt8>>(PaddedPODArray<UInt8
/** Parse the escape sequence, which can be simple (one character after backslash) or more complex (multiple characters).
* It is assumed that the cursor is located on the `\` symbol
*/
template <typename Vector>
static void parseComplexEscapeSequence(Vector & s, ReadBuffer & buf)
template <typename Vector, typename ReturnType = void>
static ReturnType parseComplexEscapeSequence(Vector & s, ReadBuffer & buf)
{
++buf.position();
if (buf.eof())
throw Exception("Cannot parse escape sequence", ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE);
{
if constexpr (std::is_same_v<ReturnType, void>)
throw Exception("Cannot parse escape sequence", ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE);
else
return ReturnType(false);
}
char char_after_backslash = *buf.position();
@ -363,6 +368,8 @@ static void parseComplexEscapeSequence(Vector & s, ReadBuffer & buf)
s.push_back(decoded_char);
++buf.position();
}
return ReturnType(true);
}
@ -521,14 +528,18 @@ template void readEscapedStringInto<NullOutput>(NullOutput & s, ReadBuffer & buf
* backslash escape sequences are also parsed,
* that could be slightly confusing.
*/
template <char quote, bool enable_sql_style_quoting, typename Vector>
static void readAnyQuotedStringInto(Vector & s, ReadBuffer & buf)
template <char quote, bool enable_sql_style_quoting, typename Vector, typename ReturnType = void>
static ReturnType readAnyQuotedStringInto(Vector & s, ReadBuffer & buf)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
if (buf.eof() || *buf.position() != quote)
{
throw ParsingException(ErrorCodes::CANNOT_PARSE_QUOTED_STRING,
"Cannot parse quoted string: expected opening quote '{}', got '{}'",
std::string{quote}, buf.eof() ? "EOF" : std::string{*buf.position()});
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_QUOTED_STRING,
"Cannot parse quoted string: expected opening quote '{}', got '{}'",
std::string{quote}, buf.eof() ? "EOF" : std::string{*buf.position()});
else
return ReturnType(false);
}
++buf.position();
@ -554,15 +565,26 @@ static void readAnyQuotedStringInto(Vector & s, ReadBuffer & buf)
continue;
}
return;
return ReturnType(true);
}
if (*buf.position() == '\\')
parseComplexEscapeSequence(s, buf);
{
if constexpr (throw_exception)
parseComplexEscapeSequence<Vector, ReturnType>(s, buf);
else
{
if (!parseComplexEscapeSequence<Vector, ReturnType>(s, buf))
return ReturnType(false);
}
}
}
throw ParsingException("Cannot parse quoted string: expected closing quote",
ErrorCodes::CANNOT_PARSE_QUOTED_STRING);
if constexpr (throw_exception)
throw ParsingException("Cannot parse quoted string: expected closing quote",
ErrorCodes::CANNOT_PARSE_QUOTED_STRING);
else
return ReturnType(false);
}
template <bool enable_sql_style_quoting, typename Vector>
@ -571,6 +593,14 @@ void readQuotedStringInto(Vector & s, ReadBuffer & buf)
readAnyQuotedStringInto<'\'', enable_sql_style_quoting>(s, buf);
}
template <typename Vector>
bool tryReadQuotedStringInto(Vector & s, ReadBuffer & buf)
{
return readAnyQuotedStringInto<'\'', false, Vector, bool>(s, buf);
}
template bool tryReadQuotedStringInto(String & s, ReadBuffer & buf);
template <bool enable_sql_style_quoting, typename Vector>
void readDoubleQuotedStringInto(Vector & s, ReadBuffer & buf)
{
@ -934,6 +964,7 @@ template void readJSONStringInto<PaddedPODArray<UInt8>, void>(PaddedPODArray<UIn
template bool readJSONStringInto<PaddedPODArray<UInt8>, bool>(PaddedPODArray<UInt8> & s, ReadBuffer & buf);
template void readJSONStringInto<NullOutput>(NullOutput & s, ReadBuffer & buf);
template void readJSONStringInto<String>(String & s, ReadBuffer & buf);
template bool readJSONStringInto<String, bool>(String & s, ReadBuffer & buf);
template <typename Vector, typename ReturnType>
ReturnType readJSONObjectPossiblyInvalid(Vector & s, ReadBuffer & buf)
@ -1501,6 +1532,43 @@ static void readParsedValueInto(Vector & s, ReadBuffer & buf, ParseFunc parse_fu
peekable_buf.position() = end;
}
template <typename Vector>
static void readQuotedStringFieldInto(Vector & s, ReadBuffer & buf)
{
assertChar('\'', buf);
s.push_back('\'');
while (!buf.eof())
{
char * next_pos = find_first_symbols<'\\', '\''>(buf.position(), buf.buffer().end());
s.append(buf.position(), next_pos);
buf.position() = next_pos;
if (!buf.hasPendingData())
continue;
if (*buf.position() == '\'')
break;
s.push_back(*buf.position());
if (*buf.position() == '\\')
{
++buf.position();
if (!buf.eof())
{
s.push_back(*buf.position());
++buf.position();
}
}
}
if (buf.eof())
return;
++buf.position();
s.push_back('\'');
}
template <char opening_bracket, char closing_bracket, typename Vector>
static void readQuotedFieldInBracketsInto(Vector & s, ReadBuffer & buf)
{
@ -1518,20 +1586,19 @@ static void readQuotedFieldInBracketsInto(Vector & s, ReadBuffer & buf)
if (!buf.hasPendingData())
continue;
s.push_back(*buf.position());
if (*buf.position() == '\'')
{
readQuotedStringInto<false>(s, buf);
s.push_back('\'');
readQuotedStringFieldInto(s, buf);
}
else if (*buf.position() == opening_bracket)
{
s.push_back(opening_bracket);
++balance;
++buf.position();
}
else if (*buf.position() == closing_bracket)
{
s.push_back(closing_bracket);
--balance;
++buf.position();
}
@ -1554,11 +1621,7 @@ void readQuotedFieldInto(Vector & s, ReadBuffer & buf)
/// - Number: integer, float, decimal.
if (*buf.position() == '\'')
{
s.push_back('\'');
readQuotedStringInto<false>(s, buf);
s.push_back('\'');
}
readQuotedStringFieldInto(s, buf);
else if (*buf.position() == '[')
readQuotedFieldInBracketsInto<'[', ']'>(s, buf);
else if (*buf.position() == '(')

View File

@ -613,6 +613,9 @@ bool tryReadJSONStringInto(Vector & s, ReadBuffer & buf)
return readJSONStringInto<Vector, bool>(s, buf);
}
template <typename Vector>
bool tryReadQuotedStringInto(Vector & s, ReadBuffer & buf);
/// Reads chunk of data between {} in that way,
/// that it has balanced parentheses sequence of {}.
/// So, it may form a JSON object, but it can be incorrenct.

View File

@ -0,0 +1,40 @@
#include <IO/Resource/ClassifiersConfig.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int RESOURCE_NOT_FOUND;
}
ClassifierDescription::ClassifierDescription(const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
for (const auto & key : keys)
emplace(key, config.getString(config_prefix + "." + key));
}
ClassifiersConfig::ClassifiersConfig(const Poco::Util::AbstractConfiguration & config)
{
Poco::Util::AbstractConfiguration::Keys keys;
const String config_prefix = "classifiers";
config.keys(config_prefix, keys);
for (const auto & key : keys)
classifiers.emplace(std::piecewise_construct,
std::forward_as_tuple(key),
std::forward_as_tuple(config, config_prefix + "." + key));
}
const ClassifierDescription & ClassifiersConfig::get(const String & classifier_name)
{
if (auto it = classifiers.find(classifier_name); it != classifiers.end())
return it->second;
else
throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Unknown classifier '{}' to access resources", classifier_name);
}
}

View File

@ -0,0 +1,39 @@
#pragma once
#include <base/types.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <unordered_map>
namespace DB
{
/// Mapping of resource name into path string (e.g. "disk1" -> "/path/to/class")
struct ClassifierDescription : std::unordered_map<String, String>
{
ClassifierDescription(const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
};
/*
* Loads a config with the following format:
* <classifiers>
* <classifier1>
* <resource1>/path/to/queue</resource1>
* <resource2>/path/to/another/queue</resource2>
* </classifier1>
* ...
* <classifierN>...</classifierN>
* </classifiers>
*/
class ClassifiersConfig
{
public:
ClassifiersConfig() = default;
explicit ClassifiersConfig(const Poco::Util::AbstractConfiguration & config);
const ClassifierDescription & get(const String & classifier_name);
private:
std::unordered_map<String, ClassifierDescription> classifiers; // by classifier_name
};
}

View File

@ -0,0 +1,13 @@
#include <IO/Resource/FifoQueue.h>
#include <IO/SchedulerNodeFactory.h>
namespace DB
{
void registerFifoQueue(SchedulerNodeFactory & factory)
{
factory.registerMethod<FifoQueue>("fifo");
}
}

View File

@ -0,0 +1,91 @@
#pragma once
#include <Common/Stopwatch.h>
#include <IO/ISchedulerQueue.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <deque>
#include <mutex>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_SCHEDULER_NODE;
}
/*
* FIFO queue to hold pending resource requests
*/
class FifoQueue : public ISchedulerQueue
{
public:
FifoQueue(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
: ISchedulerQueue(event_queue_, config, config_prefix)
{}
bool equals(ISchedulerNode * other) override
{
if (auto * o = dynamic_cast<FifoQueue *>(other))
return true;
return false;
}
void enqueueRequest(ResourceRequest * request) override
{
std::unique_lock lock(mutex);
request->enqueue_ns = clock_gettime_ns();
bool was_empty = requests.empty();
requests.push_back(request);
if (was_empty)
scheduleActivation();
}
std::pair<ResourceRequest *, bool> dequeueRequest() override
{
std::unique_lock lock(mutex);
if (requests.empty())
return {nullptr, false};
ResourceRequest * result = requests.front();
requests.pop_front();
return {result, !requests.empty()};
}
bool isActive() override
{
std::unique_lock lock(mutex);
return !requests.empty();
}
void activateChild(ISchedulerNode *) override
{
assert(false); // queue cannot have children
}
void attachChild(const SchedulerNodePtr &) override
{
throw Exception(
ErrorCodes::INVALID_SCHEDULER_NODE,
"Cannot add child to leaf scheduler queue: {}",
getPath());
}
void removeChild(ISchedulerNode *) override
{
}
ISchedulerNode * getChild(const String &) override
{
return nullptr;
}
private:
std::mutex mutex;
std::deque<ResourceRequest *> requests;
};
}

View File

@ -0,0 +1,13 @@
#include <IO/Resource/PriorityPolicy.h>
#include <IO/SchedulerNodeFactory.h>
namespace DB
{
void registerPriorityPolicy(SchedulerNodeFactory & factory)
{
factory.registerMethod<PriorityPolicy>("priority");
}
}

View File

@ -0,0 +1,143 @@
#pragma once
#include <IO/ISchedulerQueue.h>
#include <IO/SchedulerRoot.h>
#include <algorithm>
#include <unordered_map>
#include <vector>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_SCHEDULER_NODE;
}
/*
* Scheduler node that implements priority scheduling policy.
* Requests are scheduled in order of priorities.
*/
class PriorityPolicy : public ISchedulerNode
{
/// Scheduling state of a child
struct Item
{
ISchedulerNode * child = nullptr;
Int64 priority = 0; // higher value means higher priority
/// For max-heap by priority
bool operator<(const Item& rhs) const noexcept
{
return priority < rhs.priority;
}
};
public:
PriorityPolicy(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
: ISchedulerNode(event_queue_, config, config_prefix)
{}
bool equals(ISchedulerNode * other) override
{
if (auto * o = dynamic_cast<PriorityPolicy *>(other))
return true;
return false;
}
void attachChild(const SchedulerNodePtr & child) override
{
// Take ownership
chassert(child->parent == nullptr);
if (auto [it, inserted] = children.emplace(child->basename, child); !inserted)
throw Exception(
ErrorCodes::INVALID_SCHEDULER_NODE,
"Can't add another child with the same path: {}",
it->second->getPath());
// Attach
child->setParent(this);
// Activate child if it is not empty
if (child->isActive())
activateChild(child.get());
}
void removeChild(ISchedulerNode * child) override
{
if (auto iter = children.find(child->basename); iter != children.end())
{
SchedulerNodePtr removed = iter->second;
// Deactivate: detach is not very common operation, so we can afford O(N) here
for (auto i = items.begin(), e = items.end(); i != e; ++i)
{
if (i->child == removed.get())
{
items.erase(i);
// Element was removed from inside of heap -- heap must be rebuilt
std::make_heap(items.begin(), items.end());
break;
}
}
// Detach
removed->setParent(nullptr);
// Get rid of ownership
children.erase(iter);
}
}
ISchedulerNode * getChild(const String & child_name) override
{
if (auto iter = children.find(child_name); iter != children.end())
return iter->second.get();
else
return nullptr;
}
std::pair<ResourceRequest *, bool> dequeueRequest() override
{
if (items.empty())
return {nullptr, false};
// Recursively pull request from child
auto [request, child_active] = items.front().child->dequeueRequest();
assert(request != nullptr);
// Deactivate child if it is empty
if (!child_active)
{
std::pop_heap(items.begin(), items.end());
items.pop_back();
}
return {request, !items.empty()};
}
bool isActive() override
{
return !items.empty();
}
void activateChild(ISchedulerNode * child) override
{
bool activate_parent = items.empty();
items.emplace_back(Item{child, child->info.priority});
std::push_heap(items.begin(), items.end());
if (activate_parent && parent)
parent->activateChild(this);
}
private:
/// Heap of active children
std::vector<Item> items;
/// All children with ownership
std::unordered_map<String, SchedulerNodePtr> children; // basename -> child
};
}

View File

@ -0,0 +1,13 @@
#include <IO/Resource/SemaphoreConstraint.h>
#include <IO/SchedulerNodeFactory.h>
namespace DB
{
void registerSemaphoreConstraint(SchedulerNodeFactory & factory)
{
factory.registerMethod<SemaphoreConstraint>("inflight_limit");
}
}

View File

@ -0,0 +1,138 @@
#pragma once
#include <IO/ISchedulerConstraint.h>
#include <IO/SchedulerRoot.h>
#include <mutex>
#include <limits>
#include <utility>
namespace DB
{
/*
* Limited concurrency constraint.
* Blocks if either number of concurrent in-flight requests exceeds `max_requests`, or their total cost exceeds `max_cost`
*/
class SemaphoreConstraint : public ISchedulerConstraint
{
static constexpr Int64 default_max_requests = std::numeric_limits<Int64>::max();
static constexpr Int64 default_max_cost = std::numeric_limits<Int64>::max();
public:
SemaphoreConstraint(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
: ISchedulerConstraint(event_queue_, config, config_prefix)
, max_requests(config.getInt64(config_prefix + ".max_requests", default_max_requests))
, max_cost(config.getInt64(config_prefix + ".max_cost", config.getInt64(config_prefix + ".max_bytes", default_max_cost)))
{}
bool equals(ISchedulerNode * other) override
{
if (auto * o = dynamic_cast<SemaphoreConstraint *>(other))
return max_requests == o->max_requests && max_cost == o->max_cost;
return false;
}
void attachChild(const std::shared_ptr<ISchedulerNode> & child_) override
{
// Take ownership
child = child_;
child->setParent(this);
// Activate if required
if (child->isActive())
activateChild(child.get());
}
void removeChild(ISchedulerNode * child_) override
{
if (child.get() == child_)
{
child_active = false; // deactivate
child->setParent(nullptr); // detach
child.reset();
}
}
ISchedulerNode * getChild(const String & child_name) override
{
if (child->basename == child_name)
return child.get();
else
return nullptr;
}
std::pair<ResourceRequest *, bool> dequeueRequest() override
{
// Dequeue request from the child
auto [request, child_now_active] = child->dequeueRequest();
if (!request)
return {nullptr, false};
// Request has reference to the first (closest to leaf) `constraint`, which can have `parent_constraint`.
// The former is initialized here dynamically and the latter is initialized once during hierarchy construction.
if (!request->constraint)
request->constraint = this;
// Update state on request arrival
std::unique_lock lock(mutex);
requests++;
cost += request->cost;
child_active = child_now_active;
return {request, active()};
}
void finishRequest(ResourceRequest * request) override
{
// Recursive traverse of parent flow controls in reverse order
if (parent_constraint)
parent_constraint->finishRequest(request);
// Update state on request departure
std::unique_lock lock(mutex);
bool was_active = active();
requests--;
cost -= request->cost;
// Schedule activation on transition from inactive state
if (!was_active && active())
scheduleActivation();
}
void activateChild(ISchedulerNode * child_) override
{
std::unique_lock lock(mutex);
if (child_ == child.get())
if (!std::exchange(child_active, true) && satisfied() && parent)
parent->activateChild(this);
}
bool isActive() override
{
std::unique_lock lock(mutex);
return active();
}
private:
bool satisfied() const
{
return requests < max_requests && cost < max_cost;
}
bool active() const
{
return satisfied() && child_active;
}
private:
std::mutex mutex;
Int64 requests = 0;
Int64 cost = 0;
bool child_active = false;
SchedulerNodePtr child;
Int64 max_requests = default_max_requests;
Int64 max_cost = default_max_cost;
};
}

View File

@ -0,0 +1,138 @@
#include <IO/Resource/StaticResourceManager.h>
#include <IO/SchedulerNodeFactory.h>
#include <IO/ResourceManagerFactory.h>
#include <IO/ISchedulerQueue.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <map>
#include <tuple>
#include <algorithm>
namespace DB
{
namespace ErrorCodes
{
extern const int RESOURCE_ACCESS_DENIED;
extern const int RESOURCE_NOT_FOUND;
extern const int INVALID_SCHEDULER_NODE;
}
StaticResourceManager::Resource::Resource(
const String & name,
EventQueue * event_queue,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix)
{
// Initialize scheduler nodes
Poco::Util::AbstractConfiguration::Keys keys;
std::sort(keys.begin(), keys.end()); // for parents to appear before children
config.keys(config_prefix, keys);
for (const auto & key : keys)
{
if (!startsWith(key, "node"))
continue;
// Validate path
String path = config.getString(config_prefix + "." + key + "[@path]", "");
if (path.empty())
throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Attribute 'path' must be specified in all nodes for resource '{}'", name);
if (path[0] != '/')
throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "path must start with '/' for resource '{}'", name);
// Create node
String type = config.getString(config_prefix + "." + key + ".type", "fifo");
SchedulerNodePtr node = SchedulerNodeFactory::instance().get(type, event_queue, config, config_prefix + "." + key);
node->basename = path.substr(1);
// Take ownership
if (auto [_, inserted] = nodes.emplace(path, node); !inserted)
throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Duplicate path '{}' for resource '{}'", path, name);
// Attach created node to parent (if not root)
if (path != "/")
{
String parent_path = path.substr(0, path.rfind('/'));
if (parent_path.empty())
parent_path = "/";
if (auto parent = nodes.find(parent_path); parent != nodes.end())
parent->second->attachChild(node);
else
throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Parent doesn't exist for path '{}' for resource '{}'", path, name);
}
}
if (nodes.find("/") == nodes.end())
throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "undefined root node path '/' for resource '{}'", name);
}
StaticResourceManager::Classifier::Classifier(const StaticResourceManager & manager, const ClassifierDescription & cfg)
{
for (auto [resource_name, path] : cfg)
{
if (auto resource_iter = manager.resources.find(resource_name); resource_iter != manager.resources.end())
{
const Resource & resource = resource_iter->second;
if (auto node_iter = resource.nodes.find(path); node_iter != resource.nodes.end())
{
if (auto * queue = dynamic_cast<ISchedulerQueue *>(node_iter->second.get()))
resources.emplace(resource_name, ResourceLink{.queue = queue});
else
throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Unable to access non-queue node at path '{}' for resource '{}'", path, resource_name);
}
else
throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Path '{}' for resource '{}' does not exist", path, resource_name);
}
else
resources.emplace(resource_name, ResourceLink{}); // resource not configured - unlimited
}
}
ResourceLink StaticResourceManager::Classifier::get(const String & resource_name)
{
if (auto iter = resources.find(resource_name); iter != resources.end())
return iter->second;
else
throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Access denied to resource '{}'", resource_name);
}
void StaticResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration & config)
{
if (!resources.empty())
return; // already initialized, configuration update is not supported
Poco::Util::AbstractConfiguration::Keys keys;
const String config_prefix = "resources";
config.keys(config_prefix, keys);
// Create resource for every element under <resources> tag
for (const auto & key : keys)
{
auto [iter, _] = resources.emplace(std::piecewise_construct,
std::forward_as_tuple(key),
std::forward_as_tuple(key, scheduler.event_queue, config, config_prefix + "." + key));
// Attach root of resource to scheduler
scheduler.attachChild(iter->second.nodes.find("/")->second);
}
// Initialize classifiers
classifiers = std::make_unique<ClassifiersConfig>(config);
// Run scheduler thread
scheduler.start();
}
ClassifierPtr StaticResourceManager::acquire(const String & classifier_name)
{
return std::make_shared<Classifier>(*this, classifiers->get(classifier_name));
}
void registerStaticResourceManager(ResourceManagerFactory & factory)
{
factory.registerMethod<StaticResourceManager>("static");
}
}

View File

@ -0,0 +1,49 @@
#pragma once
#include <IO/IResourceManager.h>
#include <IO/SchedulerRoot.h>
#include <IO/Resource/ClassifiersConfig.h>
#include <mutex>
namespace DB
{
/*
* Reads `<resources>` from config at startup and registers them in single `SchedulerRoot`.
* Do not support configuration updates, server restart is required.
*/
class StaticResourceManager : public IResourceManager
{
public:
// Just initialization, any further updates are ignored for the sake of simplicity
// NOTE: manager must be initialized before any acquire() calls to avoid races
void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override;
ClassifierPtr acquire(const String & classifier_name) override;
private:
struct Resource
{
std::unordered_map<String, SchedulerNodePtr> nodes; // by paths
Resource(
const String & name,
EventQueue * event_queue,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix);
};
struct Classifier : public IClassifier
{
Classifier(const StaticResourceManager & manager, const ClassifierDescription & cfg);
ResourceLink get(const String & resource_name) override;
std::unordered_map<String, ResourceLink> resources; // accessible resources by names
};
SchedulerRoot scheduler;
std::unordered_map<String, Resource> resources; // by name
std::unique_ptr<ClassifiersConfig> classifiers;
};
}

View File

@ -0,0 +1,15 @@
#include <IO/Resource/registerResourceManagers.h>
#include <IO/ResourceManagerFactory.h>
namespace DB
{
void registerStaticResourceManager(ResourceManagerFactory &);
void registerResourceManagers()
{
auto & factory = ResourceManagerFactory::instance();
registerStaticResourceManager(factory);
}
}

View File

@ -0,0 +1,8 @@
#pragma once
namespace DB
{
void registerResourceManagers();
}

View File

@ -0,0 +1,28 @@
#include <IO/Resource/registerSchedulerNodes.h>
#include <IO/ISchedulerNode.h>
#include <IO/ISchedulerConstraint.h>
#include <IO/SchedulerNodeFactory.h>
namespace DB
{
void registerPriorityPolicy(SchedulerNodeFactory &);
void registerSemaphoreConstraint(SchedulerNodeFactory &);
void registerFifoQueue(SchedulerNodeFactory &);
void registerSchedulerNodes()
{
auto & factory = SchedulerNodeFactory::instance();
// ISchedulerNode
registerPriorityPolicy(factory);
// ISchedulerConstraint
registerSemaphoreConstraint(factory);
// ISchedulerQueue
registerFifoQueue(factory);
}
}

View File

@ -0,0 +1,8 @@
#pragma once
namespace DB
{
void registerSchedulerNodes();
}

View File

@ -0,0 +1,309 @@
#pragma once
#include <IO/IResourceManager.h>
#include <IO/SchedulerRoot.h>
#include <IO/ResourceGuard.h>
#include <IO/SchedulerNodeFactory.h>
#include <IO/Resource/PriorityPolicy.h>
#include <IO/Resource/FifoQueue.h>
#include <IO/Resource/SemaphoreConstraint.h>
#include <IO/Resource/registerSchedulerNodes.h>
#include <IO/Resource/registerResourceManagers.h>
#include <Poco/Util/XMLConfiguration.h>
#include <atomic>
#include <barrier>
#include <unordered_map>
#include <mutex>
#include <set>
#include <sstream>
namespace DB
{
struct ResourceTestBase
{
ResourceTestBase()
{
[[maybe_unused]] static bool typesRegistered = [] { registerSchedulerNodes(); registerResourceManagers(); return true; }();
}
template <class TClass>
static TClass * add(EventQueue * event_queue, SchedulerNodePtr & root_node, const String & path, const String & xml = {})
{
std::stringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
stream << "<resource><node path=\"" << path << "\">" << xml << "</node></resource>";
Poco::AutoPtr config{new Poco::Util::XMLConfiguration(stream)};
String config_prefix = "node";
if (path == "/")
{
EXPECT_TRUE(root_node.get() == nullptr);
root_node.reset(new TClass(event_queue, *config, config_prefix));
return static_cast<TClass *>(root_node.get());
}
EXPECT_TRUE(root_node.get() != nullptr); // root should be initialized first
ISchedulerNode * parent = root_node.get();
size_t pos = 1;
String child_name;
while (pos < path.length())
{
size_t slash = path.find('/', pos);
if (slash != String::npos)
{
parent = parent->getChild(path.substr(pos, slash - pos));
EXPECT_TRUE(parent != nullptr); // parent does not exist
pos = slash + 1;
}
else
{
child_name = path.substr(pos);
pos = String::npos;
}
}
EXPECT_TRUE(!child_name.empty()); // wrong path
SchedulerNodePtr node = std::make_shared<TClass>(event_queue, *config, config_prefix);
node->basename = child_name;
parent->attachChild(node);
return static_cast<TClass *>(node.get());
}
};
struct ConstraintTest : public SemaphoreConstraint
{
ConstraintTest(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
: SemaphoreConstraint(event_queue_, config, config_prefix)
{}
std::pair<ResourceRequest *, bool> dequeueRequest() override
{
auto [request, active] = SemaphoreConstraint::dequeueRequest();
if (request)
{
std::unique_lock lock(mutex);
requests.insert(request);
}
return {request, active};
}
void finishRequest(ResourceRequest * request) override
{
{
std::unique_lock lock(mutex);
requests.erase(request);
}
SemaphoreConstraint::finishRequest(request);
}
std::mutex mutex;
std::set<ResourceRequest *> requests;
};
class ResourceTestClass : public ResourceTestBase
{
struct Request : public ResourceRequest
{
String name;
Request(ResourceCost cost_, const String & name_)
: ResourceRequest(cost_)
, name(name_)
{}
void execute() override
{
}
};
public:
template <class TClass>
void add(const String & path, const String & xml = {})
{
ResourceTestBase::add<TClass>(&event_queue, root_node, path, xml);
}
void enqueue(const String & path, const std::vector<ResourceCost> & costs)
{
ASSERT_TRUE(root_node.get() != nullptr); // root should be initialized first
ISchedulerNode * node = root_node.get();
size_t pos = 1;
while (pos < path.length())
{
size_t slash = path.find('/', pos);
if (slash != String::npos)
{
node = node->getChild(path.substr(pos, slash - pos));
ASSERT_TRUE(node != nullptr); // does not exist
pos = slash + 1;
}
else
{
node = node->getChild(path.substr(pos));
pos = String::npos;
}
}
ISchedulerQueue * queue = dynamic_cast<ISchedulerQueue *>(node);
ASSERT_TRUE(queue != nullptr); // not a queue
for (ResourceCost cost : costs)
{
queue->enqueueRequest(new Request(cost, queue->basename));
}
processEvents(); // to activate queues
}
void dequeue(size_t count_limit = size_t(-1), ResourceCost cost_limit = ResourceCostMax)
{
while (count_limit > 0 && cost_limit > 0)
{
if (auto [request, _] = root_node->dequeueRequest(); request)
{
count_limit--;
cost_limit -= request->cost;
handle(static_cast<Request *>(request));
}
else
{
break;
}
}
}
void handle(Request * request)
{
consumed_cost[request->name] += request->cost;
delete request;
}
void consumed(const String & name, ResourceCost value, ResourceCost error = 0)
{
EXPECT_TRUE(consumed_cost[name] >= value - error);
EXPECT_TRUE(consumed_cost[name] <= value + error);
consumed_cost[name] -= value;
}
void processEvents()
{
while (event_queue.tryProcess()) {}
}
private:
EventQueue event_queue;
SchedulerNodePtr root_node;
std::unordered_map<String, ResourceCost> consumed_cost;
};
template <class TManager>
struct ResourceTestManager : public ResourceTestBase
{
ResourceManagerPtr manager;
std::vector<ThreadFromGlobalPool> threads;
std::barrier<> busy_period;
struct Guard : public ResourceGuard
{
ResourceTestManager & t;
Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost)
: ResourceGuard(link_, cost, PostponeLocking)
, t(t_)
{
t.onEnqueue(link);
lock();
t.onExecute(link);
}
};
struct TItem
{
std::atomic<Int64> enqueued = 0; // number of enqueued requests
std::atomic<Int64> left = 0; // number of requests left to be executed
};
struct ResourceQueueHash
{
size_t operator()(const ResourceLink & link) const
{
return std::hash<ISchedulerQueue*>()(link.queue);
}
};
std::mutex link_data_mutex;
std::unordered_map<ResourceLink, TItem, ResourceQueueHash> link_data;
explicit ResourceTestManager(size_t thread_count = 1)
: manager(new TManager)
, busy_period(thread_count)
{}
~ResourceTestManager()
{
for (auto & thread : threads)
thread.join();
}
void update(const String & xml)
{
std::istringstream stream(xml); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
Poco::AutoPtr config{new Poco::Util::XMLConfiguration(stream)};
manager->updateConfiguration(*config);
}
auto & getLinkData(ResourceLink link)
{
std::unique_lock lock{link_data_mutex};
return link_data[link];
}
// Use at least two threads for each queue to avoid queue being deactivated:
// while the first request is executing, the second request is in queue - holding it active.
// use onEnqueue() and onExecute() functions for this purpose.
void onEnqueue(ResourceLink link)
{
getLinkData(link).enqueued.fetch_add(1, std::memory_order_relaxed);
}
void onExecute(ResourceLink link)
{
auto & data = getLinkData(link);
Int64 left = data.left.fetch_sub(1, std::memory_order_relaxed) - 1;
Int64 enqueued = data.enqueued.fetch_sub(1, std::memory_order_relaxed) - 1;
while (left > 0 && enqueued <= 0) // Ensure at least one thread has already enqueued itself (or there is no more requests)
{
std::this_thread::yield();
left = data.left.load();
enqueued = data.enqueued.load();
}
}
// This is required for proper busy period start, i.e. everyone to be seen by scheduler as appeared at the same time:
// - resource is blocked with queries by leader thread;
// - leader thread notifies followers to enqueue their requests;
// - leader thread unblocks resource;
// - busy period begins.
// NOTE: actually leader's request(s) make their own small busy period.
void blockResource(ResourceLink link)
{
ResourceGuard g(link, 1, ResourceGuard::PostponeLocking);
g.lock();
// NOTE: at this point we assume resource to be blocked by single request (<max_requests>1</max_requests>)
busy_period.arrive_and_wait(); // (1) notify all followers that resource is blocked
busy_period.arrive_and_wait(); // (2) wait all followers to enqueue their requests
}
void startBusyPeriod(ResourceLink link, ResourceCost cost, Int64 total_requests)
{
getLinkData(link).left += total_requests + 1;
busy_period.arrive_and_wait(); // (1) wait leader to block resource
ResourceGuard g(link, cost, ResourceGuard::PostponeLocking);
onEnqueue(link);
busy_period.arrive_and_wait(); // (2) notify leader to unblock
g.lock();
onExecute(link);
}
};
}

View File

@ -0,0 +1,122 @@
#include <gtest/gtest.h>
#include <IO/Resource/tests/ResourceTest.h>
#include <IO/Resource/PriorityPolicy.h>
using namespace DB;
using ResourceTest = ResourceTestClass;
TEST(IOResourcePriorityPolicy, Factory)
{
ResourceTest t;
Poco::AutoPtr cfg = new Poco::Util::XMLConfiguration();
SchedulerNodePtr prio = SchedulerNodeFactory::instance().get("priority", /* event_queue = */ nullptr, *cfg, "");
EXPECT_TRUE(dynamic_cast<PriorityPolicy *>(prio.get()) != nullptr);
}
TEST(IOResourcePriorityPolicy, Priorities)
{
ResourceTest t;
t.add<PriorityPolicy>("/");
t.add<FifoQueue>("/A", "<priority>1</priority>");
t.add<FifoQueue>("/B", "<priority>2</priority>");
t.add<FifoQueue>("/C", "<priority>3</priority>");
t.enqueue("/A", {10, 10, 10});
t.enqueue("/B", {10, 10, 10});
t.enqueue("/C", {10, 10, 10});
t.dequeue(2);
t.consumed("A", 0);
t.consumed("B", 0);
t.consumed("C", 20);
t.dequeue(2);
t.consumed("A", 0);
t.consumed("B", 10);
t.consumed("C", 10);
t.dequeue(2);
t.consumed("A", 0);
t.consumed("B", 20);
t.consumed("C", 0);
t.dequeue();
t.consumed("A", 30);
t.consumed("B", 0);
t.consumed("C", 0);
}
TEST(IOResourcePriorityPolicy, Activation)
{
ResourceTest t;
t.add<PriorityPolicy>("/");
t.add<FifoQueue>("/A", "<priority>1</priority>");
t.add<FifoQueue>("/B", "<priority>2</priority>");
t.add<FifoQueue>("/C", "<priority>3</priority>");
t.enqueue("/A", {10, 10, 10, 10, 10, 10});
t.enqueue("/B", {10});
t.enqueue("/C", {10, 10});
t.dequeue(3);
t.consumed("A", 0);
t.consumed("B", 10);
t.consumed("C", 20);
t.dequeue(2);
t.consumed("A", 20);
t.consumed("B", 0);
t.consumed("C", 0);
t.enqueue("/B", {10, 10, 10});
t.dequeue(2);
t.consumed("A", 0);
t.consumed("B", 20);
t.consumed("C", 0);
t.enqueue("/C", {10, 10});
t.dequeue(3);
t.consumed("A", 0);
t.consumed("B", 10);
t.consumed("C", 20);
t.dequeue(2);
t.consumed("A", 20);
t.consumed("B", 0);
t.consumed("C", 0);
}
TEST(IOResourcePriorityPolicy, SinglePriority)
{
ResourceTest t;
t.add<PriorityPolicy>("/");
t.add<FifoQueue>("/A");
for (int i = 0; i < 3; i++)
{
t.enqueue("/A", {10, 10});
t.dequeue(1);
t.consumed("A", 10);
for (int j = 0; j < 3; j++)
{
t.enqueue("/A", {10, 10, 10});
t.dequeue(1);
t.consumed("A", 10);
t.dequeue(1);
t.consumed("A", 10);
t.dequeue(1);
t.consumed("A", 10);
}
t.dequeue(1);
t.consumed("A", 10);
}
}

View File

@ -0,0 +1,103 @@
#include <gtest/gtest.h>
#include <IO/Resource/tests/ResourceTest.h>
#include <IO/Resource/StaticResourceManager.h>
#include <Poco/Util/XMLConfiguration.h>
using namespace DB;
using ResourceTest = ResourceTestManager<StaticResourceManager>;
using TestGuard = ResourceTest::Guard;
TEST(IOResourceStaticResourceManager, Smoke)
{
ResourceTest t;
t.update(R"CONFIG(
<clickhouse>
<resources>
<res1>
<node path="/"> <type>inflight_limit</type><max_requests>10</max_requests></node>
<node path="/prio"> <type>priority</type></node>
<node path="/prio/A"></node>
<node path="/prio/B"><priority>1</priority></node>
</res1>
</resources>
<classifiers>
<A><res1>/prio/A</res1></A>
<B><res1>/prio/B</res1></B>
</classifiers>
</clickhouse>
)CONFIG");
ClassifierPtr ca = t.manager->acquire("A");
ClassifierPtr cb = t.manager->acquire("B");
for (int i = 0; i < 10; i++)
{
ResourceGuard ga(ca->get("res1"));
ResourceGuard gb(cb->get("res1"));
}
}
TEST(IOResourceStaticResourceManager, Prioritization)
{
constexpr size_t threads_per_queue = 2;
int requests_per_thead = 100;
ResourceTest t(4 * threads_per_queue + 1);
t.update(R"CONFIG(
<clickhouse>
<resources>
<res1>
<node path="/"> <type>inflight_limit</type><max_requests>1</max_requests></node>
<node path="/prio"> <type>priority</type></node>
<node path="/prio/A"> <priority>-1</priority></node>
<node path="/prio/B"> <priority>1</priority></node>
<node path="/prio/C"> </node>
<node path="/prio/D"> </node>
<node path="/prio/leader"></node>
</res1>
</resources>
<classifiers>
<A><res1>/prio/A</res1></A>
<B><res1>/prio/B</res1></B>
<C><res1>/prio/C</res1></C>
<D><res1>/prio/D</res1></D>
<leader><res1>/prio/leader</res1></leader>
</classifiers>
</clickhouse>
)CONFIG");
std::optional<Int64> last_priority;
auto check = [&] (Int64 priority)
{
// Lock is not required here because this is called during request execution and we have max_requests = 1
if (last_priority)
EXPECT_TRUE(priority <= *last_priority); // Should be true if every queue arrived at the same time at busy period start
last_priority = priority;
};
for (String name : {"A", "B", "C", "D"})
{
for (int thr = 0; thr < threads_per_queue; thr++)
{
t.threads.emplace_back([&, name]
{
ClassifierPtr c = t.manager->acquire(name);
ResourceLink link = c->get("res1");
t.startBusyPeriod(link, 1, requests_per_thead);
for (int req = 0; req < requests_per_thead; req++)
{
TestGuard g(t, link, 1);
check(link.queue->info.priority);
}
});
}
}
ClassifierPtr c = t.manager->acquire("leader");
ResourceLink link = c->get("res1");
t.blockResource(link);
}

View File

@ -0,0 +1,113 @@
#include <gtest/gtest.h>
#include <IO/SchedulerRoot.h>
#include <IO/Resource/tests/ResourceTest.h>
#include <future>
using namespace DB;
struct ResourceTest : public ResourceTestBase
{
SchedulerRoot scheduler;
ResourceTest()
{
scheduler.start();
}
~ResourceTest()
{
scheduler.stop(true);
}
};
struct ResourceHolder
{
ResourceTest & t;
SchedulerNodePtr root_node;
explicit ResourceHolder(ResourceTest & t_)
: t(t_)
{}
~ResourceHolder()
{
unregisterResource();
}
template <class TClass>
TClass * add(const String & path, const String & xml = {})
{
return ResourceTest::add<TClass>(t.scheduler.event_queue, root_node, path, xml);
}
ResourceLink addQueue(const String & path, const String & xml = {})
{
return {.queue = static_cast<ISchedulerQueue *>(ResourceTest::add<FifoQueue>(t.scheduler.event_queue, root_node, path, xml))};
}
void registerResource()
{
std::promise<void> p;
auto f = p.get_future();
t.scheduler.event_queue->enqueue([this, &p]
{
t.scheduler.attachChild(root_node);
p.set_value();
});
f.get();
}
void unregisterResource()
{
std::promise<void> p;
auto f = p.get_future();
t.scheduler.event_queue->enqueue([this, &p]
{
t.scheduler.removeChild(root_node.get());
p.set_value();
});
f.get();
}
};
TEST(IOSchedulerRoot, Smoke)
{
ResourceTest t;
ResourceHolder r1(t);
auto * fc1 = r1.add<ConstraintTest>("/", "<max_requests>1</max_requests>");
r1.add<PriorityPolicy>("/prio");
auto a = r1.addQueue("/prio/A", "<priority>1</priority>");
auto b = r1.addQueue("/prio/B", "<priority>2</priority>");
r1.registerResource();
ResourceHolder r2(t);
auto * fc2 = r2.add<ConstraintTest>("/", "<max_requests>1</max_requests>");
r2.add<PriorityPolicy>("/prio");
auto c = r2.addQueue("/prio/C", "<priority>-1</priority>");
auto d = r2.addQueue("/prio/D", "<priority>-2</priority>");
r2.registerResource();
{
ResourceGuard rg(a);
EXPECT_TRUE(fc1->requests.contains(&rg.request));
}
{
ResourceGuard rg(b);
EXPECT_TRUE(fc1->requests.contains(&rg.request));
}
{
ResourceGuard rg(c);
EXPECT_TRUE(fc2->requests.contains(&rg.request));
}
{
ResourceGuard rg(d);
EXPECT_TRUE(fc2->requests.contains(&rg.request));
}
}

93
src/IO/ResourceGuard.h Normal file
View File

@ -0,0 +1,93 @@
#pragma once
#include <base/types.h>
#include <IO/ResourceRequest.h>
#include <IO/ISchedulerQueue.h>
#include <IO/ISchedulerConstraint.h>
#include <future>
namespace DB
{
/*
* Scoped resource guard.
* Waits for resource to be available in constructor and releases resource in destructor
*/
class ResourceGuard
{
public:
enum ResourceGuardCtor
{
LockStraightAway, /// Lock inside constructor (default)
PostponeLocking /// Don't lock in constructor, but during later `lock()` call
};
struct Request : public ResourceRequest
{
/// Promise to be set on request execution
std::promise<void> dequeued;
explicit Request(ResourceCost cost_ = 1)
: ResourceRequest(cost_)
{}
void execute() override
{
// This function is executed inside scheduler thread and wakes thread issued this `request` (using ResourceGuard)
// That thread will continue execution and do real consumption of requested resource synchronously.
dequeued.set_value();
}
};
/// Creates pending request for resource; blocks while resource is not available (unless `PostponeLocking`)
explicit ResourceGuard(ResourceLink link_, ResourceCost cost = 1, ResourceGuardCtor ctor = LockStraightAway)
: link(link_)
, request(cost)
{
if (link.queue)
{
dequeued_future = request.dequeued.get_future();
link.queue->enqueueRequest(&request);
if (ctor == LockStraightAway)
lock();
}
}
~ResourceGuard()
{
unlock();
}
/// Blocks until resource is available
void lock()
{
if (link.queue)
dequeued_future.get();
}
/// Report request execution has finished
void unlock()
{
if (link.queue)
{
assert(!dequeued_future.valid()); // unlock must be called only after lock()
if (request.constraint)
request.constraint->finishRequest(&request);
}
}
/// Mark request as unsuccessful; by default request is considered to be successful
void setFailure()
{
request.successful = false;
}
public:
ResourceLink link;
Request request;
std::future<void> dequeued_future;
};
}

View File

@ -0,0 +1,55 @@
#pragma once
#include <Common/ErrorCodes.h>
#include <Common/Exception.h>
#include <IO/IResourceManager.h>
#include <boost/noncopyable.hpp>
#include <memory>
#include <mutex>
#include <unordered_map>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_SCHEDULER_NODE;
}
class ResourceManagerFactory : private boost::noncopyable
{
public:
static ResourceManagerFactory & instance()
{
static ResourceManagerFactory ret;
return ret;
}
ResourceManagerPtr get(const String & name)
{
std::lock_guard lock{mutex};
if (auto iter = methods.find(name); iter != methods.end())
return iter->second();
throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unknown scheduler node type: {}", name);
}
template <class TDerived>
void registerMethod(const String & name)
{
std::lock_guard lock{mutex};
methods[name] = [] ()
{
return std::make_shared<TDerived>();
};
}
private:
std::mutex mutex;
using Method = std::function<ResourceManagerPtr()>;
std::unordered_map<String, Method> methods;
};
}

92
src/IO/ResourceRequest.h Normal file
View File

@ -0,0 +1,92 @@
#pragma once
#include <base/types.h>
#include <limits>
namespace DB
{
// Forward declarations
class ISchedulerQueue;
class ISchedulerNode;
class ISchedulerConstraint;
/// Cost in terms of used resource (e.g. bytes for network IO)
using ResourceCost = Int64;
constexpr ResourceCost ResourceCostMax = std::numeric_limits<int>::max();
/// Internal identifier of a resource (for arrays; unique per scheduler)
using ResourceIdx = size_t;
constexpr ResourceIdx ResourceIdxNotSet = ResourceIdx(-1);
/// Timestamps (nanoseconds since epoch)
using ResourceNs = UInt64;
/*
* Info required for resource consumption.
*/
struct ResourceLink
{
ISchedulerQueue * queue = nullptr;
bool operator==(const ResourceLink &) const = default;
};
/*
* Request for a resource consumption. The main moving part of the scheduling subsystem.
* Resource requests processing workflow:
*
* ----1=2222222222222=3=4=555555555555555=6-----> time
* ^ ^ ^ ^ ^ ^
* | | | | | |
* enqueue wait dequeue execute consume finish
*
* 1) Request is enqueued using ISchedulerQueue::enqueueRequest().
* 2) Request competes with others for access to a resource; effectively just waiting in a queue.
* 3) Scheduler calls ISchedulerNode::dequeueRequest() that returns the request.
* 4) Callback ResourceRequest::execute() is called to provide access to the resource.
* 5) The resource consumption is happening outside of the scheduling subsystem.
* 6) request->constraint->finishRequest() is called when consumption is finished.
*
* Steps (5) and (6) can be omitted if constraint is not used by the resource.
*
* Request can be created on stack or heap.
* Request ownership is done outside of the scheduling subsystem.
* After (6) request can be destructed safely.
*
* Request cancelling is not supported yet.
*/
class ResourceRequest
{
public:
/// Cost of request execution; should be filled before request enqueueing.
/// NOTE: If cost is not known in advance, credit model can be used:
/// NOTE: for the first request use 1 and
ResourceCost cost;
/// Request outcome
/// Should be filled during resource consumption
bool successful = true;
/// Scheduler node to be notified on consumption finish
/// Auto-filled during request enqueue/dequeue
ISchedulerConstraint * constraint = nullptr;
/// Timestamps for introspection
ResourceNs enqueue_ns = 0;
ResourceNs execute_ns = 0;
ResourceNs finish_ns = 0;
explicit ResourceRequest(ResourceCost cost_ = 1)
: cost(cost_)
{}
virtual ~ResourceRequest() = default;
/// Callback to trigger resource consumption.
/// IMPORTANT: is called from scheduler thread and must be fast,
/// just triggering start of a consumption, not doing the consumption itself
/// (e.g. setting an std::promise or creating a job in a thread pool)
virtual void execute() = 0;
};
}

View File

@ -0,0 +1,57 @@
#pragma once
#include <Common/ErrorCodes.h>
#include <Common/Exception.h>
#include <IO/ISchedulerNode.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <boost/noncopyable.hpp>
#include <memory>
#include <mutex>
#include <unordered_map>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_SCHEDULER_NODE;
}
class SchedulerNodeFactory : private boost::noncopyable
{
public:
static SchedulerNodeFactory & instance()
{
static SchedulerNodeFactory ret;
return ret;
}
SchedulerNodePtr get(const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
{
std::lock_guard lock{mutex};
if (auto iter = methods.find(name); iter != methods.end())
return iter->second(event_queue, config, config_prefix);
throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unknown scheduler node type: {}", name);
}
template <class TDerived>
void registerMethod(const String & name)
{
std::lock_guard lock{mutex};
methods[name] = [] (EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
{
return std::make_shared<TDerived>(event_queue, config, config_prefix);
};
}
private:
std::mutex mutex;
using Method = std::function<SchedulerNodePtr(EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const String & config_prefix)>;
std::unordered_map<String, Method> methods;
};
}

250
src/IO/SchedulerRoot.h Normal file
View File

@ -0,0 +1,250 @@
#pragma once
#include <base/defines.h>
#include <Common/Stopwatch.h>
#include <Common/ThreadPool.h>
#include <IO/ISchedulerNode.h>
#include <IO/ISchedulerConstraint.h>
#include <Poco/Util/XMLConfiguration.h>
#include <unordered_map>
#include <map>
#include <memory>
#include <atomic>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_SCHEDULER_NODE;
}
/*
* Resource scheduler root node with a dedicated thread.
* Immediate children correspond to different resources.
*/
class SchedulerRoot : public ISchedulerNode
{
private:
struct TResource
{
SchedulerNodePtr root;
// Intrusive cyclic list of active resources
TResource * next = nullptr;
TResource * prev = nullptr;
explicit TResource(const SchedulerNodePtr & root_)
: root(root_)
{
root->info.parent.ptr = this;
}
// Get pointer stored by ctor in info
static TResource * get(SchedulerNodeInfo & info)
{
return reinterpret_cast<TResource *>(info.parent.ptr);
}
};
public:
SchedulerRoot()
: ISchedulerNode(&events)
{}
~SchedulerRoot() override
{
stop();
}
/// Runs separate scheduler thread
void start()
{
if (!scheduler.joinable())
scheduler = ThreadFromGlobalPool([this] { schedulerThread(); });
}
/// Joins scheduler threads and execute every pending request iff graceful
void stop(bool graceful = true)
{
if (scheduler.joinable())
{
stop_flag.store(true);
events.enqueue([]{}); // just to wake up thread
scheduler.join();
if (graceful)
{
// Do the same cycle as schedulerThread() but never block, just exit instead
bool has_work = true;
while (has_work)
{
auto [request, _] = dequeueRequest();
if (request)
execute(request);
else
has_work = false;
while (events.tryProcess())
has_work = true;
}
}
}
}
bool equals(ISchedulerNode * other) override
{
if (auto * o = dynamic_cast<SchedulerRoot *>(other))
return true;
return false;
}
void attachChild(const SchedulerNodePtr & child) override
{
// Take ownership
assert(child->parent == nullptr);
if (auto [it, inserted] = children.emplace(child.get(), child); !inserted)
throw Exception(
ErrorCodes::INVALID_SCHEDULER_NODE,
"Can't add the same scheduler node twice");
// Attach
child->setParent(this);
// Activate child if required
if (child->isActive())
activateChild(child.get());
}
void removeChild(ISchedulerNode * child) override
{
if (auto iter = children.find(child); iter != children.end())
{
SchedulerNodePtr removed = iter->second.root;
// Deactivate if required
deactivate(&iter->second);
// Detach
removed->setParent(nullptr);
// Remove ownership
children.erase(iter);
}
}
ISchedulerNode * getChild(const String &) override
{
abort(); // scheduler is allowed to have multiple children with the same name
}
std::pair<ResourceRequest *, bool> dequeueRequest() override
{
if (current == nullptr) // No active resources
return {nullptr, false};
// Dequeue request from current resource
auto [request, resource_active] = current->root->dequeueRequest();
assert(request != nullptr);
// Deactivate resource if required
if (!resource_active)
deactivate(current);
else
current = current->next; // Just move round-robin pointer
return {request, current != nullptr};
}
bool isActive() override
{
return current != nullptr;
}
void activateChild(ISchedulerNode * child) override
{
activate(TResource::get(child->info));
}
void setParent(ISchedulerNode *) override
{
abort(); // scheduler must be the root and this function should not be called
}
private:
void activate(TResource * value)
{
assert(value->next == nullptr && value->prev == nullptr);
if (current == nullptr) // No active children
{
current = value;
value->prev = value;
value->next = value;
}
else
{
current->prev->next = value;
value->prev = current->prev;
current->prev = value;
value->next = current;
}
}
void deactivate(TResource * value)
{
if (value->next == nullptr)
return; // Already deactivated
assert(current != nullptr);
if (current == value)
{
if (current->next == current) // We are going to remove the last active child
{
value->next = nullptr;
value->prev = nullptr;
current = nullptr;
return;
}
else // Just move current to next to avoid invalidation
current = current->next;
}
value->prev->next = value->next;
value->next->prev = value->prev;
value->prev = nullptr;
value->next = nullptr;
}
private:
void schedulerThread()
{
while (!stop_flag.load())
{
// Dequeue and execute single request
auto [request, _] = dequeueRequest();
if (request)
execute(request);
else // No more requests -- block until any event happens
events.process();
// Process all events before dequeuing to ensure fair competition
while (events.tryProcess()) {}
}
}
void execute(ResourceRequest * request)
{
request->execute_ns = clock_gettime_ns();
request->execute();
}
private:
TResource * current = nullptr; // round-robin pointer
std::unordered_map<ISchedulerNode *, TResource> children; // resources by pointer
std::atomic<bool> stop_flag = false;
EventQueue events;
ThreadFromGlobalPool scheduler;
};
}

View File

@ -72,7 +72,7 @@ WriteBufferFromS3::WriteBufferFromS3(
std::shared_ptr<const Aws::S3::S3Client> client_ptr_,
const String & bucket_,
const String & key_,
const S3Settings::RequestSettings & request_settings_,
const S3Settings::RequestSettings & request_settings,
std::optional<std::map<String, String>> object_metadata_,
size_t buffer_size_,
ThreadPoolCallbackRunner<void> schedule_,
@ -80,10 +80,12 @@ WriteBufferFromS3::WriteBufferFromS3(
: BufferWithOwnMemory<WriteBuffer>(buffer_size_, nullptr, 0)
, bucket(bucket_)
, key(key_)
, request_settings(request_settings_)
, settings(request_settings.getUploadSettings())
, check_objects_after_upload(request_settings.check_objects_after_upload)
, max_unexpected_write_error_retries(request_settings.max_unexpected_write_error_retries)
, client_ptr(std::move(client_ptr_))
, object_metadata(std::move(object_metadata_))
, upload_part_size(request_settings_.min_upload_part_size)
, upload_part_size(settings.min_upload_part_size)
, schedule(std::move(schedule_))
, write_settings(write_settings_)
{
@ -108,9 +110,10 @@ void WriteBufferFromS3::nextImpl()
write_settings.remote_throttler->add(offset());
/// Data size exceeds singlepart upload threshold, need to use multipart upload.
if (multipart_upload_id.empty() && last_part_size > request_settings.max_single_part_upload_size)
if (multipart_upload_id.empty() && last_part_size > settings.max_single_part_upload_size)
createMultipartUpload();
chassert(upload_part_size > 0);
if (!multipart_upload_id.empty() && last_part_size > upload_part_size)
{
writePart();
@ -175,7 +178,7 @@ void WriteBufferFromS3::finalizeImpl()
if (!multipart_upload_id.empty())
completeMultipartUpload();
if (request_settings.check_objects_after_upload)
if (check_objects_after_upload)
{
LOG_TRACE(log, "Checking object {} exists after upload", key);
@ -300,15 +303,15 @@ void WriteBufferFromS3::fillUploadRequest(Aws::S3::Model::UploadPartRequest & re
{
/// Increase part number.
++part_number;
if (!multipart_upload_id.empty() && (part_number > request_settings.max_part_number))
if (!multipart_upload_id.empty() && (part_number > settings.max_part_number))
{
throw Exception(
ErrorCodes::INVALID_CONFIG_PARAMETER,
"Part number exceeded {} while writing {} bytes to S3. Check min_upload_part_size = {}, max_upload_part_size = {}, "
"upload_part_size_multiply_factor = {}, upload_part_size_multiply_parts_count_threshold = {}, max_single_part_upload_size = {}",
request_settings.max_part_number, count(), request_settings.min_upload_part_size, request_settings.max_upload_part_size,
request_settings.upload_part_size_multiply_factor, request_settings.upload_part_size_multiply_parts_count_threshold,
request_settings.max_single_part_upload_size);
settings.max_part_number, count(), settings.min_upload_part_size, settings.max_upload_part_size,
settings.upload_part_size_multiply_factor, settings.upload_part_size_multiply_parts_count_threshold,
settings.max_single_part_upload_size);
}
/// Setup request.
@ -323,10 +326,10 @@ void WriteBufferFromS3::fillUploadRequest(Aws::S3::Model::UploadPartRequest & re
req.SetContentType("binary/octet-stream");
/// Maybe increase `upload_part_size` (we need to increase it sometimes to keep `part_number` less or equal than `max_part_number`).
if (!multipart_upload_id.empty() && (part_number % request_settings.upload_part_size_multiply_parts_count_threshold == 0))
if (!multipart_upload_id.empty() && (part_number % settings.upload_part_size_multiply_parts_count_threshold == 0))
{
upload_part_size *= request_settings.upload_part_size_multiply_factor;
upload_part_size = std::min(upload_part_size, request_settings.max_upload_part_size);
upload_part_size *= settings.upload_part_size_multiply_factor;
upload_part_size = std::min(upload_part_size, settings.max_upload_part_size);
}
}
@ -371,7 +374,7 @@ void WriteBufferFromS3::completeMultipartUpload()
req.SetMultipartUpload(multipart_upload);
size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries, 1UL);
size_t max_retry = std::max(max_unexpected_write_error_retries, 1UL);
for (size_t i = 0; i < max_retry; ++i)
{
ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload);
@ -477,7 +480,7 @@ void WriteBufferFromS3::fillPutRequest(Aws::S3::Model::PutObjectRequest & req)
void WriteBufferFromS3::processPutRequest(const PutObjectTask & task)
{
size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries, 1UL);
size_t max_retry = std::max(max_unexpected_write_error_retries, 1UL);
for (size_t i = 0; i < max_retry; ++i)
{
ProfileEvents::increment(ProfileEvents::S3PutObject);

View File

@ -50,7 +50,7 @@ public:
std::shared_ptr<const Aws::S3::S3Client> client_ptr_,
const String & bucket_,
const String & key_,
const S3Settings::RequestSettings & request_settings_,
const S3Settings::RequestSettings & request_settings,
std::optional<std::map<String, String>> object_metadata_ = std::nullopt,
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
ThreadPoolCallbackRunner<void> schedule_ = {},
@ -88,7 +88,9 @@ private:
const String bucket;
const String key;
const S3Settings::RequestSettings request_settings;
const S3Settings::RequestSettings::PartUploadSettings settings;
const bool check_objects_after_upload = false;
const size_t max_unexpected_write_error_retries = 4;
const std::shared_ptr<const Aws::S3::S3Client> client_ptr;
const std::optional<std::map<String, String>> object_metadata;

View File

@ -1537,12 +1537,39 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBeforeArrayJoin(const NameSet &
return res;
}
ActionsDAG::NodeRawConstPtrs ActionsDAG::getParents(const Node * target) const
{
NodeRawConstPtrs parents;
for (const auto & node : getNodes())
{
for (const auto & child : node.children)
{
if (child == target)
{
parents.push_back(&node);
break;
}
}
}
return parents;
}
ActionsDAG::SplitResult ActionsDAG::splitActionsBySortingDescription(const NameSet & sort_columns) const
{
std::unordered_set<const Node *> split_nodes;
for (const auto & sort_column : sort_columns)
if (const auto * node = tryFindInOutputs(sort_column))
{
split_nodes.insert(node);
/// Sorting can materialize const columns, so if we have const expression used in sorting,
/// we should also add all it's parents, otherwise, we can break the header
/// (function can expect const column, but will get materialized).
if (node->column && isColumnConst(*node->column))
{
auto parents = getParents(node);
split_nodes.insert(parents.begin(), parents.end());
}
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Sorting column {} wasn't found in the ActionsDAG's outputs. DAG:\n{}",

View File

@ -343,6 +343,8 @@ public:
const ContextPtr & context);
private:
NodeRawConstPtrs getParents(const Node * target) const;
Node & addNode(Node node);
#if USE_EMBEDDED_COMPILER

View File

@ -54,6 +54,7 @@
#include <Access/SettingsConstraintsAndProfileIDs.h>
#include <Access/ExternalAuthenticators.h>
#include <Access/GSSAcceptor.h>
#include <IO/ResourceManagerFactory.h>
#include <Backups/BackupsWorker.h>
#include <Dictionaries/Embedded/GeoDictionariesLoader.h>
#include <Interpreters/EmbeddedDictionaries.h>
@ -221,6 +222,7 @@ struct ContextSharedPart : boost::noncopyable
String system_profile_name; /// Profile used by system processes
String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying
std::unique_ptr<AccessControl> access_control;
mutable ResourceManagerPtr resource_manager;
mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks.
mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files.
mutable std::unique_ptr<ThreadPool> load_marks_threadpool; /// Threadpool for loading marks cache.
@ -1065,6 +1067,21 @@ std::vector<UUID> Context::getEnabledProfiles() const
}
ResourceManagerPtr Context::getResourceManager() const
{
auto lock = getLock();
if (!shared->resource_manager)
shared->resource_manager = ResourceManagerFactory::instance().get(getConfigRef().getString("resource_manager", "static"));
return shared->resource_manager;
}
ClassifierPtr Context::getClassifier() const
{
auto lock = getLock();
return getResourceManager()->acquire(getSettingsRef().workload);
}
const Scalars & Context::getScalars() const
{
return scalars;

View File

@ -19,6 +19,7 @@
#include <base/types.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
#include <Storages/ColumnsDescription.h>
#include <IO/IResourceManager.h>
#include <Server/HTTP/HTTPContext.h>
@ -540,6 +541,10 @@ public:
std::shared_ptr<const EnabledQuota> getQuota() const;
std::optional<QuotaUsage> getQuotaUsage() const;
/// Resource management related
ResourceManagerPtr getResourceManager() const;
ClassifierPtr getClassifier() const;
/// We have to copy external tables inside executeQuery() to track limits. Therefore, set callback for it. Must set once.
void setExternalTablesInitializer(ExternalTablesInitializer && initializer);
/// This method is called in executeQuery() and will call the external tables initializer.

View File

@ -15,6 +15,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <Columns/IColumn.h>
#include <Interpreters/Aggregator.h>
#include <Interpreters/ArrayJoinAction.h>
#include <Interpreters/Context.h>
#include <Interpreters/ConcurrentHashJoin.h>
@ -33,6 +34,7 @@
#include <Interpreters/replaceForPositionalArguments.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/AggregatingStep.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
@ -1831,7 +1833,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
ssize_t where_step_num = -1;
ssize_t having_step_num = -1;
auto finalize_chain = [&](ExpressionActionsChain & chain)
auto finalize_chain = [&](ExpressionActionsChain & chain) -> ColumnsWithTypeAndName
{
if (prewhere_step_num >= 0)
{
@ -1852,7 +1854,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
finalize(chain, prewhere_step_num, where_step_num, having_step_num, query);
auto res = chain.getLastStep().getResultColumns();
chain.clear();
return res;
};
{
@ -1970,7 +1974,55 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
if (settings.group_by_use_nulls)
query_analyzer.appendGroupByModifiers(before_aggregation, chain, only_types);
finalize_chain(chain);
auto columns_before_aggregation = finalize_chain(chain);
/// Here we want to check that columns after aggregation have the same type as
/// were promised in query_analyzer.aggregated_columns
/// Ideally, they should be equal. In practice, this may be not true.
/// As an example, we don't build sets for IN inside ExpressionAnalysis::analyzeAggregation,
/// so that constant folding for expression (1 in 1) will not work. This may change the return type
/// for functions with LowCardinality argument: function "substr(toLowCardinality('abc'), 1 IN 1)"
/// should usually return LowCardinality(String) when (1 IN 1) is constant, but without built set
/// for (1 IN 1) constant is not propagated and "substr" returns String type.
/// See 02503_in_lc_const_args_bug.sql
///
/// As a temporary solution, we add converting actions to the next chain.
/// Hopefully, later we can
/// * use a new analyzer where this issue is absent
/// * or remove ExpressionActionsChain completely and re-implement its logic on top of the query plan
{
for (auto & col : columns_before_aggregation)
if (!col.column)
col.column = col.type->createColumn();
Block header_before_aggregation(std::move(columns_before_aggregation));
auto keys = query_analyzer.aggregationKeys().getNames();
const auto & aggregates = query_analyzer.aggregates();
bool has_grouping = query_analyzer.group_by_kind != GroupByKind::ORDINARY;
auto actual_header = Aggregator::Params::getHeader(
header_before_aggregation, /*only_merge*/ false, keys, aggregates, /*final*/ true);
actual_header = AggregatingStep::appendGroupingColumn(
std::move(actual_header), keys, has_grouping, settings.group_by_use_nulls);
Block expected_header;
for (const auto & expected : query_analyzer.aggregated_columns)
expected_header.insert(ColumnWithTypeAndName(expected.type, expected.name));
if (!blocksHaveEqualStructure(actual_header, expected_header))
{
auto converting = ActionsDAG::makeConvertingActions(
actual_header.getColumnsWithTypeAndName(),
expected_header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name,
true);
auto & step = chain.lastStep(query_analyzer.aggregated_columns);
auto & actions = step.actions();
actions = ActionsDAG::merge(std::move(*actions), std::move(*converting));
}
}
if (query_analyzer.appendHaving(chain, only_types || !second_stage))
{

View File

@ -236,22 +236,32 @@ private:
Poco::Logger * log;
};
static void flushBlocksToBuckets(Blocks & blocks, const GraceHashJoin::Buckets & buckets_snapshot)
namespace
{
assert(blocks.size() == buckets_snapshot.size());
template <JoinTableSide table_side>
void flushBlocksToBuckets(Blocks & blocks, const GraceHashJoin::Buckets & buckets)
{
chassert(blocks.size() == buckets.size());
retryForEach(
generateRandomPermutation(1, buckets_snapshot.size()),
generateRandomPermutation(1, buckets.size()), // skipping 0 block, since we join it in memory w/o spilling on disk
[&](size_t i)
{
if (!blocks[i].rows())
return true;
bool flushed = buckets_snapshot[i]->tryAddRightBlock(blocks[i]);
bool flushed = false;
if constexpr (table_side == JoinTableSide::Left)
flushed = buckets[i]->tryAddLeftBlock(blocks[i]);
if constexpr (table_side == JoinTableSide::Right)
flushed = buckets[i]->tryAddRightBlock(blocks[i]);
if (flushed)
blocks[i].clear();
return flushed;
});
}
}
GraceHashJoin::GraceHashJoin(
ContextPtr context_, std::shared_ptr<TableJoin> table_join_,
@ -274,7 +284,6 @@ GraceHashJoin::GraceHashJoin(
{
if (!GraceHashJoin::isSupported(table_join))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "GraceHashJoin is not supported for this join type");
}
void GraceHashJoin::initBuckets()
@ -382,8 +391,9 @@ void GraceHashJoin::joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_p
materializeBlockInplace(block);
Buckets buckets_snapshot = getCurrentBuckets();
size_t num_buckets = buckets_snapshot.size();
/// number of buckets doesn't change after right table is split to buckets, i.e. read-only access to buckets
/// so, no need to copy buckets here
size_t num_buckets = getNumBuckets();
Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets);
block = std::move(blocks[current_bucket->idx]);
@ -392,15 +402,7 @@ void GraceHashJoin::joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_p
if (not_processed)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unhandled not processed block in GraceHashJoin");
// We need to skip the first bucket that is already joined in memory, so we start with 1.
retryForEach(
generateRandomPermutation(1, num_buckets),
[&blocks, &buckets_snapshot](size_t idx)
{
if (blocks[idx].rows() == 0)
return true;
return buckets_snapshot[idx]->tryAddLeftBlock(blocks[idx]);
});
flushBlocksToBuckets<JoinTableSide::Left>(blocks, buckets);
}
void GraceHashJoin::setTotals(const Block & block)
@ -428,9 +430,11 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const
if (!isInnerOrRight(table_join->kind()))
return false;
std::shared_lock lock(rehash_mutex);
bool file_buckets_are_empty = std::all_of(buckets.begin(), buckets.end(), [](const auto & bucket) { return bucket->empty(); });
bool file_buckets_are_empty = [this]()
{
std::shared_lock lock(rehash_mutex);
return std::all_of(buckets.begin(), buckets.end(), [](const auto & bucket) { return bucket->empty(); });
}();
bool hash_join_is_empty = hash_join && hash_join->alwaysReturnsEmptySet();
return hash_join_is_empty && file_buckets_are_empty;
@ -610,7 +614,7 @@ void GraceHashJoin::addJoinedBlockImpl(Block block)
blocks[bucket_index].clear();
}
flushBlocksToBuckets(blocks, buckets_snapshot);
flushBlocksToBuckets<JoinTableSide::Right>(blocks, buckets_snapshot);
}
size_t GraceHashJoin::getNumBuckets() const

View File

@ -17,7 +17,7 @@ namespace ErrorCodes
namespace DB
{
static bool parseQueryWithOnClusterAndMaybeTable(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos,
[[nodiscard]] static bool parseQueryWithOnClusterAndMaybeTable(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos,
Expected & expected, bool require_table, bool allow_string_literal)
{
/// Better form for user: SYSTEM <ACTION> table ON CLUSTER cluster
@ -71,7 +71,7 @@ enum class SystemQueryTargetType
Disk
};
static bool parseQueryWithOnClusterAndTarget(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos, Expected & expected, SystemQueryTargetType target_type)
[[nodiscard]] static bool parseQueryWithOnClusterAndTarget(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos, Expected & expected, SystemQueryTargetType target_type)
{
/// Better form for user: SYSTEM <ACTION> target_name ON CLUSTER cluster
/// Query rewritten form + form while executing on cluster: SYSTEM <ACTION> ON CLUSTER cluster target_name
@ -136,7 +136,7 @@ static bool parseQueryWithOnClusterAndTarget(std::shared_ptr<ASTSystemQuery> & r
return true;
}
static bool parseQueryWithOnCluster(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos,
[[nodiscard]] static bool parseQueryWithOnCluster(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos,
Expected & expected)
{
String cluster_str;
@ -196,7 +196,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
}
case Type::DROP_REPLICA:
{
parseQueryWithOnCluster(res, pos, expected);
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
@ -239,7 +240,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
case Type::RESTART_REPLICA:
case Type::SYNC_REPLICA:
{
parseQueryWithOnCluster(res, pos, expected);
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
if (!parseDatabaseAndTableAsAST(pos, expected, res->database, res->table))
return false;
break;
@ -247,7 +249,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
case Type::SYNC_DATABASE_REPLICA:
{
parseQueryWithOnCluster(res, pos, expected);
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
if (!parseDatabaseAsAST(pos, expected, res->database))
return false;
break;
@ -310,7 +313,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
}
else
{
parseQueryWithOnCluster(res, pos, expected);
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
if (ParserKeyword{"ON VOLUME"}.ignore(pos, expected))
{
if (!parse_on_volume())
@ -335,13 +339,15 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
case Type::START_REPLICATED_SENDS:
case Type::STOP_REPLICATION_QUEUES:
case Type::START_REPLICATION_QUEUES:
parseQueryWithOnCluster(res, pos, expected);
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
parseDatabaseAndTableAsAST(pos, expected, res->database, res->table);
break;
case Type::SUSPEND:
{
parseQueryWithOnCluster(res, pos, expected);
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
ASTPtr seconds;
if (!(ParserKeyword{"FOR"}.ignore(pos, expected)
@ -360,7 +366,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
ASTPtr ast;
if (path_parser.parse(pos, ast, expected))
res->filesystem_cache_path = ast->as<ASTLiteral>()->value.safeGet<String>();
parseQueryWithOnCluster(res, pos, expected);
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
break;
}
case Type::DROP_SCHEMA_CACHE:
@ -397,7 +404,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
default:
{
parseQueryWithOnCluster(res, pos, expected);
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
break;
}
}

View File

@ -1,6 +1,5 @@
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/ReadSchemaUtils.h>
#include <Formats/EscapingRuleUtils.h>
#include <Formats/SchemaInferenceUtils.h>
#include <DataTypes/DataTypeString.h>
#include <Interpreters/parseColumnsListForTableFunction.h>
#include <boost/algorithm/string.hpp>
@ -11,65 +10,29 @@ namespace DB
namespace ErrorCodes
{
extern const int ONLY_NULLS_WHILE_READING_SCHEMA;
extern const int TYPE_MISMATCH;
extern const int INCORRECT_DATA;
extern const int EMPTY_DATA_PASSED;
extern const int BAD_ARGUMENTS;
}
void chooseResultColumnType(
DataTypePtr & type,
DataTypePtr & new_type,
std::function<void(DataTypePtr &, DataTypePtr &)> transform_types_if_needed,
const DataTypePtr & default_type,
const String & column_name,
size_t row)
void checkFinalInferredType(DataTypePtr & type, const String & name, const FormatSettings & settings, const DataTypePtr & default_type, size_t rows_read)
{
if (!type)
{
type = new_type;
return;
}
if (!new_type || type->equals(*new_type))
return;
transform_types_if_needed(type, new_type);
if (type->equals(*new_type))
return;
/// If the new type and the previous type for this column are different,
/// we will use default type if we have it or throw an exception.
if (default_type)
type = default_type;
else
{
throw Exception(
ErrorCodes::TYPE_MISMATCH,
"Automatically defined type {} for column '{}' in row {} differs from type defined by previous rows: {}. "
"You can specify the type for this column using setting schema_inference_hints",
type->getName(),
column_name,
row,
new_type->getName());
}
}
void checkResultColumnTypeAndAppend(NamesAndTypesList & result, DataTypePtr & type, const String & name, const DataTypePtr & default_type, size_t rows_read)
{
if (!type)
if (!checkIfTypeIsComplete(type))
{
if (!default_type)
throw Exception(
ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA,
"Cannot determine type for column '{}' by first {} rows of data, most likely this column contains only Nulls or empty "
"Arrays/Maps. You can specify the type for this column using setting schema_inference_hints",
"Arrays/Maps. You can specify the type for this column using setting schema_inference_hints. "
"If your data contains complex JSON objects, try enabling one of the settings allow_experimental_object_type/input_format_json_read_objects_as_strings",
name,
rows_read);
type = default_type;
}
result.emplace_back(name, type);
if (settings.schema_inference_make_columns_nullable)
type = makeNullableRecursively(type);
}
IIRowSchemaReader::IIRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, DataTypePtr default_type_)
@ -88,6 +51,11 @@ void IIRowSchemaReader::setContext(ContextPtr & context)
}
}
void IIRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
{
transformInferredTypesIfNeeded(type, new_type, format_settings);
}
IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: IIRowSchemaReader(in_, format_settings_), column_names(splitColumnNames(format_settings.column_names_for_schema_inference))
{
@ -160,23 +128,28 @@ NamesAndTypesList IRowSchemaReader::readSchema()
if (new_data_types.size() != data_types.size())
throw Exception(ErrorCodes::INCORRECT_DATA, "Rows have different amount of values");
for (size_t i = 0; i != data_types.size(); ++i)
for (field_index = 0; field_index != data_types.size(); ++field_index)
{
/// Check if we couldn't determine the type of this column in a new row
/// or the type for this column was taken from hints.
if (!new_data_types[i] || hints.contains(column_names[i]))
if (!new_data_types[field_index] || hints.contains(column_names[field_index]))
continue;
auto transform_types_if_needed = [&](DataTypePtr & type, DataTypePtr & new_type){ transformTypesIfNeeded(type, new_type, i); };
chooseResultColumnType(data_types[i], new_data_types[i], transform_types_if_needed, getDefaultType(i), std::to_string(i + 1), rows_read);
chooseResultColumnType(*this, data_types[field_index], new_data_types[field_index], getDefaultType(field_index), std::to_string(field_index + 1), rows_read);
}
}
NamesAndTypesList result;
for (size_t i = 0; i != data_types.size(); ++i)
for (field_index = 0; field_index != data_types.size(); ++field_index)
{
/// Check that we could determine the type of this column.
checkResultColumnTypeAndAppend(result, data_types[i], column_names[i], getDefaultType(i), rows_read);
/// Don't check/change types from hints.
if (!hints.contains(column_names[field_index]))
{
transformFinalTypeIfNeeded(data_types[field_index]);
/// Check that we could determine the type of this column.
checkFinalInferredType(data_types[field_index], column_names[field_index], format_settings, getDefaultType(field_index), rows_read);
}
result.emplace_back(column_names[field_index], data_types[field_index]);
}
return result;
@ -208,11 +181,6 @@ DataTypePtr IRowSchemaReader::getDefaultType(size_t column) const
return nullptr;
}
void IRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t)
{
transformInferredTypesIfNeeded(type, new_type, format_settings);
}
IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, DataTypePtr default_type_)
: IIRowSchemaReader(in_, format_settings_, default_type_)
{
@ -245,7 +213,6 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
names_order.push_back(name);
}
auto transform_types_if_needed = [&](DataTypePtr & type, DataTypePtr & new_type){ transformTypesIfNeeded(type, new_type); };
for (rows_read = 1; rows_read < max_rows_to_read; ++rows_read)
{
auto new_names_and_types = readRowAndGetNamesAndDataTypes(eof);
@ -277,7 +244,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
continue;
auto & type = it->second;
chooseResultColumnType(type, new_type, transform_types_if_needed, default_type, name, rows_read);
chooseResultColumnType(*this, type, new_type, default_type, name, rows_read);
}
}
@ -285,20 +252,21 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
if (names_to_types.empty())
throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "Cannot read rows from the data");
NamesAndTypesList result;
NamesAndTypesList result = getStaticNamesAndTypes();
for (auto & name : names_order)
{
auto & type = names_to_types[name];
/// Check that we could determine the type of this column.
checkResultColumnTypeAndAppend(result, type, name, default_type, rows_read);
/// Don't check/change types from hints.
if (!hints.contains(name))
{
transformFinalTypeIfNeeded(type);
/// Check that we could determine the type of this column.
checkFinalInferredType(type, name, format_settings, default_type, rows_read);
}
result.emplace_back(name, type);
}
return result;
}
void IRowWithNamesSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
{
transformInferredTypesIfNeeded(type, new_type, format_settings);
}
}

View File

@ -9,6 +9,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
/// Base class for schema inference for the data in some specific format.
/// It reads some data from read buffer and try to determine the schema
/// from read data.
@ -45,10 +50,14 @@ public:
bool needContext() const override { return !hints_str.empty(); }
void setContext(ContextPtr & context) override;
virtual void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type);
protected:
void setMaxRowsToRead(size_t max_rows) override { max_rows_to_read = max_rows; }
size_t getNumRowsRead() const override { return rows_read; }
virtual void transformFinalTypeIfNeeded(DataTypePtr &) {}
size_t max_rows_to_read;
size_t rows_read = 0;
DataTypePtr default_type;
@ -82,7 +91,7 @@ protected:
void setColumnNames(const std::vector<String> & names) { column_names = names; }
virtual void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t index);
size_t field_index;
private:
DataTypePtr getDefaultType(size_t column) const;
@ -111,7 +120,10 @@ protected:
/// Set eof = true if can't read more data.
virtual NamesAndTypesList readRowAndGetNamesAndDataTypes(bool & eof) = 0;
virtual void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type);
/// Get special static types that have the same name/type for each row.
/// For example, in JSONObjectEachRow format we have static column with
/// type String and name from a settings for object keys.
virtual NamesAndTypesList getStaticNamesAndTypes() { return {}; }
};
/// Base class for schema inference for formats that don't need any data to
@ -125,16 +137,46 @@ public:
virtual ~IExternalSchemaReader() = default;
};
template <class SchemaReader>
void chooseResultColumnType(
SchemaReader & schema_reader,
DataTypePtr & type,
DataTypePtr & new_type,
std::function<void(DataTypePtr &, DataTypePtr &)> transform_types_if_needed,
const DataTypePtr & default_type,
const String & column_name,
size_t row);
size_t row)
{
if (!type)
{
type = new_type;
return;
}
void checkResultColumnTypeAndAppend(
NamesAndTypesList & result, DataTypePtr & type, const String & name, const DataTypePtr & default_type, size_t rows_read);
if (!new_type || type->equals(*new_type))
return;
schema_reader.transformTypesIfNeeded(type, new_type);
if (type->equals(*new_type))
return;
/// If the new type and the previous type for this column are different,
/// we will use default type if we have it or throw an exception.
if (default_type)
type = default_type;
else
{
throw Exception(
ErrorCodes::TYPE_MISMATCH,
"Automatically defined type {} for column '{}' in row {} differs from type defined by previous rows: {}. "
"You can specify the type for this column using setting schema_inference_hints",
type->getName(),
column_name,
row,
new_type->getName());
}
}
void checkFinalInferredType(DataTypePtr & type, const String & name, const FormatSettings & settings, const DataTypePtr & default_type, size_t rows_read);
Strings splitColumnNames(const String & column_names_str);

View File

@ -3,7 +3,7 @@
#if USE_ARROW
#include <Formats/FormatFactory.h>
#include <Formats/ReadSchemaUtils.h>
#include <Formats/SchemaInferenceUtils.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>

View File

@ -772,27 +772,27 @@ DataTypePtr BSONEachRowSchemaReader::getDataTypeFromBSONField(BSONType type, boo
case BSONType::DOUBLE:
{
in.ignore(sizeof(Float64));
return makeNullable(std::make_shared<DataTypeFloat64>());
return std::make_shared<DataTypeFloat64>();
}
case BSONType::BOOL:
{
in.ignore(sizeof(UInt8));
return makeNullable(DataTypeFactory::instance().get("Bool"));
return DataTypeFactory::instance().get("Bool");
}
case BSONType::INT64:
{
in.ignore(sizeof(Int64));
return makeNullable(std::make_shared<DataTypeInt64>());
return std::make_shared<DataTypeInt64>();
}
case BSONType::DATETIME:
{
in.ignore(sizeof(Int64));
return makeNullable(std::make_shared<DataTypeDateTime64>(6, "UTC"));
return std::make_shared<DataTypeDateTime64>(6, "UTC");
}
case BSONType::INT32:
{
in.ignore(sizeof(Int32));
return makeNullable(std::make_shared<DataTypeInt32>());
return std::make_shared<DataTypeInt32>();
}
case BSONType::SYMBOL: [[fallthrough]];
case BSONType::JAVA_SCRIPT_CODE: [[fallthrough]];
@ -802,7 +802,7 @@ DataTypePtr BSONEachRowSchemaReader::getDataTypeFromBSONField(BSONType type, boo
BSONSizeT size;
readBinary(size, in);
in.ignore(size);
return makeNullable(std::make_shared<DataTypeString>());
return std::make_shared<DataTypeString>();
}
case BSONType::DOCUMENT:
{
@ -856,10 +856,10 @@ DataTypePtr BSONEachRowSchemaReader::getDataTypeFromBSONField(BSONType type, boo
{
case BSONBinarySubtype::BINARY_OLD: [[fallthrough]];
case BSONBinarySubtype::BINARY:
return makeNullable(std::make_shared<DataTypeString>());
return std::make_shared<DataTypeString>();
case BSONBinarySubtype::UUID_OLD: [[fallthrough]];
case BSONBinarySubtype::UUID:
return makeNullable(std::make_shared<DataTypeUUID>());
return std::make_shared<DataTypeUUID>();
default:
throw Exception(ErrorCodes::UNKNOWN_TYPE, "BSON binary subtype {} is not supported", getBSONBinarySubtypeName(subtype));
}

View File

@ -274,15 +274,15 @@ void CSVFormatReader::skipPrefixBeforeHeader()
}
CSVSchemaReader::CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_)
CSVSchemaReader::CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_settings_)
: FormatWithNamesAndTypesSchemaReader(
in_,
format_setting_,
format_settings_,
with_names_,
with_types_,
&reader,
getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::CSV))
, reader(in_, format_setting_)
, reader(in_, format_settings_)
{
}
@ -293,7 +293,7 @@ DataTypes CSVSchemaReader::readRowAndGetDataTypes()
return {};
auto fields = reader.readRow();
return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), FormatSettings::EscapingRule::CSV);
return tryInferDataTypesByEscapingRule(fields, reader.getFormatSettings(), FormatSettings::EscapingRule::CSV);
}

View File

@ -75,7 +75,7 @@ public:
class CSVSchemaReader : public FormatWithNamesAndTypesSchemaReader
{
public:
CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_);
CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_settings_);
private:
DataTypes readRowAndGetDataTypes() override;

View File

@ -1,6 +1,7 @@
#include <Processors/Formats/Impl/CustomSeparatedRowInputFormat.h>
#include <Processors/Formats/Impl/TemplateRowInputFormat.h>
#include <Formats/EscapingRuleUtils.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Formats/registerWithNamesAndTypes.h>
#include <IO/Operators.h>
@ -370,12 +371,12 @@ DataTypes CustomSeparatedSchemaReader::readRowAndGetDataTypes()
first_row = false;
auto fields = reader.readRow();
return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule());
return tryInferDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule(), &json_inference_info);
}
void CustomSeparatedSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t)
void CustomSeparatedSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
{
transformInferredTypesIfNeeded(type, new_type, format_settings, reader.getEscapingRule());
transformInferredTypesByEscapingRuleIfNeeded(type, new_type, format_settings, reader.getEscapingRule(), &json_inference_info);
}
void registerInputFormatCustomSeparated(FormatFactory & factory)

View File

@ -2,6 +2,7 @@
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <Formats/ParsedTemplateFormatString.h>
#include <Formats/SchemaInferenceUtils.h>
#include <IO/PeekableReadBuffer.h>
#include <IO/ReadHelpers.h>
@ -100,11 +101,12 @@ public:
private:
DataTypes readRowAndGetDataTypes() override;
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) override;
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
PeekableReadBuffer buf;
CustomSeparatedFormatReader reader;
bool first_row = true;
JSONInferenceInfo json_inference_info;
};
}

View File

@ -2,8 +2,11 @@
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/JSONUtils.h>
#include <Formats/EscapingRuleUtils.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Interpreters/parseColumnsListForTableFunction.h>
#include <IO/ReadHelpers.h>
#include <base/find_symbols.h>
#include <Common/logger_useful.h>
namespace DB
{
@ -170,19 +173,25 @@ JSONColumnsSchemaReaderBase::JSONColumnsSchemaReaderBase(
ReadBuffer & in_, const FormatSettings & format_settings_, std::unique_ptr<JSONColumnsReaderBase> reader_)
: ISchemaReader(in_)
, format_settings(format_settings_)
, hints_str(format_settings_.schema_inference_hints)
, reader(std::move(reader_))
, column_names_from_settings(splitColumnNames(format_settings_.column_names_for_schema_inference))
{
}
void JSONColumnsSchemaReaderBase::chooseResulType(DataTypePtr & type, DataTypePtr & new_type, const String & column_name, size_t row) const
void JSONColumnsSchemaReaderBase::setContext(ContextPtr & ctx)
{
auto convert_types_if_needed = [&](DataTypePtr & first, DataTypePtr & second)
ColumnsDescription columns;
if (tryParseColumnsListFromString(hints_str, columns, ctx))
{
DataTypes types = {first, second};
transformInferredJSONTypesIfNeeded(types, format_settings);
};
chooseResultColumnType(type, new_type, convert_types_if_needed, nullptr, column_name, row);
for (const auto & [name, type] : columns.getAll())
hints[name] = type;
}
}
void JSONColumnsSchemaReaderBase::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
{
transformInferredJSONTypesIfNeeded(type, new_type, format_settings, &inference_info);
}
NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema()
@ -220,9 +229,18 @@ NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema()
if (!names_to_types.contains(column_name))
names_order.push_back(column_name);
rows_in_block = 0;
auto column_type = readColumnAndGetDataType(column_name, rows_in_block, format_settings.max_rows_to_read_for_schema_inference - total_rows_read);
chooseResulType(names_to_types[column_name], column_type, column_name, total_rows_read + 1);
if (const auto it = hints.find(column_name); it != hints.end())
{
names_to_types[column_name] = it->second;
}
else
{
rows_in_block = 0;
auto column_type = readColumnAndGetDataType(
column_name, rows_in_block, format_settings.max_rows_to_read_for_schema_inference - total_rows_read);
chooseResultColumnType(*this, names_to_types[column_name], column_type, nullptr, column_name, total_rows_read + 1);
}
++iteration;
}
while (!reader->checkChunkEndOrSkipColumnDelimiter());
@ -237,8 +255,14 @@ NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema()
for (auto & name : names_order)
{
auto & type = names_to_types[name];
/// Check that we could determine the type of this column.
checkResultColumnTypeAndAppend(result, type, name, nullptr, format_settings.max_rows_to_read_for_schema_inference);
/// Don't check/change types from hints.
if (!hints.contains(name))
{
transformJSONTupleToArrayIfPossible(type, format_settings, &inference_info);
/// Check that we could determine the type of this column.
checkFinalInferredType(type, name, format_settings, nullptr, format_settings.max_rows_to_read_for_schema_inference);
}
result.emplace_back(name, type);
}
return result;
@ -262,8 +286,8 @@ DataTypePtr JSONColumnsSchemaReaderBase::readColumnAndGetDataType(const String &
}
readJSONField(field, in);
DataTypePtr field_type = JSONUtils::getDataTypeFromField(field, format_settings);
chooseResulType(column_type, field_type, column_name, rows_read);
DataTypePtr field_type = tryInferDataTypeForSingleJSONField(field, format_settings, &inference_info);
chooseResultColumnType(*this, column_type, field_type, nullptr, column_name, rows_read);
++rows_read;
}
while (!reader->checkColumnEndOrSkipFieldDelimiter());

View File

@ -1,6 +1,7 @@
#pragma once
#include <Formats/FormatSettings.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
@ -76,18 +77,23 @@ class JSONColumnsSchemaReaderBase : public ISchemaReader
public:
JSONColumnsSchemaReaderBase(ReadBuffer & in_, const FormatSettings & format_settings_, std::unique_ptr<JSONColumnsReaderBase> reader_);
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type);
bool needContext() const override { return !hints_str.empty(); }
void setContext(ContextPtr & ctx) override;
private:
NamesAndTypesList readSchema() override;
/// Read whole column in the block (up to max_rows_to_read rows) and extract the data type.
DataTypePtr readColumnAndGetDataType(const String & column_name, size_t & rows_read, size_t max_rows_to_read);
/// Choose result type for column from two inferred types from different rows.
void chooseResulType(DataTypePtr & type, DataTypePtr & new_type, const String & column_name, size_t row) const;
const FormatSettings format_settings;
String hints_str;
std::unordered_map<String, DataTypePtr> hints;
std::unique_ptr<JSONColumnsReaderBase> reader;
Names column_names_from_settings;
JSONInferenceInfo inference_info;
};
}

View File

@ -7,6 +7,7 @@
#include <Formats/verbosePrintString.h>
#include <Formats/JSONUtils.h>
#include <Formats/EscapingRuleUtils.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Formats/registerWithNamesAndTypes.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/Serializations/SerializationNullable.h>
@ -202,12 +203,17 @@ DataTypes JSONCompactEachRowRowSchemaReader::readRowAndGetDataTypes()
if (in.eof())
return {};
return JSONUtils::readRowAndGetDataTypesForJSONCompactEachRow(in, format_settings, reader.yieldStrings());
return JSONUtils::readRowAndGetDataTypesForJSONCompactEachRow(in, format_settings, &inference_info);
}
void JSONCompactEachRowRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t)
void JSONCompactEachRowRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
{
transformInferredJSONTypesIfNeeded(type, new_type, format_settings);
transformInferredJSONTypesIfNeeded(type, new_type, format_settings, &inference_info);
}
void JSONCompactEachRowRowSchemaReader::transformFinalTypeIfNeeded(DataTypePtr & type)
{
transformJSONTupleToArrayIfPossible(type, format_settings, &inference_info);
}
void registerInputFormatJSONCompactEachRow(FormatFactory & factory)

View File

@ -4,6 +4,7 @@
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Common/HashTable/HashMap.h>
namespace DB
@ -80,10 +81,12 @@ public:
private:
DataTypes readRowAndGetDataTypes() override;
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) override;
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
void transformFinalTypeIfNeeded(DataTypePtr & type) override;
JSONCompactEachRowFormatReader reader;
bool first_row = true;
JSONInferenceInfo inference_info;
};
}

View File

@ -4,6 +4,7 @@
#include <Processors/Formats/Impl/JSONEachRowRowInputFormat.h>
#include <Formats/JSONUtils.h>
#include <Formats/EscapingRuleUtils.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/Serializations/SerializationNullable.h>
@ -300,9 +301,8 @@ void JSONEachRowRowInputFormat::readSuffix()
assertEOF(*in);
}
JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings_, const FormatSettings & format_settings_)
JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: IRowWithNamesSchemaReader(in_, format_settings_)
, json_strings(json_strings_)
{
}
@ -336,12 +336,17 @@ NamesAndTypesList JSONEachRowSchemaReader::readRowAndGetNamesAndDataTypes(bool &
return {};
}
return JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, format_settings, json_strings);
return JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, format_settings, &inference_info);
}
void JSONEachRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
{
transformInferredJSONTypesIfNeeded(type, new_type, format_settings);
transformInferredJSONTypesIfNeeded(type, new_type, format_settings, &inference_info);
}
void JSONEachRowSchemaReader::transformFinalTypeIfNeeded(DataTypePtr & type)
{
transformJSONTupleToArrayIfPossible(type, format_settings, &inference_info);
}
void registerInputFormatJSONEachRow(FormatFactory & factory)
@ -391,11 +396,11 @@ void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory
void registerJSONEachRowSchemaReader(FormatFactory & factory)
{
auto register_schema_reader = [&](const String & format_name, bool json_strings)
auto register_schema_reader = [&](const String & format_name)
{
factory.registerSchemaReader(format_name, [json_strings](ReadBuffer & buf, const FormatSettings & settings)
factory.registerSchemaReader(format_name, [](ReadBuffer & buf, const FormatSettings & settings)
{
return std::make_unique<JSONEachRowSchemaReader>(buf, json_strings, settings);
return std::make_unique<JSONEachRowSchemaReader>(buf, settings);
});
factory.registerAdditionalInfoForSchemaCacheGetter(format_name, [](const FormatSettings & settings)
{
@ -403,10 +408,10 @@ void registerJSONEachRowSchemaReader(FormatFactory & factory)
});
};
register_schema_reader("JSONEachRow", false);
register_schema_reader("JSONLines", false);
register_schema_reader("NDJSON", false);
register_schema_reader("JSONStringsEachRow", true);
register_schema_reader("JSONEachRow");
register_schema_reader("JSONLines");
register_schema_reader("NDJSON");
register_schema_reader("JSONStringsEachRow");
}
}

View File

@ -4,6 +4,7 @@
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Common/HashTable/HashMap.h>
@ -94,15 +95,16 @@ protected:
class JSONEachRowSchemaReader : public IRowWithNamesSchemaReader
{
public:
JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings, const FormatSettings & format_settings_);
JSONEachRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_);
private:
NamesAndTypesList readRowAndGetNamesAndDataTypes(bool & eof) override;
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
void transformFinalTypeIfNeeded(DataTypePtr & type) override;
bool json_strings;
bool first_row = true;
bool data_in_square_brackets = false;
JSONInferenceInfo inference_info;
};
}

View File

@ -2,6 +2,7 @@
#include <Formats/JSONUtils.h>
#include <Formats/FormatFactory.h>
#include <Formats/EscapingRuleUtils.h>
#include <Formats/SchemaInferenceUtils.h>
#include <DataTypes/DataTypeString.h>
namespace DB
@ -85,15 +86,25 @@ NamesAndTypesList JSONObjectEachRowSchemaReader::readRowAndGetNamesAndDataTypes(
JSONUtils::skipComma(in);
JSONUtils::readFieldName(in);
auto names_and_types = JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, format_settings, false);
return JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, format_settings, &inference_info);
}
NamesAndTypesList JSONObjectEachRowSchemaReader::getStaticNamesAndTypes()
{
if (!format_settings.json_object_each_row.column_for_object_name.empty())
names_and_types.emplace_front(format_settings.json_object_each_row.column_for_object_name, std::make_shared<DataTypeString>());
return names_and_types;
return {{format_settings.json_object_each_row.column_for_object_name, std::make_shared<DataTypeString>()}};
return {};
}
void JSONObjectEachRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
{
transformInferredJSONTypesIfNeeded(type, new_type, format_settings);
transformInferredJSONTypesIfNeeded(type, new_type, format_settings, &inference_info);
}
void JSONObjectEachRowSchemaReader::transformFinalTypeIfNeeded(DataTypePtr & type)
{
transformJSONTupleToArrayIfPossible(type, format_settings, &inference_info);
}
void registerInputFormatJSONObjectEachRow(FormatFactory & factory)

View File

@ -4,6 +4,7 @@
#include <Processors/Formats/Impl/JSONEachRowRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Common/HashTable/HashMap.h>
@ -41,9 +42,12 @@ public:
private:
NamesAndTypesList readRowAndGetNamesAndDataTypes(bool & eof) override;
NamesAndTypesList getStaticNamesAndTypes() override;
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
void transformFinalTypeIfNeeded(DataTypePtr & type) override;
bool first_row = true;
JSONInferenceInfo inference_info;
};
std::optional<size_t> getColumnIndexForJSONObjectEachRowObjectName(const Block & header, const FormatSettings & settings);

View File

@ -247,6 +247,14 @@ static void insertNull(IColumn & column, DataTypePtr type)
static void insertUUID(IColumn & column, DataTypePtr type, const char * value, size_t size)
{
auto insert_func = [&](IColumn & column_, DataTypePtr type_)
{
insertUUID(column_, type_, value, size);
};
if (checkAndInsertNullable(column, type, insert_func) || checkAndInsertLowCardinality(column, type, insert_func))
return;
if (!isUUID(type))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack UUID into column with type {}.", type->getName());
ReadBufferFromMemory buf(value, size);
@ -470,16 +478,16 @@ DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object)
{
case msgpack::type::object_type::POSITIVE_INTEGER: [[fallthrough]];
case msgpack::type::object_type::NEGATIVE_INTEGER:
return makeNullable(std::make_shared<DataTypeInt64>());
return std::make_shared<DataTypeInt64>();
case msgpack::type::object_type::FLOAT32:
return makeNullable(std::make_shared<DataTypeFloat32>());
return std::make_shared<DataTypeFloat32>();
case msgpack::type::object_type::FLOAT64:
return makeNullable(std::make_shared<DataTypeFloat64>());
return std::make_shared<DataTypeFloat64>();
case msgpack::type::object_type::BOOLEAN:
return makeNullable(std::make_shared<DataTypeUInt8>());
return std::make_shared<DataTypeUInt8>();
case msgpack::type::object_type::BIN: [[fallthrough]];
case msgpack::type::object_type::STR:
return makeNullable(std::make_shared<DataTypeString>());
return std::make_shared<DataTypeString>();
case msgpack::type::object_type::ARRAY:
{
msgpack::object_array object_array = object.via.array;

View File

@ -435,7 +435,7 @@ DataTypes MySQLDumpSchemaReader::readRowAndGetDataTypes()
skipFieldDelimiter(in);
readQuotedField(value, in);
auto type = determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Quoted);
auto type = tryInferDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Quoted);
data_types.push_back(std::move(type));
}
skipEndOfRow(in, table_name);

View File

@ -3,7 +3,7 @@
#if USE_ORC
#include <Formats/FormatFactory.h>
#include <Formats/ReadSchemaUtils.h>
#include <Formats/SchemaInferenceUtils.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
@ -101,7 +101,7 @@ static size_t countIndicesForType(std::shared_ptr<arrow::DataType> type)
if (type->id() == arrow::Type::MAP)
{
auto * map_type = static_cast<arrow::MapType *>(type.get());
return countIndicesForType(map_type->key_type()) + countIndicesForType(map_type->item_type());
return countIndicesForType(map_type->key_type()) + countIndicesForType(map_type->item_type()) + 1;
}
return 1;

View File

@ -4,7 +4,7 @@
#if USE_PARQUET
#include <Formats/FormatFactory.h>
#include <Formats/ReadSchemaUtils.h>
#include <Formats/SchemaInferenceUtils.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/copyData.h>
#include <arrow/api.h>

View File

@ -3,6 +3,7 @@
#include <Processors/Formats/Impl/RegexpRowInputFormat.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <Formats/EscapingRuleUtils.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Formats/newLineSegmentationEngine.h>
#include <IO/ReadHelpers.h>
@ -155,15 +156,15 @@ DataTypes RegexpSchemaReader::readRowAndGetDataTypes()
for (size_t i = 0; i != field_extractor.getMatchedFieldsSize(); ++i)
{
String field(field_extractor.getField(i));
data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, format_settings.regexp.escaping_rule));
data_types.push_back(tryInferDataTypeByEscapingRule(field, format_settings, format_settings.regexp.escaping_rule, &json_inference_info));
}
return data_types;
}
void RegexpSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t)
void RegexpSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
{
transformInferredTypesIfNeeded(type, new_type, format_settings, format_settings.regexp.escaping_rule);
transformInferredTypesByEscapingRuleIfNeeded(type, new_type, format_settings, format_settings.regexp.escaping_rule, &json_inference_info);
}

View File

@ -5,12 +5,13 @@
#include <string>
#include <vector>
#include <Core/Block.h>
#include <IO/PeekableReadBuffer.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <Formats/FormatFactory.h>
#include <IO/PeekableReadBuffer.h>
#include <Formats/ParsedTemplateFormatString.h>
#include <Formats/SchemaInferenceUtils.h>
namespace DB
@ -81,12 +82,13 @@ public:
private:
DataTypes readRowAndGetDataTypes() override;
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) override;
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
using EscapingRule = FormatSettings::EscapingRule;
RegexpFieldExtractor field_extractor;
PeekableReadBuffer buf;
JSONInferenceInfo json_inference_info;
};
}

View File

@ -249,7 +249,7 @@ NamesAndTypesList TSKVSchemaReader::readRowAndGetNamesAndDataTypes(bool & eof)
if (has_value)
{
readEscapedString(value, in);
names_and_types.emplace_back(std::move(name), determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Escaped));
names_and_types.emplace_back(std::move(name), tryInferDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Escaped));
}
else
{

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