Merge branch 'ClickHouse:master' into master

This commit is contained in:
AntiTopQuark 2024-07-18 08:44:11 +08:00 committed by GitHub
commit e3cb0690c1
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
308 changed files with 11151 additions and 1707 deletions

View File

@ -27,6 +27,8 @@ Checks: [
'-bugprone-not-null-terminated-result',
'-bugprone-reserved-identifier', # useful but too slow, TODO retry when https://reviews.llvm.org/rG1c282052624f9d0bd273bde0b47b30c96699c6c7 is merged
'-bugprone-unchecked-optional-access',
'-bugprone-crtp-constructor-accessibility',
'-bugprone-suspicious-stringview-data-usage',
'-cert-dcl16-c',
'-cert-dcl37-c',
@ -36,6 +38,7 @@ Checks: [
'-cert-msc51-cpp',
'-cert-oop54-cpp',
'-cert-oop57-cpp',
'-cert-err33-c', # Misreports on clang-19: it warns about all functions containing 'remove' in the name, not only about the standard library.
'-clang-analyzer-optin.performance.Padding',
@ -99,6 +102,7 @@ Checks: [
'-modernize-use-emplace',
'-modernize-use-nodiscard',
'-modernize-use-trailing-return-type',
'-modernize-use-designated-initializers',
'-performance-enum-size',
'-performance-inefficient-string-concatenation',

View File

@ -62,7 +62,7 @@ jobs:
BuildDockers:
needs: [RunConfig]
if: ${{ !failure() && !cancelled() }}
uses: ./.github/workflows/reusable_docker.yml
uses: ./.github/workflows/docker_test_images.yml
with:
data: ${{ needs.RunConfig.outputs.data }}
CompatibilityCheckX86:

View File

@ -94,7 +94,7 @@ jobs:
echo "Generate Security"
python3 ./utils/security-generator/generate_security.py > SECURITY.md
git diff HEAD
- name: Generate ChangeLog
- name: Create ChangeLog PR
if: ${{ inputs.type == 'patch' && ! inputs.dry-run }}
uses: peter-evans/create-pull-request@v6
with:

View File

@ -58,7 +58,7 @@ jobs:
# BuildDockers:
# needs: [RunConfig]
# if: ${{ !failure() && !cancelled() }}
# uses: ./.github/workflows/reusable_docker.yml
# uses: ./.github/workflows/docker_test_images.yml
# with:
# data: ${{ needs.RunConfig.outputs.data }}
# StyleCheck:

View File

@ -51,7 +51,7 @@ jobs:
BuildDockers:
needs: [RunConfig]
if: ${{ !failure() && !cancelled() && toJson(fromJson(needs.RunConfig.outputs.data).docker_data.missing_multi) != '[]' }}
uses: ./.github/workflows/reusable_docker.yml
uses: ./.github/workflows/docker_test_images.yml
with:
data: ${{ needs.RunConfig.outputs.data }}
StyleCheck:

View File

@ -40,7 +40,7 @@ jobs:
} >> "$GITHUB_OUTPUT"
BuildDockers:
needs: [RunConfig]
uses: ./.github/workflows/reusable_docker.yml
uses: ./.github/workflows/docker_test_images.yml
with:
data: "${{ needs.RunConfig.outputs.data }}"
set_latest: true

View File

@ -72,7 +72,7 @@ jobs:
BuildDockers:
needs: [RunConfig]
if: ${{ !failure() && !cancelled() && toJson(fromJson(needs.RunConfig.outputs.data).docker_data.missing_multi) != '[]' }}
uses: ./.github/workflows/reusable_docker.yml
uses: ./.github/workflows/docker_test_images.yml
with:
data: ${{ needs.RunConfig.outputs.data }}
StyleCheck:

View File

@ -57,7 +57,7 @@ jobs:
BuildDockers:
needs: [RunConfig]
if: ${{ !failure() && !cancelled() }}
uses: ./.github/workflows/reusable_docker.yml
uses: ./.github/workflows/docker_test_images.yml
with:
data: ${{ needs.RunConfig.outputs.data }}
CompatibilityCheckX86:

View File

@ -3,8 +3,9 @@
#include <base/defines.h>
#include <fstream>
#include <sstream>
#include <string>
namespace fs = std::filesystem;
bool cgroupsV2Enabled()
{
@ -13,11 +14,11 @@ bool cgroupsV2Enabled()
{
/// This file exists iff the host has cgroups v2 enabled.
auto controllers_file = default_cgroups_mount / "cgroup.controllers";
if (!std::filesystem::exists(controllers_file))
if (!fs::exists(controllers_file))
return false;
return true;
}
catch (const std::filesystem::filesystem_error &) /// all "underlying OS API errors", typically: permission denied
catch (const fs::filesystem_error &) /// all "underlying OS API errors", typically: permission denied
{
return false; /// not logging the exception as most callers fall back to cgroups v1
}
@ -33,8 +34,9 @@ bool cgroupsV2MemoryControllerEnabled()
/// According to https://docs.kernel.org/admin-guide/cgroup-v2.html, file "cgroup.controllers" defines which controllers are available
/// for the current + child cgroups. The set of available controllers can be restricted from level to level using file
/// "cgroups.subtree_control". It is therefore sufficient to check the bottom-most nested "cgroup.controllers" file.
std::string cgroup = cgroupV2OfProcess();
auto cgroup_dir = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup);
fs::path cgroup_dir = cgroupV2PathOfProcess();
if (cgroup_dir.empty())
return false;
std::ifstream controllers_file(cgroup_dir / "cgroup.controllers");
if (!controllers_file.is_open())
return false;
@ -46,7 +48,7 @@ bool cgroupsV2MemoryControllerEnabled()
#endif
}
std::string cgroupV2OfProcess()
fs::path cgroupV2PathOfProcess()
{
#if defined(OS_LINUX)
chassert(cgroupsV2Enabled());
@ -54,17 +56,18 @@ std::string cgroupV2OfProcess()
/// A simpler way to get the membership is:
std::ifstream cgroup_name_file("/proc/self/cgroup");
if (!cgroup_name_file.is_open())
return "";
return {};
/// With cgroups v2, there will be a *single* line with prefix "0::/"
/// (see https://docs.kernel.org/admin-guide/cgroup-v2.html)
std::string cgroup;
std::getline(cgroup_name_file, cgroup);
static const std::string v2_prefix = "0::/";
if (!cgroup.starts_with(v2_prefix))
return "";
return {};
cgroup = cgroup.substr(v2_prefix.length());
return cgroup;
/// Note: The 'root' cgroup can have an empty cgroup name, this is valid
return default_cgroups_mount / cgroup;
#else
return "";
return {};
#endif
}

View File

@ -1,7 +1,6 @@
#pragma once
#include <filesystem>
#include <string>
#if defined(OS_LINUX)
/// I think it is possible to mount the cgroups hierarchy somewhere else (e.g. when in containers).
@ -16,7 +15,7 @@ bool cgroupsV2Enabled();
/// Assumes that cgroupsV2Enabled() is enabled.
bool cgroupsV2MemoryControllerEnabled();
/// Which cgroup does the process belong to?
/// Returns an empty string if the cgroup cannot be determined.
/// Detects which cgroup v2 the process belongs to and returns the filesystem path to the cgroup.
/// Returns an empty path the cgroup cannot be determined.
/// Assumes that cgroupsV2Enabled() is enabled.
std::string cgroupV2OfProcess();
std::filesystem::path cgroupV2PathOfProcess();

View File

@ -23,8 +23,9 @@ std::optional<uint64_t> getCgroupsV2MemoryLimit()
if (!cgroupsV2MemoryControllerEnabled())
return {};
std::string cgroup = cgroupV2OfProcess();
auto current_cgroup = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup);
std::filesystem::path current_cgroup = cgroupV2PathOfProcess();
if (current_cgroup.empty())
return {};
/// Open the bottom-most nested memory limit setting file. If there is no such file at the current
/// level, try again at the parent level as memory settings are inherited.

2
contrib/grpc vendored

@ -1 +1 @@
Subproject commit f5b7fdc2dff09ada06dbf6c75df298fb40f898df
Subproject commit 1716359d2e28d304a250f9df0e6c0ccad03de8db

2
contrib/libunwind vendored

@ -1 +1 @@
Subproject commit d6a01c46327e56fd86beb8aaa31591fcd9a6b7df
Subproject commit 8f28e64d15819d2d096badd598c7d85bebddb1f2

View File

@ -4,9 +4,6 @@ set(LIBUNWIND_CXX_SOURCES
"${LIBUNWIND_SOURCE_DIR}/src/libunwind.cpp"
"${LIBUNWIND_SOURCE_DIR}/src/Unwind-EHABI.cpp"
"${LIBUNWIND_SOURCE_DIR}/src/Unwind-seh.cpp")
if (APPLE)
set(LIBUNWIND_CXX_SOURCES ${LIBUNWIND_CXX_SOURCES} "${LIBUNWIND_SOURCE_DIR}/src/Unwind_AppleExtras.cpp")
endif ()
set(LIBUNWIND_C_SOURCES
"${LIBUNWIND_SOURCE_DIR}/src/UnwindLevel1.c"
@ -32,6 +29,7 @@ set_target_properties(unwind PROPERTIES FOLDER "contrib/libunwind-cmake")
target_include_directories(unwind SYSTEM BEFORE PUBLIC $<BUILD_INTERFACE:${LIBUNWIND_SOURCE_DIR}/include>)
target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_NO_HEAP=1)
target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_REMEMBER_STACK_ALLOC=1)
# NOTE: from this macros sizeof(unw_context_t)/sizeof(unw_cursor_t) is depends, so it should be set always
target_compile_definitions(unwind PUBLIC -D_LIBUNWIND_IS_NATIVE_ONLY)

View File

@ -26,7 +26,10 @@ RUN apt-get update \
zstd \
--yes --no-install-recommends \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \
&& groupadd --system --gid 1000 clickhouse \
&& useradd --system --gid 1000 --uid 1000 -m clickhouse
# ^ For some reason, groupadd and useradd are needed for tests with 'expect', but I don't know, why.
COPY requirements.txt /
RUN pip3 install --no-cache-dir -r /requirements.txt

View File

@ -9,7 +9,7 @@ trap 'kill $(jobs -pr) ||:' EXIT
stage=${stage:-}
# Compiler version, normally set by Dockerfile
export LLVM_VERSION=${LLVM_VERSION:-17}
export LLVM_VERSION=${LLVM_VERSION:-18}
# A variable to pass additional flags to CMake.
# Here we explicitly default it to nothing so that bash doesn't complain about

View File

@ -33,13 +33,9 @@ RUN apt-get update \
COPY requirements.txt /
RUN pip3 install --no-cache-dir -r /requirements.txt
COPY * /
ENV FUZZER_ARGS="-max_total_time=60"
SHELL ["/bin/bash", "-c"]
CMD set -o pipefail \
&& timeout -s 9 1h /run_libfuzzer.py 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee main.log
# docker run --network=host --volume <workspace>:/workspace -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/libfuzzer

View File

@ -12,7 +12,7 @@ MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 7200 : MAX_RUN_TIME))
USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0}
USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0}
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=1
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] || [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0

View File

@ -17,6 +17,7 @@ ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=18
RUN apt-get update \
&& apt-get install \
sudo \
apt-transport-https \
apt-utils \
ca-certificates \

View File

@ -11,7 +11,7 @@ This is for the case when you have Linux machine and want to use it to build `cl
The cross-build for RISC-V 64 is based on the [Build instructions](../development/build.md), follow them first.
## Install Clang-16
## Install Clang-18
Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup or do
```

View File

@ -185,6 +185,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va
- `--format, -f` Use the specified default format to output the result.
- `--vertical, -E` If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables.
- `--time, -t` If specified, print the query execution time to stderr in non-interactive mode.
- `--memory-usage` If specified, print memory usage to stderr in non-interactive mode]. Possible values: 'none' - do not print memory usage, 'default' - print number of bytes, 'readable' - print memory usage in human-readable format.
- `--stacktrace` If specified, also print the stack trace if an exception occurs.
- `--config-file` The name of the configuration file.
- `--secure` If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl).
@ -339,7 +340,7 @@ clickhouse-client clickhouse://some_user%40some_mail.com@localhost:9000
Connect to one of provides hosts: `192.168.1.15`, `192.168.1.25`.
``` bash
clickhouse-client clickhouse://192.168.1.15,192.168.1.25
clickhouse-client clickhouse://192.168.1.15,192.168.1.25
```
### Configuration Files {#configuration_files}
@ -367,7 +368,7 @@ Example of a config file:
```
Or the same config in a YAML format:
```yaml
user: username
password: 'password'

View File

@ -6,7 +6,20 @@ sidebar_label: MySQL Interface
# MySQL Interface
ClickHouse supports the MySQL wire protocol. This allow tools that are MySQL-compatible to interact with ClickHouse seamlessly (e.g. [Looker Studio](../integrations/data-visualization/looker-studio-and-clickhouse.md)).
ClickHouse supports the MySQL wire protocol. This allows certain clients that do not have native ClickHouse connectors leverage the MySQL protocol instead, and it has been validated with the following BI tools:
- [Looker Studio](../integrations/data-visualization/looker-studio-and-clickhouse.md)
- [Tableau Online](../integrations/tableau-online)
- [QuickSight](../integrations/quicksight)
If you are trying other untested clients or integrations, keep in mind that there could be the following limitations:
- SSL implementation might not be fully compatible; there could be potential [TLS SNI](https://www.cloudflare.com/learning/ssl/what-is-sni/) issues.
- A particular tool might require dialect features (e.g., MySQL-specific functions or settings) that are not implemented yet.
If there is a native driver available (e.g., [DBeaver](../integrations/dbeaver)), it is always preferred to use it instead of the MySQL interface. Additionally, while most of the MySQL language clients should work fine, MySQL interface is not guaranteed to be a drop-in replacement for a codebase with existing MySQL queries.
If your use case involves a particular tool that does not have a native ClickHouse driver, and you would like to use it via the MySQL interface and you found certain incompatibilities - please [create an issue](https://github.com/ClickHouse/ClickHouse/issues) in the ClickHouse repository.
## Enabling the MySQL Interface On ClickHouse Cloud

View File

@ -0,0 +1,35 @@
---
slug: /en/operations/system-tables/detached_tables
---
# detached_tables
Contains information of each detached table.
Columns:
- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the table is in.
- `table` ([String](../../sql-reference/data-types/string.md)) — Table name.
- `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Table uuid (Atomic database).
- `metadata_path` ([String](../../sql-reference/data-types/string.md)) - Path to the table metadata in the file system.
- `is_permanently` ([UInt8](../../sql-reference/data-types/int-uint.md)) - Flag indicates that the table was detached PERMANENTLY.
**Example**
```sql
SELECT * FROM system.detached_tables FORMAT Vertical;
```
```text
Row 1:
──────
database: base
table: t1
uuid: 81b1c20a-b7c6-4116-a2ce-7583fb6b6736
metadata_path: /var/lib/clickhouse/store/461/461cf698-fd0b-406d-8c01-5d8fd5748a91/t1.sql
is_permanently: 1
```

View File

@ -96,3 +96,22 @@ Result:
│ 1 │ [2] │ [[4,1]] │
└───────────┴───────────┴───────────┘
```
## Reading nested subcolumns from Array
If nested type `T` inside `Array` has subcolumns (for example, if it's a [named tuple](./tuple.md)), you can read its subcolumns from an `Array(T)` type with the same subcolumn names. The type of a subcolumn will be `Array` of the type of original subcolumn.
**Example**
```sql
CREATE TABLE t_arr (arr Array(Tuple(field1 UInt32, field2 String))) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO t_arr VALUES ([(1, 'Hello'), (2, 'World')]), ([(3, 'This'), (4, 'is'), (5, 'subcolumn')]);
SELECT arr.field1, toTypeName(arr.field1), arr.field2, toTypeName(arr.field2) from t_arr;
```
```test
┌─arr.field1─┬─toTypeName(arr.field1)─┬─arr.field2────────────────┬─toTypeName(arr.field2)─┐
│ [1,2] │ Array(UInt32) │ ['Hello','World'] │ Array(String) │
│ [3,4,5] │ Array(UInt32) │ ['This','is','subcolumn'] │ Array(String) │
└────────────┴────────────────────────┴───────────────────────────┴────────────────────────┘
```

View File

@ -56,7 +56,6 @@ Functions:
## Related content
- [Reducing ClickHouse Storage Cost with the Low Cardinality Type Lessons from an Instana Engineer](https://altinity.com/blog/2020-5-20-reducing-clickhouse-storage-cost-with-the-low-cardinality-type-lessons-from-an-instana-engineer)
- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf)
- Blog: [Optimizing ClickHouse with Schemas and Codecs](https://clickhouse.com/blog/optimize-clickhouse-codecs-compression-schema)
- Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse)
- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf)

View File

@ -76,7 +76,7 @@ WHERE macro = 'test';
└───────┴──────────────┘
```
## FQDN
## fqdn
Returns the fully qualified domain name of the ClickHouse server.
@ -86,7 +86,7 @@ Returns the fully qualified domain name of the ClickHouse server.
fqdn();
```
Aliases: `fullHostName`, 'FQDN'.
Aliases: `fullHostName`, `FQDN`.
**Returned value**

View File

@ -567,12 +567,13 @@ While no standard or recommendation exists for the epoch of Snowflake IDs, imple
**Syntax**
``` sql
generateSnowflakeID([expr])
generateSnowflakeID([expr, [machine_id]])
```
**Arguments**
- `expr` — An arbitrary [expression](../../sql-reference/syntax.md#syntax-expressions) used to bypass [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in a query. The value of the expression has no effect on the returned Snowflake ID. Optional.
- `machine_id` — A machine ID, the lowest 10 bits are used. [Int64](../data-types/int-uint.md). Optional.
**Returned value**
@ -608,6 +609,16 @@ SELECT generateSnowflakeID(1), generateSnowflakeID(2);
└────────────────────────┴────────────────────────┘
```
**Example with expression and a machine ID**
```
SELECT generateSnowflakeID('expr', 1);
┌─generateSnowflakeID('expr', 1)─┐
│ 7201148511606784002 │
└────────────────────────────────┘
```
## snowflakeToDateTime
:::warning

View File

@ -356,7 +356,7 @@ sidebar_label: "\u53D8\u66F4\u65E5\u5FD7"
#### 新功能 {#new-feature-1}
- 添加 `deduplicate_blocks_in_dependent_materialized_views` 用于控制具有实例化视图的表中幂等插入的行为的选项。 这个新功能是由Altinity的特殊要求添加到错误修正版本中的。
- 添加 `deduplicate_blocks_in_dependent_materialized_views` 用于控制具有实例化视图的表中幂等插入的行为的选项。
[#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy)
### ClickHouse版本v20.1.2.4,2020-01-22 {#clickhouse-release-v20-1-2-4-2020-01-22}

View File

@ -82,14 +82,14 @@ FROM LEFT_RIGHT
SELECT
left,
right,
if(left < right, 'left is smaller than right', 'right is greater or equal than left') AS is_smaller
if(left < right, 'left is smaller than right', 'right is smaller or equal than left') AS is_smaller
FROM LEFT_RIGHT
WHERE isNotNull(left) AND isNotNull(right)
┌─left─┬─right─┬─is_smaller──────────────────────────┐
│ 1 │ 3 │ left is smaller than right │
│ 2 │ 2 │ right is greater or equal than left │
│ 3 │ 1 │ right is greater or equal than left │
│ 2 │ 2 │ right is smaller or equal than left │
│ 3 │ 1 │ right is smaller or equal than left │
└──────┴───────┴─────────────────────────────────────┘
```

View File

@ -4,6 +4,9 @@ if (USE_CLANG_TIDY)
set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}")
endif ()
set(MAX_LINKER_MEMORY 3500)
include(../cmake/limit_jobs.cmake)
include(${ClickHouse_SOURCE_DIR}/cmake/split_debug_symbols.cmake)
# The `clickhouse` binary is a multi purpose tool that contains multiple execution modes (client, server, etc.),

View File

@ -186,6 +186,8 @@ void Client::parseConnectionsCredentials(Poco::Util::AbstractConfiguration & con
history_file = home_path + "/" + history_file.substr(1);
config.setString("history_file", history_file);
}
if (config.has(prefix + ".accept-invalid-certificate"))
config.setBool("accept-invalid-certificate", config.getBool(prefix + ".accept-invalid-certificate"));
}
if (!connection_name.empty() && !connection_found)
@ -277,6 +279,12 @@ void Client::initialize(Poco::Util::Application & self)
else if (config().has("connection"))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "--connection was specified, but config does not exist");
if (config().has("accept-invalid-certificate"))
{
config().setString("openSSL.client.invalidCertificateHandler.name", "AcceptCertificateHandler");
config().setString("openSSL.client.verificationMode", "none");
}
/** getenv is thread-safe in Linux glibc and in all sane libc implementations.
* But the standard does not guarantee that subsequent calls will not rewrite the value by returned pointer.
*
@ -731,7 +739,7 @@ bool Client::processWithFuzzing(const String & full_query)
}
if (auto *q = orig_ast->as<ASTSetQuery>())
{
if (auto *setDialect = q->changes.tryGet("dialect"); setDialect && setDialect->safeGet<String>() == "kusto")
if (auto *set_dialect = q->changes.tryGet("dialect"); set_dialect && set_dialect->safeGet<String>() == "kusto")
return true;
}

View File

@ -1,5 +1,6 @@
<!-- Config set into /etc/clickhouse-client/. It's used if no other configs are found. -->
<config>
<!-- Shorthand for self-signed combination in openSSL section below: <accept-invalid-certificate>1</accept-invalid-certificate> -->
<openSSL>
<client> <!-- Used for connection to server's secure tcp port -->
<loadDefaultCAFile>true</loadDefaultCAFile>
@ -72,6 +73,7 @@
Default: "hostname" will be used. -->
<name>default</name>
<!-- For self-signed server certificate when connecting to secure tcp: <accept-invalid-certificate>1</accept-invalid-certificate> -->
<!-- Host that will be used for connection. -->
<hostname>127.0.0.1</hostname>
<port>9000</port>

View File

@ -12,6 +12,7 @@
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
namespace DB
{
@ -616,6 +617,7 @@ private:
bool is_any_nullable = false;
Tuple args;
args.reserve(equals_functions.size());
DataTypes tuple_element_types;
/// first we create tuple from RHS of equals functions
for (const auto & equals : equals_functions)
{
@ -628,16 +630,18 @@ private:
if (const auto * rhs_literal = equals_arguments[1]->as<ConstantNode>())
{
args.push_back(rhs_literal->getValue());
tuple_element_types.push_back(rhs_literal->getResultType());
}
else
{
const auto * lhs_literal = equals_arguments[0]->as<ConstantNode>();
assert(lhs_literal);
args.push_back(lhs_literal->getValue());
tuple_element_types.push_back(lhs_literal->getResultType());
}
}
auto rhs_node = std::make_shared<ConstantNode>(std::move(args));
auto rhs_node = std::make_shared<ConstantNode>(std::move(args), std::make_shared<DataTypeTuple>(std::move(tuple_element_types)));
auto in_function = std::make_shared<FunctionNode>("in");

View File

@ -1,6 +1,7 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeObject.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/NestedUtils.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
@ -680,9 +681,33 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromStorage(
bool match_full_identifier = false;
const auto & identifier_full_name = identifier_without_column_qualifier.getFullName();
auto it = table_expression_data.column_name_to_column_node.find(identifier_full_name);
bool can_resolve_directly_from_storage = it != table_expression_data.column_name_to_column_node.end();
if (can_resolve_directly_from_storage && table_expression_data.subcolumn_names.contains(identifier_full_name))
ColumnNodePtr result_column_node;
bool can_resolve_directly_from_storage = false;
bool is_subcolumn = false;
if (auto it = table_expression_data.column_name_to_column_node.find(identifier_full_name); it != table_expression_data.column_name_to_column_node.end())
{
can_resolve_directly_from_storage = true;
is_subcolumn = table_expression_data.subcolumn_names.contains(identifier_full_name);
result_column_node = it->second;
}
/// Check if it's a dynamic subcolumn
else
{
auto [column_name, dynamic_subcolumn_name] = Nested::splitName(identifier_full_name);
auto jt = table_expression_data.column_name_to_column_node.find(column_name);
if (jt != table_expression_data.column_name_to_column_node.end() && jt->second->getColumnType()->hasDynamicSubcolumns())
{
if (auto dynamic_subcolumn_type = jt->second->getColumnType()->tryGetSubcolumnType(dynamic_subcolumn_name))
{
result_column_node = std::make_shared<ColumnNode>(NameAndTypePair{identifier_full_name, dynamic_subcolumn_type}, jt->second->getColumnSource());
can_resolve_directly_from_storage = true;
is_subcolumn = true;
}
}
}
if (can_resolve_directly_from_storage && is_subcolumn)
{
/** In the case when we have an ARRAY JOIN, we should not resolve subcolumns directly from storage.
* For example, consider the following SQL query:
@ -698,11 +723,11 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromStorage(
if (can_resolve_directly_from_storage)
{
match_full_identifier = true;
result_expression = it->second;
result_expression = result_column_node;
}
else
{
it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.at(0));
auto it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.at(0));
if (it != table_expression_data.column_name_to_column_node.end())
result_expression = it->second;
}

View File

@ -3416,14 +3416,14 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
function_base = function->build(argument_columns);
/// Do not constant fold get scalar functions
bool disable_constant_folding = function_name == "__getScalar" || function_name == "shardNum" ||
function_name == "shardCount" || function_name == "hostName" || function_name == "tcpPort";
// bool disable_constant_folding = function_name == "__getScalar" || function_name == "shardNum" ||
// function_name == "shardCount" || function_name == "hostName" || function_name == "tcpPort";
/** If function is suitable for constant folding try to convert it to constant.
* Example: SELECT plus(1, 1);
* Result: SELECT 2;
*/
if (function_base->isSuitableForConstantFolding() && !disable_constant_folding)
if (function_base->isSuitableForConstantFolding()) // && !disable_constant_folding)
{
auto result_type = function_base->getResultType();
auto executable_function = function_base->prepare(argument_columns);
@ -3832,6 +3832,10 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(
node->convertToNullable();
break;
}
/// Check parent scopes until find current query scope.
if (scope_ptr->scope_node->getNodeType() == QueryTreeNodeType::QUERY)
break;
}
}

View File

@ -55,9 +55,9 @@ namespace
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
settings.auth_settings.region,
context->getRemoteHostFilter(),
static_cast<unsigned>(global_settings.s3_max_redirects),
static_cast<unsigned>(global_settings.s3_retry_attempts),
global_settings.enable_s3_requests_logging,
static_cast<unsigned>(local_settings.s3_max_redirects),
static_cast<unsigned>(local_settings.backup_restore_s3_retry_attempts),
local_settings.enable_s3_requests_logging,
/* for_disk_s3 = */ false,
request_settings.get_request_throttler,
request_settings.put_request_throttler,

View File

@ -80,6 +80,7 @@
#include <Common/config_version.h>
#include "config.h"
namespace fs = std::filesystem;
using namespace std::literals;
@ -2069,9 +2070,18 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
progress_indication.writeFinalProgress();
output_stream << std::endl << std::endl;
}
else if (getClientConfiguration().getBool("print-time-to-stderr", false))
else
{
error_stream << progress_indication.elapsedSeconds() << "\n";
const auto & config = getClientConfiguration();
if (config.getBool("print-time-to-stderr", false))
error_stream << progress_indication.elapsedSeconds() << "\n";
const auto & print_memory_mode = config.getString("print-memory-to-stderr", "");
auto peak_memeory_usage = std::max<Int64>(progress_indication.getMemoryUsage().peak, 0);
if (print_memory_mode == "default")
error_stream << peak_memeory_usage << "\n";
else if (print_memory_mode == "readable")
error_stream << formatReadableSizeWithBinarySuffix(peak_memeory_usage) << "\n";
}
if (!is_interactive && getClientConfiguration().getBool("print-num-processed-rows", false))
@ -2565,12 +2575,12 @@ void ClientBase::runInteractive()
word_break_characters,
highlight_callback);
#else
(void)word_break_characters;
LineReader lr(
history_file,
getClientConfiguration().has("multiline"),
query_extenders,
query_delimiters,
word_break_characters);
query_delimiters);
#endif
static const std::initializer_list<std::pair<String, String>> backslash_aliases =
@ -3035,6 +3045,7 @@ void ClientBase::init(int argc, char ** argv)
("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.")
("wait_for_suggestions_to_load", "Load suggestion data synchonously.")
("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)")
("memory-usage", po::value<std::string>()->implicit_value("default")->default_value("none"), "print memory usage to stderr in non-interactive mode (for benchmarks). Values: 'none', 'default', 'readable'")
("echo", "in batch mode, print query before execution")
@ -3120,6 +3131,14 @@ void ClientBase::init(int argc, char ** argv)
/// Output execution time to stderr in batch mode.
if (options.count("time"))
getClientConfiguration().setBool("print-time-to-stderr", true);
if (options.count("memory-usage"))
{
const auto & memory_usage_mode = options["memory-usage"].as<std::string>();
if (memory_usage_mode != "none" && memory_usage_mode != "default" && memory_usage_mode != "readable")
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown memory-usage mode: {}", memory_usage_mode);
getClientConfiguration().setString("print-memory-to-stderr", memory_usage_mode);
}
if (options.count("query"))
queries = options["query"].as<std::vector<std::string>>();
if (options.count("query_id"))

View File

@ -103,7 +103,7 @@ public:
Entry get(const ConnectionTimeouts & timeouts, /// NOLINT
const Settings & settings,
bool force_connected = true) override;
bool force_connected) override;
std::string getDescription() const
{

View File

@ -54,8 +54,6 @@ public:
struct ReplicaInfo
{
bool collaborate_with_initiator{false};
size_t all_replicas_count{0};
size_t number_of_current_replica{0};
};

View File

@ -46,8 +46,7 @@ public:
Patterns delimiters,
std::istream & input_stream_ = std::cin,
std::ostream & output_stream_ = std::cout,
int in_fd_ = STDIN_FILENO
);
int in_fd_ = STDIN_FILENO);
virtual ~LineReader() = default;

View File

@ -142,13 +142,12 @@ void MultiplexedConnections::sendQuery(
modified_settings.group_by_two_level_threshold = 0;
modified_settings.group_by_two_level_threshold_bytes = 0;
}
}
if (replica_info)
{
client_info.collaborate_with_initiator = true;
client_info.count_participating_replicas = replica_info->all_replicas_count;
client_info.number_of_current_replica = replica_info->number_of_current_replica;
}
if (replica_info)
{
client_info.collaborate_with_initiator = true;
client_info.number_of_current_replica = replica_info->number_of_current_replica;
}
/// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting.

View File

@ -362,6 +362,9 @@ ReplxxLineReader::ReplxxLineReader(
rx.bind_key(Replxx::KEY::control('N'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::HISTORY_NEXT, code); });
rx.bind_key(Replxx::KEY::control('P'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::HISTORY_PREVIOUS, code); });
/// We don't want the default, "suspend" behavior, it confuses people.
rx.bind_key_internal(replxx::Replxx::KEY::control('Z'), "insert_character");
auto commit_action = [this](char32_t code)
{
/// If we allow multiline and there is already something in the input, start a newline.

View File

@ -1101,4 +1101,10 @@ void ColumnObject::finalize()
checkObjectHasNoAmbiguosPaths(getKeys());
}
void ColumnObject::updateHashFast(SipHash & hash) const
{
for (const auto & entry : subcolumns)
for (auto & part : entry->data.data)
part->updateHashFast(hash);
}
}

View File

@ -253,7 +253,7 @@ public:
const char * skipSerializedInArena(const char *) const override { throwMustBeConcrete(); }
void updateHashWithValue(size_t, SipHash &) const override { throwMustBeConcrete(); }
void updateWeakHash32(WeakHash32 &) const override { throwMustBeConcrete(); }
void updateHashFast(SipHash &) const override { throwMustBeConcrete(); }
void updateHashFast(SipHash & hash) const override;
void expand(const Filter &, bool) override { throwMustBeConcrete(); }
bool hasEqualValues() const override { throwMustBeConcrete(); }
size_t byteSizeAt(size_t) const override { throwMustBeConcrete(); }

View File

@ -49,6 +49,7 @@ void logAboutProgress(LoggerPtr log, size_t processed, size_t total, AtomicStopw
AsyncLoader::Pool::Pool(const AsyncLoader::PoolInitializer & init)
: name(init.name)
, priority(init.priority)
, max_threads(init.max_threads > 0 ? init.max_threads : getNumberOfPhysicalCPUCores())
, thread_pool(std::make_unique<ThreadPool>(
init.metric_threads,
init.metric_active_threads,
@ -56,17 +57,16 @@ AsyncLoader::Pool::Pool(const AsyncLoader::PoolInitializer & init)
/* max_threads = */ std::numeric_limits<size_t>::max(), // Unlimited number of threads, we do worker management ourselves
/* max_free_threads = */ 0, // We do not require free threads
/* queue_size = */0)) // Unlimited queue to avoid blocking during worker spawning
, max_threads(init.max_threads > 0 ? init.max_threads : getNumberOfPhysicalCPUCores())
{}
AsyncLoader::Pool::Pool(Pool&& o) noexcept
: name(o.name)
, priority(o.priority)
, thread_pool(std::move(o.thread_pool))
, ready_queue(std::move(o.ready_queue))
, max_threads(o.max_threads)
, workers(o.workers)
, suspended_workers(o.suspended_workers.load()) // All these constructors are needed because std::atomic is neither copy-constructible, nor move-constructible. We never move pools after init, so it is safe.
, thread_pool(std::move(o.thread_pool))
{}
void cancelOnDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel)

View File

@ -365,11 +365,11 @@ private:
{
const String name;
const Priority priority;
std::unique_ptr<ThreadPool> thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools.
std::map<UInt64, LoadJobPtr> ready_queue; // FIFO queue of jobs to be executed in this pool. Map is used for faster erasing. Key is `ready_seqno`
size_t max_threads; // Max number of workers to be spawn
size_t workers = 0; // Number of currently executing workers
std::atomic<size_t> suspended_workers{0}; // Number of workers that are blocked by `wait()` call on a job executing in the same pool (for deadlock resolution)
std::unique_ptr<ThreadPool> thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools.
explicit Pool(const PoolInitializer & init);
Pool(Pool&& o) noexcept;

View File

@ -25,6 +25,7 @@
#endif
using namespace DB;
namespace fs = std::filesystem;
namespace DB
{
@ -69,7 +70,7 @@ uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & ke
struct CgroupsV1Reader : ICgroupsReader
{
explicit CgroupsV1Reader(const std::filesystem::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { }
explicit CgroupsV1Reader(const fs::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { }
uint64_t readMemoryUsage() override
{
@ -85,7 +86,7 @@ private:
struct CgroupsV2Reader : ICgroupsReader
{
explicit CgroupsV2Reader(const std::filesystem::path & stat_file_dir)
explicit CgroupsV2Reader(const fs::path & stat_file_dir)
: current_buf(stat_file_dir / "memory.current"), stat_buf(stat_file_dir / "memory.stat")
{
}
@ -129,8 +130,9 @@ std::optional<std::string> getCgroupsV2Path()
if (!cgroupsV2MemoryControllerEnabled())
return {};
String cgroup = cgroupV2OfProcess();
auto current_cgroup = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup);
fs::path current_cgroup = cgroupV2PathOfProcess();
if (current_cgroup.empty())
return {};
/// Return the bottom-most nested current memory file. If there is no such file at the current
/// level, try again at the parent level as memory settings are inherited.
@ -138,7 +140,7 @@ std::optional<std::string> getCgroupsV2Path()
{
const auto current_path = current_cgroup / "memory.current";
const auto stat_path = current_cgroup / "memory.stat";
if (std::filesystem::exists(current_path) && std::filesystem::exists(stat_path))
if (fs::exists(current_path) && fs::exists(stat_path))
return {current_cgroup};
current_cgroup = current_cgroup.parent_path();
}
@ -148,7 +150,7 @@ std::optional<std::string> getCgroupsV2Path()
std::optional<std::string> getCgroupsV1Path()
{
auto path = default_cgroups_mount / "memory/memory.stat";
if (!std::filesystem::exists(path))
if (!fs::exists(path))
return {};
return {default_cgroups_mount / "memory"};
}

View File

@ -0,0 +1,184 @@
#pragma once
#include <base/defines.h>
#include <Common/Exception.h>
#include <algorithm>
#include <memory>
#include <typeindex>
#include <vector>
#include <string>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/* This is a collections of objects derived from ItemBase.
* Collection contains no more than one instance for each derived type.
* The derived type is used to access the instance.
*/
template<class ItemBase>
class CollectionOfDerivedItems
{
public:
using Self = CollectionOfDerivedItems<ItemBase>;
using ItemPtr = std::shared_ptr<ItemBase>;
private:
struct Rec
{
std::type_index type_idx;
ItemPtr ptr;
bool operator<(const Rec & other) const
{
return type_idx < other.type_idx;
}
bool operator<(const std::type_index & value) const
{
return type_idx < value;
}
bool operator==(const Rec & other) const
{
return type_idx == other.type_idx;
}
};
using Records = std::vector<Rec>;
public:
void swap(Self & other) noexcept
{
records.swap(other.records);
}
void clear()
{
records.clear();
}
bool empty() const
{
return records.empty();
}
size_t size() const
{
return records.size();
}
Self clone() const
{
Self result;
result.records.reserve(records.size());
for (const auto & rec : records)
result.records.emplace_back(rec.type_idx, rec.ptr->clone());
return result;
}
void append(Self && other)
{
auto middle_idx = records.size();
std::move(other.records.begin(), other.records.end(), std::back_inserter(records));
std::inplace_merge(records.begin(), records.begin() + middle_idx, records.end());
chassert(isUniqTypes());
}
template <class T>
void add(std::shared_ptr<T> info)
{
static_assert(std::is_base_of_v<ItemBase, T>, "Template parameter must inherit items base class");
return addImpl(std::type_index(typeid(T)), std::move(info));
}
template <class T>
std::shared_ptr<T> get() const
{
static_assert(std::is_base_of_v<ItemBase, T>, "Template parameter must inherit items base class");
auto it = getImpl(std::type_index(typeid(T)));
if (it == records.cend())
return nullptr;
auto cast = std::dynamic_pointer_cast<T>(it->ptr);
chassert(cast);
return cast;
}
template <class T>
std::shared_ptr<T> extract()
{
static_assert(std::is_base_of_v<ItemBase, T>, "Template parameter must inherit items base class");
auto it = getImpl(std::type_index(typeid(T)));
if (it == records.cend())
return nullptr;
auto cast = std::dynamic_pointer_cast<T>(it->ptr);
chassert(cast);
records.erase(it);
return cast;
}
std::string debug() const
{
std::string result;
for (auto & rec : records)
{
result.append(rec.type_idx.name());
result.append(" ");
}
return result;
}
private:
bool isUniqTypes() const
{
auto uniq_it = std::adjacent_find(records.begin(), records.end());
return uniq_it == records.end();
}
void addImpl(std::type_index type_idx, ItemPtr item)
{
auto it = std::lower_bound(records.begin(), records.end(), type_idx);
if (it == records.end())
{
records.emplace_back(type_idx, item);
return;
}
if (it->type_idx == type_idx)
throw Exception(ErrorCodes::LOGICAL_ERROR, "inserted items must be unique by their type, type {} is inserted twice", type_idx.name());
records.emplace(it, type_idx, item);
chassert(isUniqTypes());
}
Records::const_iterator getImpl(std::type_index type_idx) const
{
auto it = std::lower_bound(records.cbegin(), records.cend(), type_idx);
if (it == records.cend())
return records.cend();
if (it->type_idx != type_idx)
return records.cend();
return it;
}
Records records;
};
}

View File

@ -72,11 +72,6 @@ public:
/// How much seconds passed since query execution start.
double elapsedSeconds() const { return getElapsedNanoseconds() / 1e9; }
void updateThreadEventData(HostToTimesMap & new_hosts_data);
private:
double getCPUUsage();
struct MemoryUsage
{
UInt64 total = 0;
@ -86,6 +81,11 @@ private:
MemoryUsage getMemoryUsage() const;
void updateThreadEventData(HostToTimesMap & new_hosts_data);
private:
double getCPUUsage();
UInt64 getElapsedNanoseconds() const;
/// This flag controls whether to show the progress bar. We start showing it after

View File

@ -37,12 +37,12 @@ uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count)
/// cgroupsv2
if (cgroupsV2Enabled())
{
/// First, we identify the cgroup the process belongs
std::string cgroup = cgroupV2OfProcess();
if (cgroup.empty())
/// First, we identify the path of the cgroup the process belongs
std::filesystem::path cgroup_path = cgroupV2PathOfProcess();
if (cgroup_path.empty())
return default_cpu_count;
auto current_cgroup = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup);
auto current_cgroup = cgroup_path;
// Looking for cpu.max in directories from the current cgroup to the top level
// It does not stop on the first time since the child could have a greater value than parent
@ -62,7 +62,7 @@ uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count)
}
current_cgroup = current_cgroup.parent_path();
}
current_cgroup = default_cgroups_mount / cgroup;
current_cgroup = cgroup_path;
// Looking for cpuset.cpus.effective in directories from the current cgroup to the top level
while (current_cgroup != default_cgroups_mount.parent_path())
{

View File

@ -80,7 +80,7 @@ inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align)
throw std::bad_alloc{};
}
inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept
inline ALWAYS_INLINE void * newNoExcept(std::size_t size) noexcept
{
#if USE_GWP_ASAN
if (unlikely(GWPAsan::GuardedAlloc.shouldSample()))
@ -99,7 +99,7 @@ inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept
return malloc(size);
}
inline ALWAYS_INLINE void * newNoExept(std::size_t size, std::align_val_t align) noexcept
inline ALWAYS_INLINE void * newNoExcept(std::size_t size, std::align_val_t align) noexcept
{
#if USE_GWP_ASAN
if (unlikely(GWPAsan::GuardedAlloc.shouldSample()))

View File

@ -87,7 +87,7 @@ void * operator new(std::size_t size, const std::nothrow_t &) noexcept
{
AllocationTrace trace;
std::size_t actual_size = Memory::trackMemory(size, trace);
void * ptr = Memory::newNoExept(size);
void * ptr = Memory::newNoExcept(size);
trace.onAlloc(ptr, actual_size);
return ptr;
}
@ -96,7 +96,7 @@ void * operator new[](std::size_t size, const std::nothrow_t &) noexcept
{
AllocationTrace trace;
std::size_t actual_size = Memory::trackMemory(size, trace);
void * ptr = Memory::newNoExept(size);
void * ptr = Memory::newNoExcept(size);
trace.onAlloc(ptr, actual_size);
return ptr;
}
@ -105,7 +105,7 @@ void * operator new(std::size_t size, std::align_val_t align, const std::nothrow
{
AllocationTrace trace;
std::size_t actual_size = Memory::trackMemory(size, trace, align);
void * ptr = Memory::newNoExept(size, align);
void * ptr = Memory::newNoExcept(size, align);
trace.onAlloc(ptr, actual_size);
return ptr;
}
@ -114,7 +114,7 @@ void * operator new[](std::size_t size, std::align_val_t align, const std::nothr
{
AllocationTrace trace;
std::size_t actual_size = Memory::trackMemory(size, trace, align);
void * ptr = Memory::newNoExept(size, align);
void * ptr = Memory::newNoExcept(size, align);
trace.onAlloc(ptr, actual_size);
return ptr;
}

View File

@ -36,7 +36,7 @@ class IColumn;
M(Dialect, dialect, Dialect::clickhouse, "Which dialect will be used to parse query", 0)\
M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \
M(UInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \
M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \
M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size in rows for reading", 0) \
M(UInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "The maximum block size for insertion, if we control the creation of blocks for insertion.", 0) \
M(UInt64, min_insert_block_size_rows, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough.", 0) \
M(UInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.", 0) \
@ -502,6 +502,7 @@ class IColumn;
M(UInt64, backup_restore_keeper_value_max_size, 1048576, "Maximum size of data of a [Zoo]Keeper's node during backup", 0) \
M(UInt64, backup_restore_batch_size_for_keeper_multiread, 10000, "Maximum size of batch for multiread request to [Zoo]Keeper during backup or restore", 0) \
M(UInt64, backup_restore_batch_size_for_keeper_multi, 1000, "Maximum size of batch for multi request to [Zoo]Keeper during backup or restore", 0) \
M(UInt64, backup_restore_s3_retry_attempts, 1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore.", 0) \
M(UInt64, max_backup_bandwidth, 0, "The maximum read speed in bytes per second for particular backup on server. Zero means unlimited.", 0) \
\
M(Bool, log_profile_events, true, "Log query performance statistics into the query_log, query_thread_log and query_views_log.", 0) \
@ -608,9 +609,8 @@ class IColumn;
M(Bool, optimize_time_filter_with_preimage, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')", 0) \
M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \
M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there are constants there", 0) \
M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \
M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views. Use true to always deduplicate in dependent tables.", 0) \
M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \
M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \
M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \
M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \
M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \
@ -976,6 +976,7 @@ class IColumn;
#define OBSOLETE_SETTINGS(M, ALIAS) \
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
MAKE_OBSOLETE(M, Bool, update_insert_deduplication_token_in_dependent_materialized_views, 0) \
MAKE_OBSOLETE(M, UInt64, max_memory_usage_for_all_queries, 0) \
MAKE_OBSOLETE(M, UInt64, multiple_joins_rewriter_version, 0) \
MAKE_OBSOLETE(M, Bool, enable_debug_queries, false) \

View File

@ -77,6 +77,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"},
{"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"},
{"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."},
{"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."},
{"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."},
{"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}
}},

View File

@ -1306,6 +1306,10 @@ void BaseDaemon::setupWatchdog()
int status = 0;
do
{
// Close log files to prevent keeping descriptors of unlinked rotated files.
// On next log write files will be reopened.
closeLogs(logger());
if (-1 != waitpid(pid, &status, WUNTRACED | WCONTINUED) || errno == ECHILD)
{
if (WIFSTOPPED(status))

View File

@ -39,8 +39,10 @@ namespace ErrorCodes
class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator
{
public:
explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base)
: DatabaseTablesSnapshotIterator(std::move(base)) {}
explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base) noexcept
: DatabaseTablesSnapshotIterator(std::move(base))
{
}
UUID uuid() const override { return table()->getStorageID().uuid; }
};
@ -111,12 +113,12 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String &
// it is important to call the destructors of not_in_use without
// locked mutex to avoid potential deadlock.
DetachedTables not_in_use;
StoragePtr table;
StoragePtr detached_table;
{
std::lock_guard lock(mutex);
table = DatabaseOrdinary::detachTableUnlocked(name);
detached_table = DatabaseOrdinary::detachTableUnlocked(name);
table_name_to_path.erase(name);
detached_tables.emplace(table->getStorageID().uuid, table);
detached_tables.emplace(detached_table->getStorageID().uuid, detached_table);
not_in_use = cleanupDetachedTables();
}
@ -126,7 +128,7 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String &
LOG_DEBUG(log, "Finished removing not used detached tables");
}
return table;
return detached_table;
}
void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool sync)

View File

@ -1,7 +1,8 @@
#pragma once
#include <Databases/DatabasesCommon.h>
#include <Databases/DatabaseOrdinary.h>
#include <Databases/DatabasesCommon.h>
#include <Storages/IStorage_fwd.h>
namespace DB

View File

@ -188,6 +188,13 @@ void DatabaseLazy::attachTable(ContextPtr /* context_ */, const String & table_n
it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name);
LOG_DEBUG(log, "Add info for detached table {} to snapshot.", backQuote(table_name));
if (snapshot_detached_tables.contains(table_name))
{
LOG_DEBUG(log, "Clean info about detached table {} from snapshot.", backQuote(table_name));
snapshot_detached_tables.erase(table_name);
}
CurrentMetrics::add(CurrentMetrics::AttachedTable, 1);
}
@ -204,6 +211,15 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta
if (it->second.expiration_iterator != cache_expiration_queue.end())
cache_expiration_queue.erase(it->second.expiration_iterator);
tables_cache.erase(it);
LOG_DEBUG(log, "Add info for detached table {} to snapshot.", backQuote(table_name));
snapshot_detached_tables.emplace(
table_name,
SnapshotDetachedTable{
.database = res->getStorageID().database_name,
.table = res->getStorageID().table_name,
.uuid = res->getStorageID().uuid,
.metadata_path = getObjectMetadataPath(table_name),
.is_permanently = false});
CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1);
}

View File

@ -23,6 +23,7 @@
#include <Storages/StorageFactory.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/CurrentMetrics.h>
#include <Common/Exception.h>
#include <Common/assert_cast.h>
#include <Common/escapeForFileName.h>
#include <Common/filesystemHelpers.h>
@ -308,6 +309,16 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri
try
{
FS::createFile(detached_permanently_flag);
std::lock_guard lock(mutex);
if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end())
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name);
}
else
{
it->second.is_permanently = true;
}
}
catch (Exception & e)
{

View File

@ -189,7 +189,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
size_t prev_tables_count = metadata.parsed_tables.size();
size_t prev_total_dictionaries = metadata.total_dictionaries;
auto process_metadata = [&metadata, is_startup, this](const String & file_name)
auto process_metadata = [&metadata, is_startup, local_context, this](const String & file_name)
{
fs::path path(getMetadataPath());
fs::path file_path(file_name);
@ -197,7 +197,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
try
{
auto ast = parseQueryFromMetadata(log, getContext(), full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false);
auto ast = parseQueryFromMetadata(log, local_context, full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false);
if (ast)
{
FunctionNameNormalizer::visit(ast.get());
@ -226,8 +226,23 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
if (fs::exists(full_path.string() + detached_suffix))
{
const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4));
permanently_detached_tables.push_back(table_name);
LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name));
std::lock_guard lock(mutex);
permanently_detached_tables.push_back(table_name);
const auto detached_table_name = create_query->getTable();
snapshot_detached_tables.emplace(
detached_table_name,
SnapshotDetachedTable{
.database = create_query->getDatabase(),
.table = detached_table_name,
.uuid = create_query->uuid,
.metadata_path = getObjectMetadataPath(detached_table_name),
.is_permanently = true});
LOG_TRACE(log, "Add permanently detached table {} to system.detached_tables", detached_table_name);
return;
}
@ -489,6 +504,12 @@ DatabaseTablesIteratorPtr DatabaseOrdinary::getTablesIterator(ContextPtr local_c
return DatabaseWithOwnTablesBase::getTablesIterator(local_context, filter_by_table_name, skip_not_loaded);
}
DatabaseDetachedTablesSnapshotIteratorPtr DatabaseOrdinary::getDetachedTablesIterator(
ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const
{
return DatabaseWithOwnTablesBase::getDetachedTablesIterator(local_context, filter_by_table_name, skip_not_loaded);
}
Strings DatabaseOrdinary::getAllTableNames(ContextPtr) const
{
std::set<String> unique_names;

View File

@ -57,6 +57,9 @@ public:
LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override;
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override;
DatabaseDetachedTablesSnapshotIteratorPtr getDetachedTablesIterator(
ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override;
Strings getAllTableNames(ContextPtr context) const override;
void alterTable(
@ -64,7 +67,11 @@ public:
const StorageID & table_id,
const StorageInMemoryMetadata & metadata) override;
Strings getNamesOfPermanentlyDetachedTables() const override { return permanently_detached_tables; }
Strings getNamesOfPermanentlyDetachedTables() const override
{
std::lock_guard lock(mutex);
return permanently_detached_tables;
}
protected:
virtual void commitAlterTable(
@ -74,7 +81,7 @@ protected:
const String & statement,
ContextPtr query_context);
Strings permanently_detached_tables;
Strings permanently_detached_tables TSA_GUARDED_BY(mutex);
std::unordered_map<String, LoadTaskPtr> load_table TSA_GUARDED_BY(mutex);
std::unordered_map<String, LoadTaskPtr> startup_table TSA_GUARDED_BY(mutex);

View File

@ -2,12 +2,9 @@
#include <Backups/BackupEntriesCollector.h>
#include <Backups/RestorerFromBackup.h>
#include <Common/typeid_cast.h>
#include <Common/CurrentMetrics.h>
#include <Common/escapeForFileName.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ParserCreateQuery.h>
@ -16,6 +13,10 @@
#include <Storages/StorageFactory.h>
#include <Storages/Utils.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/CurrentMetrics.h>
#include <Common/escapeForFileName.h>
#include <Common/logger_useful.h>
#include <Common/typeid_cast.h>
namespace DB
@ -237,6 +238,24 @@ DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPt
return std::make_unique<DatabaseTablesSnapshotIterator>(std::move(filtered_tables), database_name);
}
DatabaseDetachedTablesSnapshotIteratorPtr DatabaseWithOwnTablesBase::getDetachedTablesIterator(
ContextPtr, const FilterByNameFunction & filter_by_table_name, bool /* skip_not_loaded */) const
{
std::lock_guard lock(mutex);
if (!filter_by_table_name)
return std::make_unique<DatabaseDetachedTablesSnapshotIterator>(snapshot_detached_tables);
SnapshotDetachedTables filtered_detached_tables;
for (const auto & [detached_table_name, snapshot] : snapshot_detached_tables)
if (filter_by_table_name(detached_table_name))
{
filtered_detached_tables.emplace(detached_table_name, snapshot);
}
return std::make_unique<DatabaseDetachedTablesSnapshotIterator>(std::move(filtered_detached_tables));
}
bool DatabaseWithOwnTablesBase::empty() const
{
std::lock_guard lock(mutex);
@ -251,27 +270,36 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(ContextPtr /* context_ */, con
StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_name)
{
StoragePtr res;
auto it = tables.find(table_name);
if (it == tables.end())
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
backQuote(database_name), backQuote(table_name));
res = it->second;
auto table_storage = it->second;
snapshot_detached_tables.emplace(
table_name,
SnapshotDetachedTable{
.database = it->second->getStorageID().getDatabaseName(),
.table = table_name,
.uuid = it->second->getStorageID().uuid,
.metadata_path = getObjectMetadataPath(table_name),
.is_permanently = false});
tables.erase(it);
res->is_detached = true;
table_storage->is_detached = true;
if (res->isSystemStorage() == false)
CurrentMetrics::sub(getAttachedCounterForStorage(res), 1);
if (table_storage->isSystemStorage() == false)
CurrentMetrics::sub(getAttachedCounterForStorage(table_storage), 1);
auto table_id = res->getStorageID();
auto table_id = table_storage->getStorageID();
if (table_id.hasUUID())
{
assert(database_name == DatabaseCatalog::TEMPORARY_DATABASE || getUUID() != UUIDHelpers::Nil);
DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid);
}
return res;
return table_storage;
}
void DatabaseWithOwnTablesBase::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &)
@ -300,6 +328,8 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", table_id.getFullTableName());
}
snapshot_detached_tables.erase(table_name);
/// It is important to reset is_detached here since in case of RENAME in
/// non-Atomic database the is_detached is set to true before RENAME.
table->is_detached = false;
@ -337,6 +367,7 @@ void DatabaseWithOwnTablesBase::shutdown()
std::lock_guard lock(mutex);
tables.clear();
snapshot_detached_tables.clear();
}
DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase()

View File

@ -37,6 +37,9 @@ public:
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override;
DatabaseDetachedTablesSnapshotIteratorPtr
getDetachedTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override;
std::vector<std::pair<ASTPtr, StoragePtr>> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override;
void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr<IRestoreCoordination> restore_coordination, UInt64 timeout_ms) override;
@ -46,12 +49,13 @@ public:
protected:
Tables tables TSA_GUARDED_BY(mutex);
SnapshotDetachedTables snapshot_detached_tables TSA_GUARDED_BY(mutex);
LoggerPtr log;
DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context);
void attachTableUnlocked(const String & table_name, const StoragePtr & table) TSA_REQUIRES(mutex);
StoragePtr detachTableUnlocked(const String & table_name) TSA_REQUIRES(mutex);
StoragePtr detachTableUnlocked(const String & table_name) TSA_REQUIRES(mutex);
StoragePtr getTableUnlocked(const String & table_name) const TSA_REQUIRES(mutex);
StoragePtr tryGetTableNoWait(const String & table_name) const;
};

View File

@ -5,20 +5,22 @@
#include <Interpreters/Context_fwd.h>
#include <Interpreters/executeQuery.h>
#include <Parsers/IAST_fwd.h>
#include <QueryPipeline/BlockIO.h>
#include <Storages/IStorage.h>
#include <Storages/IStorage_fwd.h>
#include <base/types.h>
#include <Common/Exception.h>
#include <Common/AsyncLoader.h>
#include <Common/Exception.h>
#include <Common/PoolId.h>
#include <Common/ThreadPool_fwd.h>
#include <QueryPipeline/BlockIO.h>
#include <ctime>
#include <functional>
#include <map>
#include <memory>
#include <mutex>
#include <stdexcept>
#include <vector>
#include <map>
namespace DB
@ -110,6 +112,57 @@ public:
using DatabaseTablesIteratorPtr = std::unique_ptr<IDatabaseTablesIterator>;
struct SnapshotDetachedTable final
{
String database;
String table;
UUID uuid = UUIDHelpers::Nil;
String metadata_path;
bool is_permanently{};
};
class DatabaseDetachedTablesSnapshotIterator
{
private:
SnapshotDetachedTables snapshot;
SnapshotDetachedTables::iterator it;
protected:
DatabaseDetachedTablesSnapshotIterator(DatabaseDetachedTablesSnapshotIterator && other) noexcept
{
size_t idx = std::distance(other.snapshot.begin(), other.it);
std::swap(snapshot, other.snapshot);
other.it = other.snapshot.end();
it = snapshot.begin();
std::advance(it, idx);
}
public:
explicit DatabaseDetachedTablesSnapshotIterator(const SnapshotDetachedTables & tables_) : snapshot(tables_), it(snapshot.begin())
{
}
explicit DatabaseDetachedTablesSnapshotIterator(SnapshotDetachedTables && tables_) : snapshot(std::move(tables_)), it(snapshot.begin())
{
}
void next() { ++it; }
bool isValid() const { return it != snapshot.end(); }
String database() const { return it->second.database; }
String table() const { return it->second.table; }
UUID uuid() const { return it->second.uuid; }
String metadataPath() const { return it->second.metadata_path; }
bool isPermanently() const { return it->second.is_permanently; }
};
using DatabaseDetachedTablesSnapshotIteratorPtr = std::unique_ptr<DatabaseDetachedTablesSnapshotIterator>;
/** Database engine.
* It is responsible for:
@ -232,6 +285,12 @@ public:
/// Wait for all tables to be loaded and started up. If `skip_not_loaded` is true, then not yet loaded or not yet started up (at the moment of iterator creation) tables are excluded.
virtual DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name = {}, bool skip_not_loaded = false) const = 0; /// NOLINT
virtual DatabaseDetachedTablesSnapshotIteratorPtr getDetachedTablesIterator(
ContextPtr /*context*/, const FilterByNameFunction & /*filter_by_table_name = {}*/, bool /*skip_not_loaded = false*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get detached tables for Database{}", getEngineName());
}
/// Returns list of table names.
virtual Strings getAllTableNames(ContextPtr context) const
{

View File

@ -1184,7 +1184,7 @@ private:
if (icolumn->size() != vec_to.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument column '{}' size {} doesn't match result column size {} of function {}",
icolumn->getName(), icolumn->size(), vec_to.size(), getName());
icolumn->getName(), icolumn->size(), vec_to.size(), getName());
if constexpr (Keyed)
if (key_cols.size() != vec_to.size() && key_cols.size() != 1)
@ -1223,6 +1223,9 @@ private:
else executeGeneric<first>(key_cols, icolumn, vec_to);
}
/// Return a fixed random-looking magic number when input is empty.
static constexpr auto filler = 0xe28dbde7fe22e41c;
void executeForArgument(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to, bool & is_first) const
{
/// Flattening of tuples.
@ -1231,6 +1234,11 @@ private:
const auto & tuple_columns = tuple->getColumns();
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(*type).getElements();
size_t tuple_size = tuple_columns.size();
if (0 == tuple_size && is_first)
for (auto & hash : vec_to)
hash = static_cast<ToType>(filler);
for (size_t i = 0; i < tuple_size; ++i)
executeForArgument(key_cols, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first);
}
@ -1239,6 +1247,11 @@ private:
const auto & tuple_columns = tuple_const->getColumns();
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(*type).getElements();
size_t tuple_size = tuple_columns.size();
if (0 == tuple_size && is_first)
for (auto & hash : vec_to)
hash = static_cast<ToType>(filler);
for (size_t i = 0; i < tuple_size; ++i)
{
auto tmp = ColumnConst::create(tuple_columns[i], column->size());
@ -1300,10 +1313,7 @@ public:
constexpr size_t first_data_argument = Keyed;
if (arguments.size() <= first_data_argument)
{
/// Return a fixed random-looking magic number when input is empty
vec_to.assign(input_rows_count, static_cast<ToType>(0xe28dbde7fe22e41c));
}
vec_to.assign(input_rows_count, static_cast<ToType>(filler));
KeyColumnsType key_cols{};
if constexpr (Keyed)

View File

@ -230,6 +230,17 @@ public:
virtual bool isDeterministicInScopeOfQuery() const { return true; }
/** This is a special flags for functions which return constant value for the server,
* but the result could be different for different servers in distributed query.
*
* This functions can't support constant folding on the initiator, but can on the follower.
* We can't apply some optimizations as well (e.g. can't remove constant result from GROUP BY key).
* So, it is convenient to have a special flag for them.
*
* Examples are: "__getScalar" and every function from serverConstants.cpp
*/
virtual bool isServerConstant() const { return false; }
/** Lets you know if the function is monotonic in a range of values.
* This is used to work with the index in a sorted chunk of data.
* And allows to use the index not only when it is written, for example `date >= const`, but also, for example, `toMonth(date) >= 11`.
@ -488,6 +499,7 @@ public:
virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const { return false; }
virtual bool isDeterministic() const { return true; }
virtual bool isDeterministicInScopeOfQuery() const { return true; }
virtual bool isServerConstant() const { return false; }
virtual bool isStateful() const { return false; }
using ShortCircuitSettings = IFunctionBase::ShortCircuitSettings;

View File

@ -86,6 +86,8 @@ public:
bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); }
bool isServerConstant() const override { return function->isServerConstant(); }
bool isShortCircuit(ShortCircuitSettings & settings, size_t number_of_arguments) const override { return function->isShortCircuit(settings, number_of_arguments); }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & args) const override { return function->isSuitableForShortCircuitArgumentsExecution(args); }

View File

@ -4,10 +4,10 @@
#include <Functions/FunctionHelpers.h>
#include <Core/ServerUUID.h>
#include <Common/Logger.h>
#include <Common/ErrorCodes.h>
#include <Common/logger_useful.h>
#include "base/types.h"
namespace DB
{
@ -96,10 +96,11 @@ struct SnowflakeIdRange
/// 1. calculate Snowflake ID by current timestamp (`now`)
/// 2. `begin = max(available, now)`
/// 3. Calculate `end = begin + input_rows_count` handling `machine_seq_num` overflow
SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t input_rows_count)
SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, uint64_t machine_id, size_t input_rows_count)
{
/// 1. `now`
SnowflakeId begin = {.timestamp = getTimestamp(), .machine_id = getMachineId(), .machine_seq_num = 0};
SnowflakeId begin = {.timestamp = getTimestamp(), .machine_id = machine_id, .machine_seq_num = 0};
/// 2. `begin`
if (begin.timestamp <= available.timestamp)
@ -128,13 +129,13 @@ struct Data
/// Guarantee counter monotonicity within one timestamp across all threads generating Snowflake IDs simultaneously.
static inline std::atomic<uint64_t> lowest_available_snowflake_id = 0;
SnowflakeId reserveRange(size_t input_rows_count)
SnowflakeId reserveRange(uint64_t machine_id, size_t input_rows_count)
{
uint64_t available_snowflake_id = lowest_available_snowflake_id.load();
SnowflakeIdRange range;
do
{
range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count);
range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), machine_id, input_rows_count);
}
while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, fromSnowflakeId(range.end)));
/// CAS failed --> another thread updated `lowest_available_snowflake_id` and we re-try
@ -165,24 +166,32 @@ public:
{
FunctionArgumentDescriptors mandatory_args;
FunctionArgumentDescriptors optional_args{
{"expr", nullptr, nullptr, "Arbitrary expression"}
{"expr", nullptr, nullptr, "Arbitrary expression"},
{"machine_id", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNativeUInt), static_cast<FunctionArgumentDescriptor::ColumnValidator>(&isColumnConst), "const UInt*"}
};
validateFunctionArguments(*this, arguments, mandatory_args, optional_args);
return std::make_shared<DataTypeUInt64>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & /*arguments*/, const DataTypePtr &, size_t input_rows_count) const override
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
auto col_res = ColumnVector<UInt64>::create();
typename ColumnVector<UInt64>::Container & vec_to = col_res->getData();
if (input_rows_count != 0)
if (input_rows_count > 0)
{
vec_to.resize(input_rows_count);
uint64_t machine_id = getMachineId();
if (arguments.size() == 2)
{
machine_id = arguments[1].column->getUInt(0);
machine_id &= (1ull << machine_id_bits_count) - 1;
}
Data data;
SnowflakeId snowflake_id = data.reserveRange(input_rows_count); /// returns begin of available snowflake ids range
SnowflakeId snowflake_id = data.reserveRange(machine_id, input_rows_count);
for (UInt64 & to_row : vec_to)
{
@ -208,10 +217,13 @@ public:
REGISTER_FUNCTION(GenerateSnowflakeID)
{
FunctionDocumentation::Description description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds (41 + 1 top zero bits), followed by a machine id (10 bits), and a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)";
FunctionDocumentation::Syntax syntax = "generateSnowflakeID([expression])";
FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}};
FunctionDocumentation::Syntax syntax = "generateSnowflakeID([expression, [machine_id]])";
FunctionDocumentation::Arguments arguments = {
{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."},
{"machine_id", "A machine ID, the lowest 10 bits are used. Optional."}
};
FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64";
FunctionDocumentation::Examples examples = {{"single", "SELECT generateSnowflakeID()", "7201148511606784000"}, {"multiple", "SELECT generateSnowflakeID(1), generateSnowflakeID(2)", ""}};
FunctionDocumentation::Examples examples = {{"no_arguments", "SELECT generateSnowflakeID()", "7201148511606784000"}, {"with_machine_id", "SELECT generateSnowflakeID(1)", "7201148511606784001"}, {"with_expression_and_machine_id", "SELECT generateSnowflakeID('some_expression', 1)", "7201148511606784002"}};
FunctionDocumentation::Categories categories = {"Snowflake ID"};
factory.registerFunction<FunctionGenerateSnowflakeID>({description, syntax, arguments, returned_value, examples, categories});

View File

@ -53,6 +53,8 @@ public:
/// getMacro may return different values on different shards/replicas, so it's not constant for distributed query
bool isSuitableForConstantFolding() const override { return !is_distributed; }
bool isServerConstant() const override { return true; }
size_t getNumberOfArguments() const override
{
return 1;

View File

@ -49,6 +49,8 @@ public:
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
bool isServerConstant() const override { return true; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 1 || !isString(arguments[0].type) || !arguments[0].column || !isColumnConst(*arguments[0].column))
@ -105,6 +107,8 @@ public:
bool isDeterministic() const override { return false; }
bool isServerConstant() const override { return true; }
bool isSuitableForConstantFolding() const override { return !is_distributed; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }

View File

@ -21,117 +21,125 @@ namespace DB
namespace
{
template<typename Derived, typename T, typename ColumnT>
class FunctionServerConstantBase : public FunctionConstantBase<Derived, T, ColumnT>
{
public:
using FunctionConstantBase<Derived, T, ColumnT>::FunctionConstantBase;
bool isServerConstant() const override { return true; }
};
#if defined(__ELF__) && !defined(OS_FREEBSD)
/// buildId() - returns the compiler build id of the running binary.
class FunctionBuildId : public FunctionConstantBase<FunctionBuildId, String, DataTypeString>
class FunctionBuildId : public FunctionServerConstantBase<FunctionBuildId, String, DataTypeString>
{
public:
static constexpr auto name = "buildId";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionBuildId>(context); }
explicit FunctionBuildId(ContextPtr context) : FunctionConstantBase(SymbolIndex::instance().getBuildIDHex(), context->isDistributed()) {}
explicit FunctionBuildId(ContextPtr context) : FunctionServerConstantBase(SymbolIndex::instance().getBuildIDHex(), context->isDistributed()) {}
};
#endif
/// Get the host name. It is constant on single server, but is not constant in distributed queries.
class FunctionHostName : public FunctionConstantBase<FunctionHostName, String, DataTypeString>
class FunctionHostName : public FunctionServerConstantBase<FunctionHostName, String, DataTypeString>
{
public:
static constexpr auto name = "hostName";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionHostName>(context); }
explicit FunctionHostName(ContextPtr context) : FunctionConstantBase(DNSResolver::instance().getHostName(), context->isDistributed()) {}
explicit FunctionHostName(ContextPtr context) : FunctionServerConstantBase(DNSResolver::instance().getHostName(), context->isDistributed()) {}
};
class FunctionServerUUID : public FunctionConstantBase<FunctionServerUUID, UUID, DataTypeUUID>
class FunctionServerUUID : public FunctionServerConstantBase<FunctionServerUUID, UUID, DataTypeUUID>
{
public:
static constexpr auto name = "serverUUID";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionServerUUID>(context); }
explicit FunctionServerUUID(ContextPtr context) : FunctionConstantBase(ServerUUID::get(), context->isDistributed()) {}
explicit FunctionServerUUID(ContextPtr context) : FunctionServerConstantBase(ServerUUID::get(), context->isDistributed()) {}
};
class FunctionTCPPort : public FunctionConstantBase<FunctionTCPPort, UInt16, DataTypeUInt16>
class FunctionTCPPort : public FunctionServerConstantBase<FunctionTCPPort, UInt16, DataTypeUInt16>
{
public:
static constexpr auto name = "tcpPort";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionTCPPort>(context); }
explicit FunctionTCPPort(ContextPtr context) : FunctionConstantBase(context->getTCPPort(), context->isDistributed()) {}
explicit FunctionTCPPort(ContextPtr context) : FunctionServerConstantBase(context->getTCPPort(), context->isDistributed()) {}
};
/// Returns timezone for current session.
class FunctionTimezone : public FunctionConstantBase<FunctionTimezone, String, DataTypeString>
class FunctionTimezone : public FunctionServerConstantBase<FunctionTimezone, String, DataTypeString>
{
public:
static constexpr auto name = "timezone";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionTimezone>(context); }
explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(DateLUT::instance().getTimeZone(), context->isDistributed()) {}
explicit FunctionTimezone(ContextPtr context) : FunctionServerConstantBase(DateLUT::instance().getTimeZone(), context->isDistributed()) {}
};
/// Returns the server time zone (timezone in which server runs).
class FunctionServerTimezone : public FunctionConstantBase<FunctionServerTimezone, String, DataTypeString>
class FunctionServerTimezone : public FunctionServerConstantBase<FunctionServerTimezone, String, DataTypeString>
{
public:
static constexpr auto name = "serverTimezone";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionServerTimezone>(context); }
explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(DateLUT::serverTimezoneInstance().getTimeZone(), context->isDistributed()) {}
explicit FunctionServerTimezone(ContextPtr context) : FunctionServerConstantBase(DateLUT::serverTimezoneInstance().getTimeZone(), context->isDistributed()) {}
};
/// Returns server uptime in seconds.
class FunctionUptime : public FunctionConstantBase<FunctionUptime, UInt32, DataTypeUInt32>
class FunctionUptime : public FunctionServerConstantBase<FunctionUptime, UInt32, DataTypeUInt32>
{
public:
static constexpr auto name = "uptime";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionUptime>(context); }
explicit FunctionUptime(ContextPtr context) : FunctionConstantBase(context->getUptimeSeconds(), context->isDistributed()) {}
explicit FunctionUptime(ContextPtr context) : FunctionServerConstantBase(context->getUptimeSeconds(), context->isDistributed()) {}
};
/// version() - returns the current version as a string.
class FunctionVersion : public FunctionConstantBase<FunctionVersion, String, DataTypeString>
class FunctionVersion : public FunctionServerConstantBase<FunctionVersion, String, DataTypeString>
{
public:
static constexpr auto name = "version";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionVersion>(context); }
explicit FunctionVersion(ContextPtr context) : FunctionConstantBase(VERSION_STRING, context->isDistributed()) {}
explicit FunctionVersion(ContextPtr context) : FunctionServerConstantBase(VERSION_STRING, context->isDistributed()) {}
};
/// revision() - returns the current revision.
class FunctionRevision : public FunctionConstantBase<FunctionRevision, UInt32, DataTypeUInt32>
class FunctionRevision : public FunctionServerConstantBase<FunctionRevision, UInt32, DataTypeUInt32>
{
public:
static constexpr auto name = "revision";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionRevision>(context); }
explicit FunctionRevision(ContextPtr context) : FunctionConstantBase(ClickHouseRevision::getVersionRevision(), context->isDistributed()) {}
explicit FunctionRevision(ContextPtr context) : FunctionServerConstantBase(ClickHouseRevision::getVersionRevision(), context->isDistributed()) {}
};
class FunctionZooKeeperSessionUptime : public FunctionConstantBase<FunctionZooKeeperSessionUptime, UInt32, DataTypeUInt32>
class FunctionZooKeeperSessionUptime : public FunctionServerConstantBase<FunctionZooKeeperSessionUptime, UInt32, DataTypeUInt32>
{
public:
static constexpr auto name = "zookeeperSessionUptime";
explicit FunctionZooKeeperSessionUptime(ContextPtr context)
: FunctionConstantBase(context->getZooKeeperSessionUptime(), context->isDistributed())
: FunctionServerConstantBase(context->getZooKeeperSessionUptime(), context->isDistributed())
{
}
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionZooKeeperSessionUptime>(context); }
};
class FunctionGetOSKernelVersion : public FunctionConstantBase<FunctionGetOSKernelVersion, String, DataTypeString>
class FunctionGetOSKernelVersion : public FunctionServerConstantBase<FunctionGetOSKernelVersion, String, DataTypeString>
{
public:
static constexpr auto name = "getOSKernelVersion";
explicit FunctionGetOSKernelVersion(ContextPtr context) : FunctionConstantBase(Poco::Environment::osName() + " " + Poco::Environment::osVersion(), context->isDistributed()) {}
explicit FunctionGetOSKernelVersion(ContextPtr context) : FunctionServerConstantBase(Poco::Environment::osName() + " " + Poco::Environment::osVersion(), context->isDistributed()) {}
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionGetOSKernelVersion>(context); }
};
class FunctionDisplayName : public FunctionConstantBase<FunctionDisplayName, String, DataTypeString>
class FunctionDisplayName : public FunctionServerConstantBase<FunctionDisplayName, String, DataTypeString>
{
public:
static constexpr auto name = "displayName";
explicit FunctionDisplayName(ContextPtr context) : FunctionConstantBase(context->getConfigRef().getString("display_name", getFQDNOrHostName()), context->isDistributed()) {}
explicit FunctionDisplayName(ContextPtr context) : FunctionServerConstantBase(context->getConfigRef().getString("display_name", getFQDNOrHostName()), context->isDistributed()) {}
static FunctionPtr create(ContextPtr context) {return std::make_shared<FunctionDisplayName>(context); }
};
}

View File

@ -162,7 +162,7 @@ public:
class RetryStrategy : public Aws::Client::RetryStrategy
{
public:
explicit RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 90000);
explicit RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 5000);
/// NOLINTNEXTLINE(google-runtime-int)
bool ShouldRetry(const Aws::Client::AWSError<Aws::Client::CoreErrors>& error, long attemptedRetries) const override;

View File

@ -114,6 +114,34 @@ namespace
else if (query.grantees)
user.grantees = *query.grantees;
}
time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context)
{
if (context)
valid_until = evaluateConstantExpressionAsLiteral(valid_until, context);
const String valid_until_str = checkAndGetLiteralArgument<String>(valid_until, "valid_until");
if (valid_until_str == "infinity")
return 0;
time_t time = 0;
ReadBufferFromString in(valid_until_str);
if (context)
{
const auto & time_zone = DateLUT::instance("");
const auto & utc_time_zone = DateLUT::instance("UTC");
parseDateTimeBestEffort(time, in, time_zone, utc_time_zone);
}
else
{
readDateTimeText(time, in);
}
return time;
}
}
BlockIO InterpreterCreateUserQuery::execute()
@ -134,23 +162,7 @@ BlockIO InterpreterCreateUserQuery::execute()
std::optional<time_t> valid_until;
if (query.valid_until)
{
const ASTPtr valid_until_literal = evaluateConstantExpressionAsLiteral(query.valid_until, getContext());
const String valid_until_str = checkAndGetLiteralArgument<String>(valid_until_literal, "valid_until");
time_t time = 0;
if (valid_until_str != "infinity")
{
const auto & time_zone = DateLUT::instance("");
const auto & utc_time_zone = DateLUT::instance("UTC");
ReadBufferFromString in(valid_until_str);
parseDateTimeBestEffort(time, in, time_zone, utc_time_zone);
}
valid_until = time;
}
valid_until = getValidUntilFromAST(query.valid_until, getContext());
std::optional<RolesOrUsersSet> default_roles_from_query;
if (query.default_roles)
@ -259,7 +271,11 @@ void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreat
if (query.auth_data)
auth_data = AuthenticationData::fromAST(*query.auth_data, {}, !query.attach);
updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, {}, allow_no_password, allow_plaintext_password, true);
std::optional<time_t> valid_until;
if (query.valid_until)
valid_until = getValidUntilFromAST(query.valid_until, {});
updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, valid_until, allow_no_password, allow_plaintext_password, true);
}
void registerInterpreterCreateUserQuery(InterpreterFactory & factory)

View File

@ -301,7 +301,13 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const
auto & insert_query = query->as<ASTInsertQuery &>();
insert_query.async_insert_flush = true;
InterpreterInsertQuery interpreter(query, query_context, query_context->getSettingsRef().insert_allow_materialized_columns);
InterpreterInsertQuery interpreter(
query,
query_context,
query_context->getSettingsRef().insert_allow_materialized_columns,
/* no_squash */ false,
/* no_destination */ false,
/* async_insert */ false);
auto table = interpreter.getTable(insert_query);
auto sample_block = InterpreterInsertQuery::getSampleBlock(insert_query, table, table->getInMemoryMetadataPtr(), query_context);
@ -784,7 +790,12 @@ try
try
{
interpreter = std::make_unique<InterpreterInsertQuery>(
key.query, insert_context, key.settings.insert_allow_materialized_columns, false, false, true);
key.query,
insert_context,
key.settings.insert_allow_materialized_columns,
false,
false,
true);
pipeline = interpreter->execute().pipeline;
chassert(pipeline.pushing());
@ -1003,7 +1014,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing(
}
Chunk chunk(executor.getResultColumns(), total_rows);
chunk.setChunkInfo(std::move(chunk_info));
chunk.getChunkInfos().add(std::move(chunk_info));
return chunk;
}
@ -1055,7 +1066,7 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries(
}
Chunk chunk(std::move(result_columns), total_rows);
chunk.setChunkInfo(std::move(chunk_info));
chunk.getChunkInfos().add(std::move(chunk_info));
return chunk;
}

View File

@ -95,7 +95,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const
if (server_protocol_revision >= DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS)
{
writeVarUInt(static_cast<UInt64>(collaborate_with_initiator), out);
writeVarUInt(count_participating_replicas, out);
writeVarUInt(obsolete_count_participating_replicas, out);
writeVarUInt(number_of_current_replica, out);
}
}
@ -185,7 +185,7 @@ void ClientInfo::read(ReadBuffer & in, UInt64 client_protocol_revision)
UInt64 value;
readVarUInt(value, in);
collaborate_with_initiator = static_cast<bool>(value);
readVarUInt(count_participating_replicas, in);
readVarUInt(obsolete_count_participating_replicas, in);
readVarUInt(number_of_current_replica, in);
}
}

View File

@ -127,7 +127,7 @@ public:
/// For parallel processing on replicas
bool collaborate_with_initiator{false};
UInt64 count_participating_replicas{0};
UInt64 obsolete_count_participating_replicas{0};
UInt64 number_of_current_replica{0};
enum class BackgroundOperationType : uint8_t

View File

@ -51,7 +51,6 @@
#include <Interpreters/SessionTracker.h>
#include <Core/ServerSettings.h>
#include <Interpreters/PreparedSets.h>
#include <Core/Settings.h>
#include <Core/SettingsQuirks.h>
#include <Access/AccessControl.h>
#include <Access/ContextAccess.h>
@ -5002,13 +5001,6 @@ void Context::setConnectionClientVersion(UInt64 client_version_major, UInt64 cli
client_info.connection_tcp_protocol_version = client_tcp_protocol_version;
}
void Context::setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica)
{
client_info.collaborate_with_initiator = collaborate_with_initiator;
client_info.count_participating_replicas = all_replicas_count;
client_info.number_of_current_replica = number_of_current_replica;
}
void Context::increaseDistributedDepth()
{
++client_info.distributed_depth;

View File

@ -699,7 +699,6 @@ public:
void setInitialQueryStartTime(std::chrono::time_point<std::chrono::system_clock> initial_query_start_time);
void setQuotaClientKey(const String & quota_key);
void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version);
void setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica);
void increaseDistributedDepth();
const OpenTelemetry::TracingContext & getClientTraceContext() const { return client_info.client_trace_context; }
OpenTelemetry::TracingContext & getClientTraceContext() { return client_info.client_trace_context; }

View File

@ -125,7 +125,7 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
if (isCrossOrComma(kind))
{
data->type = Type::CROSS;
sample_block_with_columns_to_add = right_sample_block;
sample_block_with_columns_to_add = materializeBlock(right_sample_block);
}
else if (table_join->getClauses().empty())
{

View File

@ -2,6 +2,7 @@
#include <Interpreters/InterpreterFactory.h>
#include <algorithm>
#include <memory>
#include <Access/Common/AccessFlags.h>
@ -24,6 +25,7 @@
#include <Parsers/ASTCheckQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Processors/Chunk.h>
#include <Processors/IAccumulatingTransform.h>
#include <Processors/IInflatingTransform.h>
#include <Processors/ISimpleTransform.h>
@ -93,7 +95,7 @@ Chunk getChunkFromCheckResult(const String & database, const String & table, con
return Chunk(std::move(columns), 1);
}
class TableCheckTask : public ChunkInfo
class TableCheckTask : public ChunkInfoCloneable<TableCheckTask>
{
public:
TableCheckTask(StorageID table_id, const std::variant<std::monostate, ASTPtr, String> & partition_or_part, ContextPtr context)
@ -112,6 +114,12 @@ public:
context->checkAccess(AccessType::SHOW_TABLES, table_->getStorageID());
}
TableCheckTask(const TableCheckTask & other)
: table(other.table)
, check_data_tasks(other.check_data_tasks)
, is_finished(other.is_finished.load())
{}
std::optional<CheckResult> checkNext() const
{
if (isFinished())
@ -123,8 +131,8 @@ public:
std::this_thread::sleep_for(sleep_time);
});
IStorage::DataValidationTasksPtr check_data_tasks_ = check_data_tasks;
auto result = table->checkDataNext(check_data_tasks_);
IStorage::DataValidationTasksPtr tmp = check_data_tasks;
auto result = table->checkDataNext(tmp);
is_finished = !result.has_value();
return result;
}
@ -182,7 +190,7 @@ protected:
/// source should return at least one row to start pipeline
result.addColumn(ColumnUInt8::create(1, 1));
/// actual data stored in chunk info
result.setChunkInfo(std::move(current_check_task));
result.getChunkInfos().add(std::move(current_check_task));
return result;
}
@ -282,7 +290,7 @@ public:
protected:
void transform(Chunk & chunk) override
{
auto table_check_task = std::dynamic_pointer_cast<const TableCheckTask>(chunk.getChunkInfo());
auto table_check_task = chunk.getChunkInfos().get<TableCheckTask>();
auto check_result = table_check_task->checkNext();
if (!check_result)
{

View File

@ -1777,8 +1777,13 @@ BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create)
else
insert->select = create.select->clone();
return InterpreterInsertQuery(insert, getContext(),
getContext()->getSettingsRef().insert_allow_materialized_columns).execute();
return InterpreterInsertQuery(
insert,
getContext(),
getContext()->getSettingsRef().insert_allow_materialized_columns,
/* no_squash */ false,
/* no_destination */ false,
/* async_isnert */ false).execute();
}
return {};

View File

@ -535,7 +535,13 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
}
else if (dynamic_cast<const ASTInsertQuery *>(ast.getExplainedQuery().get()))
{
InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext());
InterpreterInsertQuery insert(
ast.getExplainedQuery(),
getContext(),
/* allow_materialized */ false,
/* no_squash */ false,
/* no_destination */ false,
/* async_isnert */ false);
auto io = insert.execute();
printPipeline(io.pipeline.getProcessors(), buf);
}

View File

@ -17,6 +17,7 @@
#include <Interpreters/getTableExpressions.h>
#include <Interpreters/processColumnTransformers.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Interpreters/Context_fwd.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTSelectQuery.h>
@ -27,6 +28,7 @@
#include <Processors/Transforms/CountingTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/Transforms/DeduplicationTokenTransforms.h>
#include <Processors/Transforms/SquashingTransform.h>
#include <Processors/Transforms/PlanSquashingTransform.h>
#include <Processors/Transforms/getSourceFromASTInsertQuery.h>
@ -36,9 +38,11 @@
#include <Storages/StorageMaterializedView.h>
#include <Storages/WindowView/StorageWindowView.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/logger_useful.h>
#include <Common/ThreadStatus.h>
#include <Common/checkStackSize.h>
#include <Common/ProfileEvents.h>
#include "base/defines.h"
namespace ProfileEvents
@ -395,28 +399,349 @@ Chain InterpreterInsertQuery::buildPreSinkChain(
return out;
}
std::pair<std::vector<Chain>, std::vector<Chain>> InterpreterInsertQuery::buildPreAndSinkChains(size_t presink_streams, size_t sink_streams, StoragePtr table, const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block)
{
chassert(presink_streams > 0);
chassert(sink_streams > 0);
ThreadGroupPtr running_group;
if (current_thread)
running_group = current_thread->getThreadGroup();
if (!running_group)
running_group = std::make_shared<ThreadGroup>(getContext());
std::vector<Chain> sink_chains;
std::vector<Chain> presink_chains;
for (size_t i = 0; i < sink_streams; ++i)
{
auto out = buildSink(table, metadata_snapshot, /* thread_status_holder= */ nullptr,
running_group, /* elapsed_counter_ms= */ nullptr);
sink_chains.emplace_back(std::move(out));
}
for (size_t i = 0; i < presink_streams; ++i)
{
auto out = buildPreSinkChain(sink_chains[0].getInputHeader(), table, metadata_snapshot, query_sample_block);
presink_chains.emplace_back(std::move(out));
}
return {std::move(presink_chains), std::move(sink_chains)};
}
QueryPipeline InterpreterInsertQuery::buildInsertSelectPipeline(ASTInsertQuery & query, StoragePtr table)
{
const Settings & settings = getContext()->getSettingsRef();
auto metadata_snapshot = table->getInMemoryMetadataPtr();
auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized);
bool is_trivial_insert_select = false;
if (settings.optimize_trivial_insert_select)
{
const auto & select_query = query.select->as<ASTSelectWithUnionQuery &>();
const auto & selects = select_query.list_of_selects->children;
const auto & union_modes = select_query.list_of_modes;
/// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries
const auto mode_is_all = [](const auto & mode) { return mode == SelectUnionMode::UNION_ALL; };
is_trivial_insert_select =
std::all_of(union_modes.begin(), union_modes.end(), std::move(mode_is_all))
&& std::all_of(selects.begin(), selects.end(), isTrivialSelect);
}
ContextPtr select_context = getContext();
if (is_trivial_insert_select)
{
/** When doing trivial INSERT INTO ... SELECT ... FROM table,
* don't need to process SELECT with more than max_insert_threads
* and it's reasonable to set block size for SELECT to the desired block size for INSERT
* to avoid unnecessary squashing.
*/
Settings new_settings = select_context->getSettings();
new_settings.max_threads = std::max<UInt64>(1, settings.max_insert_threads);
if (table->prefersLargeBlocks())
{
if (settings.min_insert_block_size_rows)
new_settings.max_block_size = settings.min_insert_block_size_rows;
if (settings.min_insert_block_size_bytes)
new_settings.preferred_block_size_bytes = settings.min_insert_block_size_bytes;
}
auto context_for_trivial_select = Context::createCopy(context);
context_for_trivial_select->setSettings(new_settings);
context_for_trivial_select->setInsertionTable(getContext()->getInsertionTable(), getContext()->getInsertionTableColumnNames());
select_context = context_for_trivial_select;
}
QueryPipelineBuilder pipeline;
{
auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1);
if (settings.allow_experimental_analyzer)
{
InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, select_context, select_query_options);
pipeline = interpreter_select_analyzer.buildQueryPipeline();
}
else
{
InterpreterSelectWithUnionQuery interpreter_select(query.select, select_context, select_query_options);
pipeline = interpreter_select.buildQueryPipeline();
}
}
pipeline.dropTotalsAndExtremes();
/// Allow to insert Nullable into non-Nullable columns, NULL values will be added as defaults values.
if (getContext()->getSettingsRef().insert_null_as_default)
{
const auto & input_columns = pipeline.getHeader().getColumnsWithTypeAndName();
const auto & query_columns = query_sample_block.getColumnsWithTypeAndName();
const auto & output_columns = metadata_snapshot->getColumns();
if (input_columns.size() == query_columns.size())
{
for (size_t col_idx = 0; col_idx < query_columns.size(); ++col_idx)
{
/// Change query sample block columns to Nullable to allow inserting nullable columns, where NULL values will be substituted with
/// default column values (in AddingDefaultsTransform), so all values will be cast correctly.
if (isNullableOrLowCardinalityNullable(input_columns[col_idx].type)
&& !isNullableOrLowCardinalityNullable(query_columns[col_idx].type)
&& !isVariant(query_columns[col_idx].type)
&& !isDynamic(query_columns[col_idx].type)
&& output_columns.has(query_columns[col_idx].name))
{
query_sample_block.setColumn(
col_idx,
ColumnWithTypeAndName(
makeNullableOrLowCardinalityNullable(query_columns[col_idx].column),
makeNullableOrLowCardinalityNullable(query_columns[col_idx].type),
query_columns[col_idx].name));
}
}
}
}
auto actions_dag = ActionsDAG::makeConvertingActions(
pipeline.getHeader().getColumnsWithTypeAndName(),
query_sample_block.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Position);
auto actions = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes));
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{
return std::make_shared<ExpressionTransform>(in_header, actions);
});
/// We need to convert Sparse columns to full, because it's destination storage
/// may not support it or may have different settings for applying Sparse serialization.
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{
return std::make_shared<MaterializingTransform>(in_header);
});
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{
auto context_ptr = getContext();
auto counting = std::make_shared<CountingTransform>(in_header, nullptr, context_ptr->getQuota());
counting->setProcessListElement(context_ptr->getProcessListElement());
counting->setProgressCallback(context_ptr->getProgressCallback());
return counting;
});
size_t num_select_threads = pipeline.getNumThreads();
pipeline.resize(1);
if (shouldAddSquashingFroStorage(table))
{
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{
return std::make_shared<PlanSquashingTransform>(
in_header,
table->prefersLargeBlocks() ? settings.min_insert_block_size_rows : settings.max_block_size,
table->prefersLargeBlocks() ? settings.min_insert_block_size_bytes : 0ULL);
});
}
pipeline.addSimpleTransform([&](const Block &in_header) -> ProcessorPtr
{
return std::make_shared<DeduplicationToken::AddTokenInfoTransform>(in_header);
});
if (!settings.insert_deduplication_token.value.empty())
{
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{
return std::make_shared<DeduplicationToken::SetUserTokenTransform>(settings.insert_deduplication_token.value, in_header);
});
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{
return std::make_shared<DeduplicationToken::SetSourceBlockNumberTransform>(in_header);
});
}
/// Number of streams works like this:
/// * For the SELECT, use `max_threads`, or `max_insert_threads`, or whatever
/// InterpreterSelectQuery ends up with.
/// * Use `max_insert_threads` streams for various insert-preparation steps, e.g.
/// materializing and squashing (too slow to do in one thread). That's `presink_chains`.
/// * If the table supports parallel inserts, use max_insert_threads for writing to IStorage.
/// Otherwise ResizeProcessor them down to 1 stream.
size_t presink_streams_size = std::max<size_t>(settings.max_insert_threads, pipeline.getNumStreams());
if (settings.max_insert_threads.changed)
presink_streams_size = std::max<size_t>(1, settings.max_insert_threads);
size_t sink_streams_size = table->supportsParallelInsert() ? std::max<size_t>(1, settings.max_insert_threads) : 1;
size_t views_involved = table->isView() || !DatabaseCatalog::instance().getDependentViews(table->getStorageID()).empty();
if (!settings.parallel_view_processing && views_involved)
{
sink_streams_size = 1;
}
auto [presink_chains, sink_chains] = buildPreAndSinkChains(
presink_streams_size, sink_streams_size,
table, metadata_snapshot, query_sample_block);
pipeline.resize(presink_chains.size());
if (shouldAddSquashingFroStorage(table))
{
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{
return std::make_shared<ApplySquashingTransform>(
in_header,
table->prefersLargeBlocks() ? settings.min_insert_block_size_rows : settings.max_block_size,
table->prefersLargeBlocks() ? settings.min_insert_block_size_bytes : 0ULL);
});
}
for (auto & chain : presink_chains)
pipeline.addResources(chain.detachResources());
pipeline.addChains(std::move(presink_chains));
pipeline.resize(sink_streams_size);
for (auto & chain : sink_chains)
pipeline.addResources(chain.detachResources());
pipeline.addChains(std::move(sink_chains));
if (!settings.parallel_view_processing && views_involved)
{
/// Don't use more threads for INSERT than for SELECT to reduce memory consumption.
if (pipeline.getNumThreads() > num_select_threads)
pipeline.setMaxThreads(num_select_threads);
}
pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr
{
return std::make_shared<EmptySink>(cur_header);
});
return QueryPipelineBuilder::getPipeline(std::move(pipeline));
}
QueryPipeline InterpreterInsertQuery::buildInsertPipeline(ASTInsertQuery & query, StoragePtr table)
{
const Settings & settings = getContext()->getSettingsRef();
auto metadata_snapshot = table->getInMemoryMetadataPtr();
auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized);
Chain chain;
{
auto [presink_chains, sink_chains] = buildPreAndSinkChains(
/* presink_streams */1, /* sink_streams */1,
table, metadata_snapshot, query_sample_block);
chain = std::move(presink_chains.front());
chain.appendChain(std::move(sink_chains.front()));
}
if (!settings.insert_deduplication_token.value.empty())
{
chain.addSource(std::make_shared<DeduplicationToken::SetSourceBlockNumberTransform>(chain.getInputHeader()));
chain.addSource(std::make_shared<DeduplicationToken::SetUserTokenTransform>(settings.insert_deduplication_token.value, chain.getInputHeader()));
}
chain.addSource(std::make_shared<DeduplicationToken::AddTokenInfoTransform>(chain.getInputHeader()));
if (shouldAddSquashingFroStorage(table))
{
bool table_prefers_large_blocks = table->prefersLargeBlocks();
auto squashing = std::make_shared<ApplySquashingTransform>(
chain.getInputHeader(),
table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size,
table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL);
chain.addSource(std::move(squashing));
auto balancing = std::make_shared<PlanSquashingTransform>(
chain.getInputHeader(),
table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size,
table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL);
chain.addSource(std::move(balancing));
}
auto context_ptr = getContext();
auto counting = std::make_shared<CountingTransform>(chain.getInputHeader(), nullptr, context_ptr->getQuota());
counting->setProcessListElement(context_ptr->getProcessListElement());
counting->setProgressCallback(context_ptr->getProgressCallback());
chain.addSource(std::move(counting));
QueryPipeline pipeline = QueryPipeline(std::move(chain));
pipeline.setNumThreads(std::min<size_t>(pipeline.getNumThreads(), settings.max_threads));
pipeline.setConcurrencyControl(settings.use_concurrency_control);
if (query.hasInlinedData() && !async_insert)
{
/// can execute without additional data
auto format = getInputFormatFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr);
for (auto && buffer : owned_buffers)
format->addBuffer(std::move(buffer));
auto pipe = getSourceFromInputFormat(query_ptr, std::move(format), getContext(), nullptr);
pipeline.complete(std::move(pipe));
}
return pipeline;
}
BlockIO InterpreterInsertQuery::execute()
{
const Settings & settings = getContext()->getSettingsRef();
auto & query = query_ptr->as<ASTInsertQuery &>();
QueryPipelineBuilder pipeline;
std::optional<QueryPipeline> distributed_pipeline;
QueryPlanResourceHolder resources;
StoragePtr table = getTable(query);
checkStorageSupportsTransactionsIfNeeded(table, getContext());
StoragePtr inner_table;
if (const auto * mv = dynamic_cast<const StorageMaterializedView *>(table.get()))
inner_table = mv->getTargetTable();
if (query.partition_by && !table->supportsPartitionBy())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "PARTITION BY clause is not supported by storage");
auto table_lock = table->lockForShare(getContext()->getInitialQueryId(), settings.lock_acquire_timeout);
auto metadata_snapshot = table->getInMemoryMetadataPtr();
auto metadata_snapshot = table->getInMemoryMetadataPtr();
auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized);
/// For table functions we check access while executing
@ -424,320 +749,45 @@ BlockIO InterpreterInsertQuery::execute()
if (!query.table_function)
getContext()->checkAccess(AccessType::INSERT, query.table_id, query_sample_block.getNames());
if (query.select && settings.parallel_distributed_insert_select)
// Distributed INSERT SELECT
distributed_pipeline = table->distributedWrite(query, getContext());
std::vector<Chain> presink_chains;
std::vector<Chain> sink_chains;
if (!distributed_pipeline)
if (!allow_materialized)
{
/// Number of streams works like this:
/// * For the SELECT, use `max_threads`, or `max_insert_threads`, or whatever
/// InterpreterSelectQuery ends up with.
/// * Use `max_insert_threads` streams for various insert-preparation steps, e.g.
/// materializing and squashing (too slow to do in one thread). That's `presink_chains`.
/// * If the table supports parallel inserts, use the same streams for writing to IStorage.
/// Otherwise ResizeProcessor them down to 1 stream.
/// * If it's not an INSERT SELECT, forget all that and use one stream.
size_t pre_streams_size = 1;
size_t sink_streams_size = 1;
if (query.select)
{
bool is_trivial_insert_select = false;
if (settings.optimize_trivial_insert_select)
{
const auto & select_query = query.select->as<ASTSelectWithUnionQuery &>();
const auto & selects = select_query.list_of_selects->children;
const auto & union_modes = select_query.list_of_modes;
/// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries
const auto mode_is_all = [](const auto & mode) { return mode == SelectUnionMode::UNION_ALL; };
is_trivial_insert_select =
std::all_of(union_modes.begin(), union_modes.end(), std::move(mode_is_all))
&& std::all_of(selects.begin(), selects.end(), isTrivialSelect);
}
if (is_trivial_insert_select)
{
/** When doing trivial INSERT INTO ... SELECT ... FROM table,
* don't need to process SELECT with more than max_insert_threads
* and it's reasonable to set block size for SELECT to the desired block size for INSERT
* to avoid unnecessary squashing.
*/
Settings new_settings = getContext()->getSettings();
new_settings.max_threads = std::max<UInt64>(1, settings.max_insert_threads);
if (table->prefersLargeBlocks())
{
if (settings.min_insert_block_size_rows)
new_settings.max_block_size = settings.min_insert_block_size_rows;
if (settings.min_insert_block_size_bytes)
new_settings.preferred_block_size_bytes = settings.min_insert_block_size_bytes;
}
auto new_context = Context::createCopy(context);
new_context->setSettings(new_settings);
new_context->setInsertionTable(getContext()->getInsertionTable(), getContext()->getInsertionTableColumnNames());
auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1);
if (settings.allow_experimental_analyzer)
{
InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, new_context, select_query_options);
pipeline = interpreter_select_analyzer.buildQueryPipeline();
}
else
{
InterpreterSelectWithUnionQuery interpreter_select(query.select, new_context, select_query_options);
pipeline = interpreter_select.buildQueryPipeline();
}
}
else
{
/// Passing 1 as subquery_depth will disable limiting size of intermediate result.
auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1);
if (settings.allow_experimental_analyzer)
{
InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, getContext(), select_query_options);
pipeline = interpreter_select_analyzer.buildQueryPipeline();
}
else
{
InterpreterSelectWithUnionQuery interpreter_select(query.select, getContext(), select_query_options);
pipeline = interpreter_select.buildQueryPipeline();
}
}
pipeline.dropTotalsAndExtremes();
if (settings.max_insert_threads > 1)
{
auto table_id = table->getStorageID();
auto views = DatabaseCatalog::instance().getDependentViews(table_id);
/// It breaks some views-related tests and we have dedicated `parallel_view_processing` for views, so let's just skip them.
/// Also it doesn't make sense to reshuffle data if storage doesn't support parallel inserts.
const bool resize_to_max_insert_threads = !table->isView() && views.empty() && table->supportsParallelInsert();
pre_streams_size = resize_to_max_insert_threads ? settings.max_insert_threads
: std::min<size_t>(settings.max_insert_threads, pipeline.getNumStreams());
/// Deduplication when passing insert_deduplication_token breaks if using more than one thread
if (!settings.insert_deduplication_token.toString().empty())
{
LOG_DEBUG(
getLogger("InsertQuery"),
"Insert-select query using insert_deduplication_token, setting streams to 1 to avoid deduplication issues");
pre_streams_size = 1;
}
if (table->supportsParallelInsert())
sink_streams_size = pre_streams_size;
}
pipeline.resize(pre_streams_size);
/// Allow to insert Nullable into non-Nullable columns, NULL values will be added as defaults values.
if (getContext()->getSettingsRef().insert_null_as_default)
{
const auto & input_columns = pipeline.getHeader().getColumnsWithTypeAndName();
const auto & query_columns = query_sample_block.getColumnsWithTypeAndName();
const auto & output_columns = metadata_snapshot->getColumns();
if (input_columns.size() == query_columns.size())
{
for (size_t col_idx = 0; col_idx < query_columns.size(); ++col_idx)
{
/// Change query sample block columns to Nullable to allow inserting nullable columns, where NULL values will be substituted with
/// default column values (in AddingDefaultsTransform), so all values will be cast correctly.
if (isNullableOrLowCardinalityNullable(input_columns[col_idx].type)
&& !isNullableOrLowCardinalityNullable(query_columns[col_idx].type)
&& !isVariant(query_columns[col_idx].type)
&& !isDynamic(query_columns[col_idx].type)
&& output_columns.has(query_columns[col_idx].name))
query_sample_block.setColumn(col_idx, ColumnWithTypeAndName(makeNullableOrLowCardinalityNullable(query_columns[col_idx].column), makeNullableOrLowCardinalityNullable(query_columns[col_idx].type), query_columns[col_idx].name));
}
}
}
}
ThreadGroupPtr running_group;
if (current_thread)
running_group = current_thread->getThreadGroup();
if (!running_group)
running_group = std::make_shared<ThreadGroup>(getContext());
for (size_t i = 0; i < sink_streams_size; ++i)
{
auto out = buildSink(table, metadata_snapshot, /* thread_status_holder= */ nullptr,
running_group, /* elapsed_counter_ms= */ nullptr);
sink_chains.emplace_back(std::move(out));
}
for (size_t i = 0; i < pre_streams_size; ++i)
{
auto out = buildPreSinkChain(sink_chains[0].getInputHeader(), table, metadata_snapshot, query_sample_block);
presink_chains.emplace_back(std::move(out));
}
for (const auto & column : metadata_snapshot->getColumns())
if (column.default_desc.kind == ColumnDefaultKind::Materialized && query_sample_block.has(column.name))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", column.name);
}
BlockIO res;
/// What type of query: INSERT or INSERT SELECT or INSERT WATCH?
if (distributed_pipeline)
if (query.select)
{
res.pipeline = std::move(*distributed_pipeline);
}
else if (query.select)
{
const auto & header = presink_chains.at(0).getInputHeader();
auto actions_dag = ActionsDAG::makeConvertingActions(
pipeline.getHeader().getColumnsWithTypeAndName(),
header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Position);
auto actions = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes));
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
if (settings.parallel_distributed_insert_select)
{
return std::make_shared<ExpressionTransform>(in_header, actions);
});
/// We need to convert Sparse columns to full, because it's destination storage
/// may not support it or may have different settings for applying Sparse serialization.
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{
return std::make_shared<MaterializingTransform>(in_header);
});
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{
auto context_ptr = getContext();
auto counting = std::make_shared<CountingTransform>(in_header, nullptr, context_ptr->getQuota());
counting->setProcessListElement(context_ptr->getProcessListElement());
counting->setProgressCallback(context_ptr->getProgressCallback());
return counting;
});
if (shouldAddSquashingFroStorage(table))
{
bool table_prefers_large_blocks = table->prefersLargeBlocks();
size_t threads = presink_chains.size();
pipeline.resize(1);
pipeline.addTransform(std::make_shared<PlanSquashingTransform>(
header,
table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size,
table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL));
pipeline.resize(threads);
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
auto distributed = table->distributedWrite(query, getContext());
if (distributed)
{
return std::make_shared<ApplySquashingTransform>(
in_header,
table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size,
table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL);
});
res.pipeline = std::move(*distributed);
}
else
{
res.pipeline = buildInsertSelectPipeline(query, table);
}
}
size_t num_select_threads = pipeline.getNumThreads();
for (auto & chain : presink_chains)
resources = chain.detachResources();
for (auto & chain : sink_chains)
resources = chain.detachResources();
pipeline.addChains(std::move(presink_chains));
pipeline.resize(sink_chains.size());
pipeline.addChains(std::move(sink_chains));
if (!settings.parallel_view_processing)
else
{
/// Don't use more threads for INSERT than for SELECT to reduce memory consumption.
if (pipeline.getNumThreads() > num_select_threads)
pipeline.setMaxThreads(num_select_threads);
res.pipeline = buildInsertSelectPipeline(query, table);
}
else if (pipeline.getNumThreads() < settings.max_threads)
{
/// It is possible for query to have max_threads=1, due to optimize_trivial_insert_select,
/// however in case of parallel_view_processing and multiple views, views can still be processed in parallel.
///
/// Note, number of threads will be limited by buildPushingToViewsChain() to max_threads.
pipeline.setMaxThreads(settings.max_threads);
}
pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr
{
return std::make_shared<EmptySink>(cur_header);
});
if (!allow_materialized)
{
for (const auto & column : metadata_snapshot->getColumns())
if (column.default_desc.kind == ColumnDefaultKind::Materialized && header.has(column.name))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", column.name);
}
res.pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline));
}
else
{
auto & chain = presink_chains.at(0);
chain.appendChain(std::move(sink_chains.at(0)));
if (shouldAddSquashingFroStorage(table))
{
bool table_prefers_large_blocks = table->prefersLargeBlocks();
auto squashing = std::make_shared<ApplySquashingTransform>(
chain.getInputHeader(),
table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size,
table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL);
chain.addSource(std::move(squashing));
auto balancing = std::make_shared<PlanSquashingTransform>(
chain.getInputHeader(),
table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size,
table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL);
chain.addSource(std::move(balancing));
}
auto context_ptr = getContext();
auto counting = std::make_shared<CountingTransform>(chain.getInputHeader(), nullptr, context_ptr->getQuota());
counting->setProcessListElement(context_ptr->getProcessListElement());
counting->setProgressCallback(context_ptr->getProgressCallback());
chain.addSource(std::move(counting));
res.pipeline = QueryPipeline(std::move(presink_chains[0]));
res.pipeline.setNumThreads(std::min<size_t>(res.pipeline.getNumThreads(), settings.max_threads));
res.pipeline.setConcurrencyControl(settings.use_concurrency_control);
if (query.hasInlinedData() && !async_insert)
{
/// can execute without additional data
auto format = getInputFormatFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr);
for (auto && buffer : owned_buffers)
format->addBuffer(std::move(buffer));
auto pipe = getSourceFromInputFormat(query_ptr, std::move(format), getContext(), nullptr);
res.pipeline.complete(std::move(pipe));
}
res.pipeline = buildInsertPipeline(query, table);
}
res.pipeline.addResources(std::move(resources));
res.pipeline.addStorageHolder(table);
if (inner_table)
res.pipeline.addStorageHolder(inner_table);
if (const auto * mv = dynamic_cast<const StorageMaterializedView *>(table.get()))
res.pipeline.addStorageHolder(mv->getTargetTable());
LOG_TEST(getLogger("InterpreterInsertQuery"), "Pipeline could use up to {} thread", res.pipeline.getNumThreads());
return res;
}
@ -758,17 +808,27 @@ void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, Cont
}
}
void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr context_) const
{
extendQueryLogElemImpl(elem, context_);
}
void registerInterpreterInsertQuery(InterpreterFactory & factory)
{
auto create_fn = [] (const InterpreterFactory::Arguments & args)
{
return std::make_unique<InterpreterInsertQuery>(args.query, args.context, args.allow_materialized);
return std::make_unique<InterpreterInsertQuery>(
args.query,
args.context,
args.allow_materialized,
/* no_squash */false,
/* no_destination */false,
/* async_insert */false);
};
factory.registerInterpreter("InterpreterInsertQuery", create_fn);
}
}

View File

@ -23,10 +23,10 @@ public:
InterpreterInsertQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
bool allow_materialized_ = false,
bool no_squash_ = false,
bool no_destination_ = false,
bool async_insert_ = false);
bool allow_materialized_,
bool no_squash_,
bool no_destination,
bool async_insert_);
/** Prepare a request for execution. Return block streams
* - the stream into which you can write data to execute the query, if INSERT;
@ -73,12 +73,17 @@ private:
ASTPtr query_ptr;
const bool allow_materialized;
const bool no_squash;
const bool no_destination;
bool no_squash = false;
bool no_destination = false;
const bool async_insert;
std::vector<std::unique_ptr<ReadBuffer>> owned_buffers;
std::pair<std::vector<Chain>, std::vector<Chain>> buildPreAndSinkChains(size_t presink_streams, size_t sink_streams, StoragePtr table, const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block);
QueryPipeline buildInsertSelectPipeline(ASTInsertQuery & query, StoragePtr table);
QueryPipeline buildInsertPipeline(ASTInsertQuery & query, StoragePtr table);
Chain buildSink(
const StoragePtr & table,
const StorageMetadataPtr & metadata_snapshot,

View File

@ -1,6 +1,9 @@
#include <vector>
#include <Interpreters/Squashing.h>
#include "Common/Logger.h"
#include "Common/logger_useful.h"
#include <Common/CurrentThread.h>
#include <base/defines.h>
namespace DB
@ -11,24 +14,33 @@ namespace ErrorCodes
}
Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_)
: header(header_)
, min_block_size_rows(min_block_size_rows_)
: min_block_size_rows(min_block_size_rows_)
, min_block_size_bytes(min_block_size_bytes_)
, header(header_)
{
}
Chunk Squashing::flush()
{
return convertToChunk(std::move(chunks_to_merge_vec));
if (!accumulated)
return {};
auto result = convertToChunk(extract());
chassert(result);
return result;
}
Chunk Squashing::squash(Chunk && input_chunk)
{
if (!input_chunk.hasChunkInfo())
if (!input_chunk)
return Chunk();
const auto *info = getInfoFromChunk(input_chunk);
return squash(info->chunks);
auto squash_info = input_chunk.getChunkInfos().extract<ChunksToSquash>();
if (!squash_info)
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no ChunksToSquash in ChunkInfoPtr");
return squash(std::move(squash_info->chunks), std::move(input_chunk.getChunkInfos()));
}
Chunk Squashing::add(Chunk && input_chunk)
@ -37,67 +49,62 @@ Chunk Squashing::add(Chunk && input_chunk)
return {};
/// Just read block is already enough.
if (isEnoughSize(input_chunk.getNumRows(), input_chunk.bytes()))
if (isEnoughSize(input_chunk))
{
/// If no accumulated data, return just read block.
if (chunks_to_merge_vec.empty())
if (!accumulated)
{
chunks_to_merge_vec.push_back(std::move(input_chunk));
Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec));
chunks_to_merge_vec.clear();
return res_chunk;
accumulated.add(std::move(input_chunk));
return convertToChunk(extract());
}
/// Return accumulated data (maybe it has small size) and place new block to accumulated data.
Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec));
chunks_to_merge_vec.clear();
changeCurrentSize(input_chunk.getNumRows(), input_chunk.bytes());
chunks_to_merge_vec.push_back(std::move(input_chunk));
Chunk res_chunk = convertToChunk(extract());
accumulated.add(std::move(input_chunk));
return res_chunk;
}
/// Accumulated block is already enough.
if (isEnoughSize(accumulated_size.rows, accumulated_size.bytes))
if (isEnoughSize())
{
/// Return accumulated data and place new block to accumulated data.
Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec));
chunks_to_merge_vec.clear();
changeCurrentSize(input_chunk.getNumRows(), input_chunk.bytes());
chunks_to_merge_vec.push_back(std::move(input_chunk));
Chunk res_chunk = convertToChunk(extract());
accumulated.add(std::move(input_chunk));
return res_chunk;
}
/// Pushing data into accumulating vector
expandCurrentSize(input_chunk.getNumRows(), input_chunk.bytes());
chunks_to_merge_vec.push_back(std::move(input_chunk));
accumulated.add(std::move(input_chunk));
/// If accumulated data is big enough, we send it
if (isEnoughSize(accumulated_size.rows, accumulated_size.bytes))
{
Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec));
changeCurrentSize(0, 0);
chunks_to_merge_vec.clear();
return res_chunk;
}
if (isEnoughSize())
return convertToChunk(extract());
return {};
}
Chunk Squashing::convertToChunk(std::vector<Chunk> && chunks) const
Chunk Squashing::convertToChunk(CurrentData && data) const
{
if (chunks.empty())
if (data.chunks.empty())
return {};
auto info = std::make_shared<ChunksToSquash>();
info->chunks = std::move(chunks);
info->chunks = std::move(data.chunks);
chunks.clear();
// It is imortant that chunk is not empty, it has to have columns even if they are empty
// Sometimes there are could be no columns in header but not empty rows in chunks
// That happens when we intend to add defaults for the missing columns after
auto aggr_chunk = Chunk(header.getColumns(), 0);
if (header.columns() == 0)
aggr_chunk = Chunk(header.getColumns(), data.getRows());
return Chunk(header.cloneEmptyColumns(), 0, info);
aggr_chunk.getChunkInfos().add(std::move(info));
chassert(aggr_chunk);
return aggr_chunk;
}
Chunk Squashing::squash(std::vector<Chunk> & input_chunks)
Chunk Squashing::squash(std::vector<Chunk> && input_chunks, Chunk::ChunkInfoCollection && infos)
{
Chunk accumulated_chunk;
std::vector<IColumn::MutablePtr> mutable_columns = {};
size_t rows = 0;
for (const Chunk & chunk : input_chunks)
@ -119,35 +126,17 @@ Chunk Squashing::squash(std::vector<Chunk> & input_chunks)
for (size_t j = 0, size = mutable_columns.size(); j < size; ++j)
{
const auto source_column = columns[j];
mutable_columns[j]->insertRangeFrom(*source_column, 0, source_column->size());
}
}
accumulated_chunk.setColumns(std::move(mutable_columns), rows);
return accumulated_chunk;
}
const ChunksToSquash* Squashing::getInfoFromChunk(const Chunk & chunk)
{
const auto& info = chunk.getChunkInfo();
const auto * agg_info = typeid_cast<const ChunksToSquash *>(info.get());
Chunk result;
result.setColumns(std::move(mutable_columns), rows);
result.setChunkInfos(infos);
result.getChunkInfos().append(std::move(input_chunks.back().getChunkInfos()));
if (!agg_info)
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no ChunksToSquash in ChunkInfoPtr");
return agg_info;
}
void Squashing::expandCurrentSize(size_t rows, size_t bytes)
{
accumulated_size.rows += rows;
accumulated_size.bytes += bytes;
}
void Squashing::changeCurrentSize(size_t rows, size_t bytes)
{
accumulated_size.rows = rows;
accumulated_size.bytes = bytes;
chassert(result);
return result;
}
bool Squashing::isEnoughSize(size_t rows, size_t bytes) const
@ -156,4 +145,29 @@ bool Squashing::isEnoughSize(size_t rows, size_t bytes) const
|| (min_block_size_rows && rows >= min_block_size_rows)
|| (min_block_size_bytes && bytes >= min_block_size_bytes);
}
bool Squashing::isEnoughSize() const
{
return isEnoughSize(accumulated.getRows(), accumulated.getBytes());
};
bool Squashing::isEnoughSize(const Chunk & chunk) const
{
return isEnoughSize(chunk.getNumRows(), chunk.bytes());
}
void Squashing::CurrentData::add(Chunk && chunk)
{
rows += chunk.getNumRows();
bytes += chunk.bytes();
chunks.push_back(std::move(chunk));
}
Squashing::CurrentData Squashing::extract()
{
auto result = std::move(accumulated);
accumulated = {};
return result;
}
}

View File

@ -8,9 +8,18 @@
namespace DB
{
struct ChunksToSquash : public ChunkInfo
class ChunksToSquash : public ChunkInfoCloneable<ChunksToSquash>
{
mutable std::vector<Chunk> chunks = {};
public:
ChunksToSquash() = default;
ChunksToSquash(const ChunksToSquash & other)
{
chunks.reserve(other.chunks.size());
for (const auto & chunk: other.chunks)
chunks.push_back(chunk.clone());
}
std::vector<Chunk> chunks = {};
};
/** Merging consecutive passed blocks to specified minimum size.
@ -36,34 +45,37 @@ public:
static Chunk squash(Chunk && input_chunk);
Chunk flush();
bool isDataLeft()
{
return !chunks_to_merge_vec.empty();
}
void setHeader(Block header_) { header = std::move(header_); }
const Block & getHeader() const { return header; }
Block header;
private:
struct CurrentSize
struct CurrentData
{
std::vector<Chunk> chunks = {};
size_t rows = 0;
size_t bytes = 0;
explicit operator bool () const { return !chunks.empty(); }
size_t getRows() const { return rows; }
size_t getBytes() const { return bytes; }
void add(Chunk && chunk);
};
std::vector<Chunk> chunks_to_merge_vec = {};
size_t min_block_size_rows;
size_t min_block_size_bytes;
const size_t min_block_size_rows;
const size_t min_block_size_bytes;
Block header;
CurrentSize accumulated_size;
CurrentData accumulated;
static const ChunksToSquash * getInfoFromChunk(const Chunk & chunk);
static Chunk squash(std::vector<Chunk> && input_chunks, Chunk::ChunkInfoCollection && infos);
static Chunk squash(std::vector<Chunk> & input_chunks);
void expandCurrentSize(size_t rows, size_t bytes);
void changeCurrentSize(size_t rows, size_t bytes);
bool isEnoughSize() const;
bool isEnoughSize(size_t rows, size_t bytes) const;
bool isEnoughSize(const Chunk & chunk) const;
Chunk convertToChunk(std::vector<Chunk> && chunks) const;
CurrentData extract();
Chunk convertToChunk(CurrentData && data) const;
};
}

View File

@ -544,7 +544,13 @@ void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush,
insert_context->makeQueryContext();
addSettingsForQuery(insert_context, IAST::QueryKind::Insert);
InterpreterInsertQuery interpreter(query_ptr, insert_context);
InterpreterInsertQuery interpreter(
query_ptr,
insert_context,
/* allow_materialized */ false,
/* no_squash */ false,
/* no_destination */ false,
/* async_isnert */ false);
BlockIO io = interpreter.execute();
PushingPipelineExecutor executor(io.pipeline);

View File

@ -1188,7 +1188,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
}
}
/// Check for dynamic subcolums in unknown required columns.
/// Check for dynamic subcolumns in unknown required columns.
if (!unknown_required_source_columns.empty())
{
for (const NameAndTypePair & pair : source_columns_ordinary)

View File

@ -40,7 +40,7 @@ ColumnPtr tryGetSubcolumnFromBlock(const Block & block, const DataTypePtr & requ
auto subcolumn_name = requested_subcolumn.getSubcolumnName();
/// If requested subcolumn is dynamic, we should first perform cast and then
/// extract the subcolumn, because the data of dynamic subcolumn can change after cast.
if (elem->type->hasDynamicSubcolumns() && !elem->type->equals(*requested_column_type))
if ((elem->type->hasDynamicSubcolumns() || requested_column_type->hasDynamicSubcolumns()) && !elem->type->equals(*requested_column_type))
{
auto casted_column = castColumn({elem->column, elem->type, ""}, requested_column_type);
auto elem_column = requested_column_type->tryGetSubcolumn(subcolumn_name, casted_column);

View File

@ -385,7 +385,7 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons
if (database->getEngineName() != "Ordinary")
return;
Strings permanently_detached_tables = database->getNamesOfPermanentlyDetachedTables();
const Strings permanently_detached_tables = database->getNamesOfPermanentlyDetachedTables();
if (!permanently_detached_tables.empty())
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot automatically convert database {} from Ordinary to Atomic, "

View File

@ -87,14 +87,14 @@ bool canRemoveConstantFromGroupByKey(const ConstantNode & root)
else if (function_node)
{
/// Do not allow removing constants like `hostName()`
if (!function_node->getFunctionOrThrow()->isDeterministic())
if (function_node->getFunctionOrThrow()->isServerConstant())
return false;
for (const auto & child : function_node->getArguments())
nodes.push(child.get());
}
else
return false;
// else
// return false;
}
return true;

View File

@ -19,14 +19,6 @@ Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) : columns(std::move(columns
checkNumRowsIsConsistent();
}
Chunk::Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_)
: columns(std::move(columns_))
, num_rows(num_rows_)
, chunk_info(std::move(chunk_info_))
{
checkNumRowsIsConsistent();
}
static Columns unmuteColumns(MutableColumns && mutable_columns)
{
Columns columns;
@ -43,17 +35,11 @@ Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_)
checkNumRowsIsConsistent();
}
Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_)
: columns(unmuteColumns(std::move(columns_)))
, num_rows(num_rows_)
, chunk_info(std::move(chunk_info_))
{
checkNumRowsIsConsistent();
}
Chunk Chunk::clone() const
{
return Chunk(getColumns(), getNumRows(), chunk_info);
auto tmp = Chunk(getColumns(), getNumRows());
tmp.setChunkInfos(chunk_infos.clone());
return tmp;
}
void Chunk::setColumns(Columns columns_, UInt64 num_rows_)

View File

@ -1,7 +1,9 @@
#pragma once
#include <Common/CollectionOfDerived.h>
#include <Columns/IColumn.h>
#include <unordered_map>
#include <memory>
namespace DB
{
@ -9,11 +11,29 @@ namespace DB
class ChunkInfo
{
public:
virtual ~ChunkInfo() = default;
using Ptr = std::shared_ptr<ChunkInfo>;
ChunkInfo() = default;
ChunkInfo(const ChunkInfo&) = default;
ChunkInfo(ChunkInfo&&) = default;
virtual Ptr clone() const = 0;
virtual ~ChunkInfo() = default;
};
using ChunkInfoPtr = std::shared_ptr<const ChunkInfo>;
template<class Derived>
class ChunkInfoCloneable : public ChunkInfo
{
public:
ChunkInfoCloneable() = default;
ChunkInfoCloneable(const ChunkInfoCloneable & other) = default;
Ptr clone() const override
{
return std::static_pointer_cast<ChunkInfo>(std::make_shared<Derived>(*static_cast<const Derived*>(this)));
}
};
/**
* Chunk is a list of columns with the same length.
@ -32,26 +52,26 @@ using ChunkInfoPtr = std::shared_ptr<const ChunkInfo>;
class Chunk
{
public:
using ChunkInfoCollection = CollectionOfDerivedItems<ChunkInfo>;
Chunk() = default;
Chunk(const Chunk & other) = delete;
Chunk(Chunk && other) noexcept
: columns(std::move(other.columns))
, num_rows(other.num_rows)
, chunk_info(std::move(other.chunk_info))
, chunk_infos(std::move(other.chunk_infos))
{
other.num_rows = 0;
}
Chunk(Columns columns_, UInt64 num_rows_);
Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_);
Chunk(MutableColumns columns_, UInt64 num_rows_);
Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_);
Chunk & operator=(const Chunk & other) = delete;
Chunk & operator=(Chunk && other) noexcept
{
columns = std::move(other.columns);
chunk_info = std::move(other.chunk_info);
chunk_infos = std::move(other.chunk_infos);
num_rows = other.num_rows;
other.num_rows = 0;
return *this;
@ -62,15 +82,15 @@ public:
void swap(Chunk & other) noexcept
{
columns.swap(other.columns);
chunk_info.swap(other.chunk_info);
std::swap(num_rows, other.num_rows);
chunk_infos.swap(other.chunk_infos);
}
void clear()
{
num_rows = 0;
columns.clear();
chunk_info.reset();
chunk_infos.clear();
}
const Columns & getColumns() const { return columns; }
@ -81,9 +101,9 @@ public:
/** Get empty columns with the same types as in block. */
MutableColumns cloneEmptyColumns() const;
const ChunkInfoPtr & getChunkInfo() const { return chunk_info; }
bool hasChunkInfo() const { return chunk_info != nullptr; }
void setChunkInfo(ChunkInfoPtr chunk_info_) { chunk_info = std::move(chunk_info_); }
ChunkInfoCollection & getChunkInfos() { return chunk_infos; }
const ChunkInfoCollection & getChunkInfos() const { return chunk_infos; }
void setChunkInfos(ChunkInfoCollection chunk_infos_) { chunk_infos = std::move(chunk_infos_); }
UInt64 getNumRows() const { return num_rows; }
UInt64 getNumColumns() const { return columns.size(); }
@ -107,7 +127,7 @@ public:
private:
Columns columns;
UInt64 num_rows = 0;
ChunkInfoPtr chunk_info;
ChunkInfoCollection chunk_infos;
void checkNumRowsIsConsistent();
};
@ -117,11 +137,15 @@ using Chunks = std::vector<Chunk>;
/// AsyncInsert needs two kinds of information:
/// - offsets of different sub-chunks
/// - tokens of different sub-chunks, which are assigned by setting `insert_deduplication_token`.
class AsyncInsertInfo : public ChunkInfo
class AsyncInsertInfo : public ChunkInfoCloneable<AsyncInsertInfo>
{
public:
AsyncInsertInfo() = default;
explicit AsyncInsertInfo(const std::vector<size_t> & offsets_, const std::vector<String> & tokens_) : offsets(offsets_), tokens(tokens_) {}
AsyncInsertInfo(const AsyncInsertInfo & other) = default;
AsyncInsertInfo(const std::vector<size_t> & offsets_, const std::vector<String> & tokens_)
: offsets(offsets_)
, tokens(tokens_)
{}
std::vector<size_t> offsets;
std::vector<String> tokens;
@ -130,9 +154,11 @@ public:
using AsyncInsertInfoPtr = std::shared_ptr<AsyncInsertInfo>;
/// Extension to support delayed defaults. AddingDefaultsProcessor uses it to replace missing values with column defaults.
class ChunkMissingValues : public ChunkInfo
class ChunkMissingValues : public ChunkInfoCloneable<ChunkMissingValues>
{
public:
ChunkMissingValues(const ChunkMissingValues & other) = default;
using RowsBitMask = std::vector<bool>; /// a bit per row for a column
const RowsBitMask & getDefaultsBitmask(size_t column_idx) const;

View File

@ -147,13 +147,10 @@ bool PullingAsyncPipelineExecutor::pull(Block & block, uint64_t milliseconds)
block = lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns());
if (auto chunk_info = chunk.getChunkInfo())
if (auto agg_info = chunk.getChunkInfos().get<AggregatedChunkInfo>())
{
if (const auto * agg_info = typeid_cast<const AggregatedChunkInfo *>(chunk_info.get()))
{
block.info.bucket_num = agg_info->bucket_num;
block.info.is_overflows = agg_info->is_overflows;
}
block.info.bucket_num = agg_info->bucket_num;
block.info.is_overflows = agg_info->is_overflows;
}
return true;

View File

@ -73,13 +73,10 @@ bool PullingPipelineExecutor::pull(Block & block)
}
block = pulling_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns());
if (auto chunk_info = chunk.getChunkInfo())
if (auto agg_info = chunk.getChunkInfos().get<AggregatedChunkInfo>())
{
if (const auto * agg_info = typeid_cast<const AggregatedChunkInfo *>(chunk_info.get()))
{
block.info.bucket_num = agg_info->bucket_num;
block.info.is_overflows = agg_info->is_overflows;
}
block.info.bucket_num = agg_info->bucket_num;
block.info.is_overflows = agg_info->is_overflows;
}
return true;

View File

@ -179,7 +179,9 @@ void ParquetBlockOutputFormat::consume(Chunk chunk)
columns[i]->insertRangeFrom(*concatenated.getColumns()[i], offset, count);
Chunks piece;
piece.emplace_back(std::move(columns), count, concatenated.getChunkInfo());
piece.emplace_back(std::move(columns), count);
piece.back().setChunkInfos(concatenated.getChunkInfos());
writeRowGroup(std::move(piece));
}
}

View File

@ -8,8 +8,9 @@ namespace ErrorCodes
}
IAccumulatingTransform::IAccumulatingTransform(Block input_header, Block output_header)
: IProcessor({std::move(input_header)}, {std::move(output_header)}),
input(inputs.front()), output(outputs.front())
: IProcessor({std::move(input_header)}, {std::move(output_header)})
, input(inputs.front())
, output(outputs.front())
{
}

View File

@ -53,13 +53,11 @@ void FinishAggregatingInOrderAlgorithm::consume(Input & input, size_t source_num
if (!input.chunk.hasRows())
return;
const auto & info = input.chunk.getChunkInfo();
if (!info)
if (input.chunk.getChunkInfos().empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in FinishAggregatingInOrderAlgorithm");
Int64 allocated_bytes = 0;
/// Will be set by AggregatingInOrderTransform during local aggregation; will be nullptr during merging on initiator.
if (const auto * arenas_info = typeid_cast<const ChunkInfoWithAllocatedBytes *>(info.get()))
if (auto arenas_info = input.chunk.getChunkInfos().get<ChunkInfoWithAllocatedBytes>())
allocated_bytes = arenas_info->allocated_bytes;
states[source_num] = State{input.chunk, description, allocated_bytes};
@ -136,7 +134,7 @@ Chunk FinishAggregatingInOrderAlgorithm::prepareToMerge()
info->chunk_num = chunk_num++;
Chunk chunk;
chunk.setChunkInfo(std::move(info));
chunk.getChunkInfos().add(std::move(info));
return chunk;
}
@ -163,7 +161,7 @@ void FinishAggregatingInOrderAlgorithm::addToAggregation()
chunks.emplace_back(std::move(new_columns), current_rows);
}
chunks.back().setChunkInfo(std::make_shared<AggregatedChunkInfo>());
chunks.back().getChunkInfos().add(std::make_shared<AggregatedChunkInfo>());
states[i].current_row = states[i].to_row;
/// We assume that sizes in bytes of rows are almost the same.

View File

@ -6,18 +6,22 @@ namespace DB
{
/// To carry part level if chunk is produced by a merge tree source
class MergeTreePartLevelInfo : public ChunkInfo
class MergeTreePartLevelInfo : public ChunkInfoCloneable<MergeTreePartLevelInfo>
{
public:
MergeTreePartLevelInfo() = delete;
explicit MergeTreePartLevelInfo(ssize_t part_level) : origin_merge_tree_part_level(part_level) { }
explicit MergeTreePartLevelInfo(ssize_t part_level)
: origin_merge_tree_part_level(part_level)
{ }
MergeTreePartLevelInfo(const MergeTreePartLevelInfo & other) = default;
size_t origin_merge_tree_part_level = 0;
};
inline size_t getPartLevelFromChunk(const Chunk & chunk)
{
const auto & info = chunk.getChunkInfo();
if (const auto * part_level_info = typeid_cast<const MergeTreePartLevelInfo *>(info.get()))
const auto part_level_info = chunk.getChunkInfos().get<MergeTreePartLevelInfo>();
if (part_level_info)
return part_level_info->origin_merge_tree_part_level;
return 0;
}

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