diff --git a/.clang-tidy b/.clang-tidy
index de19059d09e..bb63bf2eea6 100644
--- a/.clang-tidy
+++ b/.clang-tidy
@@ -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',
diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml
index 64c3d2f8342..60bd79560eb 100644
--- a/.github/workflows/backport_branches.yml
+++ b/.github/workflows/backport_branches.yml
@@ -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:
diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml
index 972aff90195..d8d27531f28 100644
--- a/.github/workflows/create_release.yml
+++ b/.github/workflows/create_release.yml
@@ -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:
diff --git a/.github/workflows/reusable_docker.yml b/.github/workflows/docker_test_images.yml
similarity index 100%
rename from .github/workflows/reusable_docker.yml
rename to .github/workflows/docker_test_images.yml
diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml
index 2a7e6f737ab..d27b1987532 100644
--- a/.github/workflows/master.yml
+++ b/.github/workflows/master.yml
@@ -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:
diff --git a/.github/workflows/merge_queue.yml b/.github/workflows/merge_queue.yml
index 01685ee1f5a..c08c3fb30ac 100644
--- a/.github/workflows/merge_queue.yml
+++ b/.github/workflows/merge_queue.yml
@@ -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:
diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml
index 3e1c5576e7d..bffe5b4c1bf 100644
--- a/.github/workflows/nightly.yml
+++ b/.github/workflows/nightly.yml
@@ -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
diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml
index c9f4f858825..04ce4d29ce9 100644
--- a/.github/workflows/pull_request.yml
+++ b/.github/workflows/pull_request.yml
@@ -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:
diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml
index 6bf846d7535..e1a97b91016 100644
--- a/.github/workflows/release_branches.yml
+++ b/.github/workflows/release_branches.yml
@@ -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:
diff --git a/.github/workflows/reusable_test.yml b/.github/workflows/reusable_test.yml
index c01dd8ca9d4..ca6df5df14e 100644
--- a/.github/workflows/reusable_test.yml
+++ b/.github/workflows/reusable_test.yml
@@ -102,6 +102,8 @@ jobs:
--job-name '${{inputs.test_name}}' \
--run \
--run-command '''${{inputs.run_command}}'''
+ # shellcheck disable=SC2319
+ echo "JOB_EXIT_CODE=$?" >> "$GITHUB_ENV"
- name: Post run
if: ${{ !cancelled() }}
run: |
diff --git a/base/base/cgroupsv2.cpp b/base/base/cgroupsv2.cpp
index f20b9daf22e..87f62bf377d 100644
--- a/base/base/cgroupsv2.cpp
+++ b/base/base/cgroupsv2.cpp
@@ -3,8 +3,9 @@
#include
#include
-#include
+#include
+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
}
diff --git a/base/base/cgroupsv2.h b/base/base/cgroupsv2.h
index 70219d87cd1..cfb916ff358 100644
--- a/base/base/cgroupsv2.h
+++ b/base/base/cgroupsv2.h
@@ -1,7 +1,6 @@
#pragma once
#include
-#include
#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();
diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp
index f47cba9833d..afdb6ba068a 100644
--- a/base/base/getMemoryAmount.cpp
+++ b/base/base/getMemoryAmount.cpp
@@ -23,8 +23,9 @@ std::optional 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.
diff --git a/contrib/grpc b/contrib/grpc
index f5b7fdc2dff..1716359d2e2 160000
--- a/contrib/grpc
+++ b/contrib/grpc
@@ -1 +1 @@
-Subproject commit f5b7fdc2dff09ada06dbf6c75df298fb40f898df
+Subproject commit 1716359d2e28d304a250f9df0e6c0ccad03de8db
diff --git a/contrib/libunwind b/contrib/libunwind
index d6a01c46327..8f28e64d158 160000
--- a/contrib/libunwind
+++ b/contrib/libunwind
@@ -1 +1 @@
-Subproject commit d6a01c46327e56fd86beb8aaa31591fcd9a6b7df
+Subproject commit 8f28e64d15819d2d096badd598c7d85bebddb1f2
diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt
index 37a2f29afcf..b566e8cb9b3 100644
--- a/contrib/libunwind-cmake/CMakeLists.txt
+++ b/contrib/libunwind-cmake/CMakeLists.txt
@@ -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 $)
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)
diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile
index 2512268be0f..4cac2ee6135 100644
--- a/docker/test/fasttest/Dockerfile
+++ b/docker/test/fasttest/Dockerfile
@@ -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
diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh
index 0d975d64010..970bf12a81a 100755
--- a/docker/test/fasttest/run.sh
+++ b/docker/test/fasttest/run.sh
@@ -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
diff --git a/docker/test/libfuzzer/Dockerfile b/docker/test/libfuzzer/Dockerfile
index e6eb2ae336e..3ffae0cd921 100644
--- a/docker/test/libfuzzer/Dockerfile
+++ b/docker/test/libfuzzer/Dockerfile
@@ -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 -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/libfuzzer
diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh
index 1f28d1ac6ea..35ffeee5438 100755
--- a/docker/test/stateful/run.sh
+++ b/docker/test/stateful/run.sh
@@ -25,7 +25,7 @@ source /utils.lib
azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent --inMemoryPersistence &
./setup_minio.sh stateful
-./mc admin trace clickminio > /test_output/rubbish.log &
+./mc admin trace clickminio > /test_output/minio.log &
MC_ADMIN_PID=$!
config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml
diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh
index ae7ed1bcd20..2b535f8dd23 100755
--- a/docker/test/stateless/run.sh
+++ b/docker/test/stateless/run.sh
@@ -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
@@ -54,7 +54,7 @@ source /utils.lib
/usr/share/clickhouse-test/config/install.sh
./setup_minio.sh stateless
-m./c admin trace clickminio > /test_output/rubbish.log &
+./mc admin trace clickminio > /test_output/minio.log &
MC_ADMIN_PID=$!
./setup_hdfs_minicluster.sh
diff --git a/docker/test/stateless/setup_hdfs_minicluster.sh b/docker/test/stateless/setup_hdfs_minicluster.sh
index 6671e73562a..15a54f59096 100755
--- a/docker/test/stateless/setup_hdfs_minicluster.sh
+++ b/docker/test/stateless/setup_hdfs_minicluster.sh
@@ -10,7 +10,7 @@ cd hadoop-3.3.1
export JAVA_HOME=/usr
mkdir -p target/test/data
chown clickhouse ./target/test/data
-sudo -E -u clickhouse bin/mapred minicluster -format -nomr -nnport 12222 >> /test_output/garbage.log 2>&1 &
+sudo -E -u clickhouse bin/mapred minicluster -format -nomr -nnport 12222 >> /test_output/hdfs_minicluster.log 2>&1 &
while ! nc -z localhost 12222; do
sleep 1
diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile
index f13bb576f79..babddbd228c 100644
--- a/docker/test/util/Dockerfile
+++ b/docker/test/util/Dockerfile
@@ -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 \
diff --git a/docs/en/development/build-cross-riscv.md b/docs/en/development/build-cross-riscv.md
index 759d97823e2..dd97b6081e8 100644
--- a/docs/en/development/build-cross-riscv.md
+++ b/docs/en/development/build-cross-riscv.md
@@ -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
```
diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md
index e18ff6f1a3f..849b3d4b486 100644
--- a/docs/en/interfaces/cli.md
+++ b/docs/en/interfaces/cli.md
@@ -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'
diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md
index b91b794d2d6..9c4c082bc3a 100644
--- a/docs/en/interfaces/formats.md
+++ b/docs/en/interfaces/formats.md
@@ -67,6 +67,7 @@ The supported formats are:
| [Prometheus](#prometheus) | ✗ | ✔ |
| [Protobuf](#protobuf) | ✔ | ✔ |
| [ProtobufSingle](#protobufsingle) | ✔ | ✔ |
+| [ProtobufList](#protobuflist) | ✔ | ✔ |
| [Avro](#data-format-avro) | ✔ | ✔ |
| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ |
| [Parquet](#data-format-parquet) | ✔ | ✔ |
@@ -1952,6 +1953,35 @@ SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf
Same as [Protobuf](#protobuf) but for storing/parsing single Protobuf message without length delimiters.
+## ProtobufList {#protobuflist}
+
+Similar to Protobuf but rows are represented as a sequence of sub-messages contained in a message with fixed name "Envelope".
+
+Usage example:
+
+``` sql
+SELECT * FROM test.table FORMAT ProtobufList SETTINGS format_schema = 'schemafile:MessageType'
+```
+
+``` bash
+cat protobuflist_messages.bin | clickhouse-client --query "INSERT INTO test.table FORMAT ProtobufList SETTINGS format_schema='schemafile:MessageType'"
+```
+
+where the file `schemafile.proto` looks like this:
+
+``` capnp
+syntax = "proto3";
+message Envelope {
+ message MessageType {
+ string name = 1;
+ string surname = 2;
+ uint32 birthDate = 3;
+ repeated string phoneNumbers = 4;
+ };
+ MessageType row = 1;
+};
+```
+
## Avro {#data-format-avro}
[Apache Avro](https://avro.apache.org/) is a row-oriented data serialization framework developed within Apache’s Hadoop project.
diff --git a/docs/en/interfaces/mysql.md b/docs/en/interfaces/mysql.md
index 42820505406..30a8d9569a9 100644
--- a/docs/en/interfaces/mysql.md
+++ b/docs/en/interfaces/mysql.md
@@ -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
diff --git a/docs/en/operations/system-tables/detached_tables.md b/docs/en/operations/system-tables/detached_tables.md
new file mode 100644
index 00000000000..f669c8fd251
--- /dev/null
+++ b/docs/en/operations/system-tables/detached_tables.md
@@ -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
+```
diff --git a/docs/en/sql-reference/data-types/array.md b/docs/en/sql-reference/data-types/array.md
index e5a8ce5d18b..1737fdd88b2 100644
--- a/docs/en/sql-reference/data-types/array.md
+++ b/docs/en/sql-reference/data-types/array.md
@@ -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) │
+└────────────┴────────────────────────┴───────────────────────────┴────────────────────────┘
+```
diff --git a/docs/en/sql-reference/data-types/lowcardinality.md b/docs/en/sql-reference/data-types/lowcardinality.md
index 133ac2bd72e..a92f6f4dce6 100644
--- a/docs/en/sql-reference/data-types/lowcardinality.md
+++ b/docs/en/sql-reference/data-types/lowcardinality.md
@@ -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)
diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md
index 260457b3be1..b7e4094f30e 100644
--- a/docs/en/sql-reference/functions/other-functions.md
+++ b/docs/en/sql-reference/functions/other-functions.md
@@ -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**
diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md
index e990023efbc..0bde207dcc9 100644
--- a/docs/en/sql-reference/functions/uuid-functions.md
+++ b/docs/en/sql-reference/functions/uuid-functions.md
@@ -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
diff --git a/docs/zh/changelog/index.md b/docs/zh/changelog/index.md
index c91d8bcf4d1..cd77a8c03cf 100644
--- a/docs/zh/changelog/index.md
+++ b/docs/zh/changelog/index.md
@@ -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}
diff --git a/docs/zh/sql-reference/functions/conditional-functions.md b/docs/zh/sql-reference/functions/conditional-functions.md
index e0f8f7a4777..378e4d897c0 100644
--- a/docs/zh/sql-reference/functions/conditional-functions.md
+++ b/docs/zh/sql-reference/functions/conditional-functions.md
@@ -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 │
└──────┴───────┴─────────────────────────────────────┘
```
diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt
index 6b3a0b16624..ce3a4659e0e 100644
--- a/programs/CMakeLists.txt
+++ b/programs/CMakeLists.txt
@@ -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.),
diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp
index e4b40d98819..887c5cb86bc 100644
--- a/programs/client/Client.cpp
+++ b/programs/client/Client.cpp
@@ -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())
{
- if (auto *setDialect = q->changes.tryGet("dialect"); setDialect && setDialect->safeGet() == "kusto")
+ if (auto *set_dialect = q->changes.tryGet("dialect"); set_dialect && set_dialect->safeGet() == "kusto")
return true;
}
diff --git a/programs/client/clickhouse-client.xml b/programs/client/clickhouse-client.xml
index d0deb818c1e..c32b63413e9 100644
--- a/programs/client/clickhouse-client.xml
+++ b/programs/client/clickhouse-client.xml
@@ -1,5 +1,6 @@
+
true
@@ -72,6 +73,7 @@
Default: "hostname" will be used. -->
default
+
127.0.0.1
9000
diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp
index a7265ef0de4..ad9332ceca1 100644
--- a/programs/local/LocalServer.cpp
+++ b/programs/local/LocalServer.cpp
@@ -423,6 +423,7 @@ void LocalServer::connect()
{
connection_parameters = ConnectionParameters(getClientConfiguration(), "localhost");
+ /// This is needed for table function input(...).
ReadBuffer * in;
auto table_file = getClientConfiguration().getString("table-file", "-");
if (table_file == "-" || table_file == "stdin")
diff --git a/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.cpp b/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.cpp
index 934a8dffd90..5d833796510 100644
--- a/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.cpp
+++ b/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.cpp
@@ -118,10 +118,10 @@ AggregateFunctionPtr createAggregateFunctionAnalysisOfVariance(const std::string
void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory & factory)
{
AggregateFunctionProperties properties = { .is_order_dependent = false };
- factory.registerFunction("analysisOfVariance", {createAggregateFunctionAnalysisOfVariance, properties}, AggregateFunctionFactory::CaseInsensitive);
+ factory.registerFunction("analysisOfVariance", {createAggregateFunctionAnalysisOfVariance, properties}, AggregateFunctionFactory::Case::Insensitive);
/// This is widely used term
- factory.registerAlias("anova", "analysisOfVariance", AggregateFunctionFactory::CaseInsensitive);
+ factory.registerAlias("anova", "analysisOfVariance", AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionAny.cpp b/src/AggregateFunctions/AggregateFunctionAny.cpp
index f727ab04aa9..2bcee0fdd5f 100644
--- a/src/AggregateFunctions/AggregateFunctionAny.cpp
+++ b/src/AggregateFunctions/AggregateFunctionAny.cpp
@@ -361,9 +361,9 @@ void registerAggregateFunctionsAny(AggregateFunctionFactory & factory)
AggregateFunctionProperties default_properties = {.returns_default_when_only_null = false, .is_order_dependent = true};
factory.registerFunction("any", {createAggregateFunctionAny, default_properties});
- factory.registerAlias("any_value", "any", AggregateFunctionFactory::CaseInsensitive);
- factory.registerAlias("first_value", "any", AggregateFunctionFactory::CaseInsensitive);
+ factory.registerAlias("any_value", "any", AggregateFunctionFactory::Case::Insensitive);
+ factory.registerAlias("first_value", "any", AggregateFunctionFactory::Case::Insensitive);
factory.registerFunction("anyLast", {createAggregateFunctionAnyLast, default_properties});
- factory.registerAlias("last_value", "anyLast", AggregateFunctionFactory::CaseInsensitive);
+ factory.registerAlias("last_value", "anyLast", AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp b/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp
index 7275409c151..0b6642bffac 100644
--- a/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp
+++ b/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp
@@ -221,11 +221,11 @@ void registerAggregateFunctionsAnyRespectNulls(AggregateFunctionFactory & factor
= {.returns_default_when_only_null = false, .is_order_dependent = true, .is_window_function = true};
factory.registerFunction("any_respect_nulls", {createAggregateFunctionAnyRespectNulls, default_properties_for_respect_nulls});
- factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
- factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
+ factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
+ factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionAnyLastRespectNulls, default_properties_for_respect_nulls});
- factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
+ factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
/// Must happen after registering any and anyLast
factory.registerNullsActionTransformation("any", "any_respect_nulls");
diff --git a/src/AggregateFunctions/AggregateFunctionAvg.cpp b/src/AggregateFunctions/AggregateFunctionAvg.cpp
index ac6d2cf7fb4..57b14921c99 100644
--- a/src/AggregateFunctions/AggregateFunctionAvg.cpp
+++ b/src/AggregateFunctions/AggregateFunctionAvg.cpp
@@ -46,6 +46,6 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const
void registerAggregateFunctionAvg(AggregateFunctionFactory & factory)
{
- factory.registerFunction("avg", createAggregateFunctionAvg, AggregateFunctionFactory::CaseInsensitive);
+ factory.registerFunction("avg", createAggregateFunctionAvg, AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionBitwise.cpp b/src/AggregateFunctions/AggregateFunctionBitwise.cpp
index 619251552e4..ecced5f3e32 100644
--- a/src/AggregateFunctions/AggregateFunctionBitwise.cpp
+++ b/src/AggregateFunctions/AggregateFunctionBitwise.cpp
@@ -234,9 +234,9 @@ void registerAggregateFunctionsBitwise(AggregateFunctionFactory & factory)
factory.registerFunction("groupBitXor", createAggregateFunctionBitwise);
/// Aliases for compatibility with MySQL.
- factory.registerAlias("BIT_OR", "groupBitOr", AggregateFunctionFactory::CaseInsensitive);
- factory.registerAlias("BIT_AND", "groupBitAnd", AggregateFunctionFactory::CaseInsensitive);
- factory.registerAlias("BIT_XOR", "groupBitXor", AggregateFunctionFactory::CaseInsensitive);
+ factory.registerAlias("BIT_OR", "groupBitOr", AggregateFunctionFactory::Case::Insensitive);
+ factory.registerAlias("BIT_AND", "groupBitAnd", AggregateFunctionFactory::Case::Insensitive);
+ factory.registerAlias("BIT_XOR", "groupBitXor", AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionCorr.cpp b/src/AggregateFunctions/AggregateFunctionCorr.cpp
index 2e8ff3af933..02d3a4aa912 100644
--- a/src/AggregateFunctions/AggregateFunctionCorr.cpp
+++ b/src/AggregateFunctions/AggregateFunctionCorr.cpp
@@ -9,7 +9,7 @@ template using AggregateFunctionCorr = AggregateFunct
void registerAggregateFunctionsStatisticsCorr(AggregateFunctionFactory & factory)
{
- factory.registerFunction("corr", createAggregateFunctionStatisticsBinary, AggregateFunctionFactory::CaseInsensitive);
+ factory.registerFunction("corr", createAggregateFunctionStatisticsBinary, AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionCount.cpp b/src/AggregateFunctions/AggregateFunctionCount.cpp
index 25f991ab693..ad3aee90c37 100644
--- a/src/AggregateFunctions/AggregateFunctionCount.cpp
+++ b/src/AggregateFunctions/AggregateFunctionCount.cpp
@@ -37,7 +37,7 @@ AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, cons
void registerAggregateFunctionCount(AggregateFunctionFactory & factory)
{
AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false };
- factory.registerFunction("count", {createAggregateFunctionCount, properties}, AggregateFunctionFactory::CaseInsensitive);
+ factory.registerFunction("count", {createAggregateFunctionCount, properties}, AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionCovar.cpp b/src/AggregateFunctions/AggregateFunctionCovar.cpp
index 9645685483f..e4877a0aed3 100644
--- a/src/AggregateFunctions/AggregateFunctionCovar.cpp
+++ b/src/AggregateFunctions/AggregateFunctionCovar.cpp
@@ -13,8 +13,8 @@ void registerAggregateFunctionsStatisticsCovar(AggregateFunctionFactory & factor
factory.registerFunction("covarPop", createAggregateFunctionStatisticsBinary);
/// Synonyms for compatibility.
- factory.registerAlias("COVAR_SAMP", "covarSamp", AggregateFunctionFactory::CaseInsensitive);
- factory.registerAlias("COVAR_POP", "covarPop", AggregateFunctionFactory::CaseInsensitive);
+ factory.registerAlias("COVAR_SAMP", "covarSamp", AggregateFunctionFactory::Case::Insensitive);
+ factory.registerAlias("COVAR_POP", "covarPop", AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp
index b5c6440a69c..082fa11ca8a 100644
--- a/src/AggregateFunctions/AggregateFunctionFactory.cpp
+++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp
@@ -29,7 +29,7 @@ const String & getAggregateFunctionCanonicalNameIfAny(const String & name)
return AggregateFunctionFactory::instance().getCanonicalNameIfAny(name);
}
-void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, CaseSensitiveness case_sensitiveness)
+void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, Case case_sensitiveness)
{
if (creator_with_properties.creator == nullptr)
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory: "
@@ -39,7 +39,7 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory: the aggregate function name '{}' is not unique",
name);
- if (case_sensitiveness == CaseInsensitive)
+ if (case_sensitiveness == Case::Insensitive)
{
auto key = Poco::toLower(name);
if (!case_insensitive_aggregate_functions.emplace(key, creator_with_properties).second)
diff --git a/src/AggregateFunctions/AggregateFunctionFactory.h b/src/AggregateFunctions/AggregateFunctionFactory.h
index b1dc422fcb0..a5fa3424543 100644
--- a/src/AggregateFunctions/AggregateFunctionFactory.h
+++ b/src/AggregateFunctions/AggregateFunctionFactory.h
@@ -60,7 +60,7 @@ public:
void registerFunction(
const String & name,
Value creator,
- CaseSensitiveness case_sensitiveness = CaseSensitive);
+ Case case_sensitiveness = Case::Sensitive);
/// Register how to transform from one aggregate function to other based on NullsAction
/// Registers them both ways:
diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp
index 16907e0f24f..7034e6373b1 100644
--- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp
+++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp
@@ -840,8 +840,8 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory)
AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true };
factory.registerFunction("groupArray", { createAggregateFunctionGroupArray, properties });
- factory.registerAlias("array_agg", "groupArray", AggregateFunctionFactory::CaseInsensitive);
- factory.registerAliasUnchecked("array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive);
+ factory.registerAlias("array_agg", "groupArray", AggregateFunctionFactory::Case::Insensitive);
+ factory.registerAliasUnchecked("array_concat_agg", "groupArrayArray", AggregateFunctionFactory::Case::Insensitive);
factory.registerFunction("groupArraySample", { createAggregateFunctionGroupArraySample, properties });
factory.registerFunction("groupArrayLast", { createAggregateFunctionGroupArray, properties });
}
diff --git a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp
index 1c059dc52aa..5494ef74705 100644
--- a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp
+++ b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp
@@ -277,7 +277,7 @@ void registerAggregateFunctionGroupConcat(AggregateFunctionFactory & factory)
AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true };
factory.registerFunction("groupConcat", { createAggregateFunctionGroupConcat, properties });
- factory.registerAlias("group_concat", "groupConcat", AggregateFunctionFactory::CaseInsensitive);
+ factory.registerAlias("group_concat", "groupConcat", AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp
index 736cca11f1e..04eebe9f485 100644
--- a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp
+++ b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp
@@ -350,7 +350,7 @@ AggregateFunctionPtr createAggregateFunctionKolmogorovSmirnovTest(
void registerAggregateFunctionKolmogorovSmirnovTest(AggregateFunctionFactory & factory)
{
- factory.registerFunction("kolmogorovSmirnovTest", createAggregateFunctionKolmogorovSmirnovTest, AggregateFunctionFactory::CaseInsensitive);
+ factory.registerFunction("kolmogorovSmirnovTest", createAggregateFunctionKolmogorovSmirnovTest, AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionSecondMoment.cpp b/src/AggregateFunctions/AggregateFunctionSecondMoment.cpp
index 80fbe2511d9..4aa6a0a4429 100644
--- a/src/AggregateFunctions/AggregateFunctionSecondMoment.cpp
+++ b/src/AggregateFunctions/AggregateFunctionSecondMoment.cpp
@@ -15,11 +15,11 @@ void registerAggregateFunctionsStatisticsSecondMoment(AggregateFunctionFactory &
factory.registerFunction("stddevPop", createAggregateFunctionStatisticsUnary);
/// Synonyms for compatibility.
- factory.registerAlias("VAR_SAMP", "varSamp", AggregateFunctionFactory::CaseInsensitive);
- factory.registerAlias("VAR_POP", "varPop", AggregateFunctionFactory::CaseInsensitive);
- factory.registerAlias("STDDEV_SAMP", "stddevSamp", AggregateFunctionFactory::CaseInsensitive);
- factory.registerAlias("STDDEV_POP", "stddevPop", AggregateFunctionFactory::CaseInsensitive);
- factory.registerAlias("STD", "stddevPop", AggregateFunctionFactory::CaseInsensitive);
+ factory.registerAlias("VAR_SAMP", "varSamp", AggregateFunctionFactory::Case::Insensitive);
+ factory.registerAlias("VAR_POP", "varPop", AggregateFunctionFactory::Case::Insensitive);
+ factory.registerAlias("STDDEV_SAMP", "stddevSamp", AggregateFunctionFactory::Case::Insensitive);
+ factory.registerAlias("STDDEV_POP", "stddevPop", AggregateFunctionFactory::Case::Insensitive);
+ factory.registerAlias("STD", "stddevPop", AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionSum.cpp b/src/AggregateFunctions/AggregateFunctionSum.cpp
index e393cb6dd38..910e49f388d 100644
--- a/src/AggregateFunctions/AggregateFunctionSum.cpp
+++ b/src/AggregateFunctions/AggregateFunctionSum.cpp
@@ -72,7 +72,7 @@ AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const
void registerAggregateFunctionSum(AggregateFunctionFactory & factory)
{
- factory.registerFunction("sum", createAggregateFunctionSum, AggregateFunctionFactory::CaseInsensitive);
+ factory.registerFunction("sum", createAggregateFunctionSum, AggregateFunctionFactory::Case::Insensitive);
factory.registerFunction("sumWithOverflow", createAggregateFunctionSum);
factory.registerFunction("sumKahan", createAggregateFunctionSum);
}
diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp
index 26f756abe18..f949f6b7e4a 100644
--- a/src/AggregateFunctions/AggregateFunctionTopK.cpp
+++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp
@@ -535,9 +535,9 @@ void registerAggregateFunctionTopK(AggregateFunctionFactory & factory)
factory.registerFunction("topK", { createAggregateFunctionTopK, properties });
factory.registerFunction("topKWeighted", { createAggregateFunctionTopK, properties });
- factory.registerFunction("approx_top_k", { createAggregateFunctionTopK, properties }, AggregateFunctionFactory::CaseInsensitive);
- factory.registerFunction("approx_top_sum", { createAggregateFunctionTopK, properties }, AggregateFunctionFactory::CaseInsensitive);
- factory.registerAlias("approx_top_count", "approx_top_k", AggregateFunctionFactory::CaseInsensitive);
+ factory.registerFunction("approx_top_k", { createAggregateFunctionTopK, properties }, AggregateFunctionFactory::Case::Insensitive);
+ factory.registerFunction("approx_top_sum", { createAggregateFunctionTopK, properties }, AggregateFunctionFactory::Case::Insensitive);
+ factory.registerAlias("approx_top_count", "approx_top_k", AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionsMinMax.cpp b/src/AggregateFunctions/AggregateFunctionsMinMax.cpp
index 03e21c15a75..5fa9a4ff5d1 100644
--- a/src/AggregateFunctions/AggregateFunctionsMinMax.cpp
+++ b/src/AggregateFunctions/AggregateFunctionsMinMax.cpp
@@ -195,8 +195,8 @@ AggregateFunctionPtr createAggregateFunctionMinMax(
void registerAggregateFunctionsMinMax(AggregateFunctionFactory & factory)
{
- factory.registerFunction("min", createAggregateFunctionMinMax, AggregateFunctionFactory::CaseInsensitive);
- factory.registerFunction("max", createAggregateFunctionMinMax, AggregateFunctionFactory::CaseInsensitive);
+ factory.registerFunction("min", createAggregateFunctionMinMax, AggregateFunctionFactory::Case::Insensitive);
+ factory.registerFunction("max", createAggregateFunctionMinMax, AggregateFunctionFactory::Case::Insensitive);
}
}
diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp
index 854697bca9f..4a11eae15ea 100644
--- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp
+++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp
@@ -12,6 +12,7 @@
#include
#include
+#include
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())
{
args.push_back(rhs_literal->getValue());
+ tuple_element_types.push_back(rhs_literal->getResultType());
}
else
{
const auto * lhs_literal = equals_arguments[0]->as();
assert(lhs_literal);
args.push_back(lhs_literal->getValue());
+ tuple_element_types.push_back(lhs_literal->getResultType());
}
}
- auto rhs_node = std::make_shared(std::move(args));
+ auto rhs_node = std::make_shared(std::move(args), std::make_shared(std::move(tuple_element_types)));
auto in_function = std::make_shared("in");
diff --git a/src/Analyzer/Resolve/IdentifierResolver.cpp b/src/Analyzer/Resolve/IdentifierResolver.cpp
index 9dd8aa1a05f..447bf825836 100644
--- a/src/Analyzer/Resolve/IdentifierResolver.cpp
+++ b/src/Analyzer/Resolve/IdentifierResolver.cpp
@@ -1,6 +1,7 @@
#include
#include
#include
+#include
#include
#include
@@ -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(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;
}
diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp
index 92618dfe346..f3372e9d08f 100644
--- a/src/Analyzer/Resolve/QueryAnalyzer.cpp
+++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp
@@ -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;
}
}
diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp
index aef37021227..687096d0404 100644
--- a/src/Backups/BackupIO_S3.cpp
+++ b/src/Backups/BackupIO_S3.cpp
@@ -55,9 +55,9 @@ namespace
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
settings.auth_settings.region,
context->getRemoteHostFilter(),
- static_cast(global_settings.s3_max_redirects),
- static_cast(global_settings.s3_retry_attempts),
- global_settings.enable_s3_requests_logging,
+ static_cast(local_settings.s3_max_redirects),
+ static_cast(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,
diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp
index 2dc603a307f..051efd12a85 100644
--- a/src/Client/ClientBase.cpp
+++ b/src/Client/ClientBase.cpp
@@ -80,6 +80,7 @@
#include
#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(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> 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()->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();
+ 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>();
if (options.count("query_id"))
diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h
index d83ecdd75f9..0fcb3c4e7e1 100644
--- a/src/Client/ConnectionPool.h
+++ b/src/Client/ConnectionPool.h
@@ -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
{
diff --git a/src/Client/IConnections.h b/src/Client/IConnections.h
index ebc71511834..09211de53b0 100644
--- a/src/Client/IConnections.h
+++ b/src/Client/IConnections.h
@@ -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};
};
diff --git a/src/Client/LineReader.h b/src/Client/LineReader.h
index 0172bd7ec22..8c101401190 100644
--- a/src/Client/LineReader.h
+++ b/src/Client/LineReader.h
@@ -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;
diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h
index fb6fa1b55eb..b424c5b5aa3 100644
--- a/src/Client/LocalConnection.h
+++ b/src/Client/LocalConnection.h
@@ -175,4 +175,5 @@ private:
ReadBuffer * in;
};
+
}
diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp
index 207bf6c9e07..244eccf1ed9 100644
--- a/src/Client/MultiplexedConnections.cpp
+++ b/src/Client/MultiplexedConnections.cpp
@@ -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.
diff --git a/src/Client/ReplxxLineReader.cpp b/src/Client/ReplxxLineReader.cpp
index 46600168695..3b3508d1a58 100644
--- a/src/Client/ReplxxLineReader.cpp
+++ b/src/Client/ReplxxLineReader.cpp
@@ -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.
diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp
index adcd42b16e9..9c9dade3dd8 100644
--- a/src/Columns/ColumnObject.cpp
+++ b/src/Columns/ColumnObject.cpp
@@ -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);
+}
}
diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h
index fadf2e18779..7470dfa6302 100644
--- a/src/Columns/ColumnObject.h
+++ b/src/Columns/ColumnObject.h
@@ -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(); }
diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp
index bfc85559fe8..bcc5d187047 100644
--- a/src/Common/Allocator.cpp
+++ b/src/Common/Allocator.cpp
@@ -187,12 +187,9 @@ void * Allocator::realloc(void * buf, size_t old_size,
#if USE_GWP_ASAN
if (unlikely(GWPAsan::GuardedAlloc.shouldSample()))
{
+ auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
if (void * ptr = GWPAsan::GuardedAlloc.allocate(new_size, alignment))
{
- auto trace_free = CurrentMemoryTracker::free(old_size);
- auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
- trace_free.onFree(buf, old_size);
-
memcpy(ptr, buf, std::min(old_size, new_size));
free(buf, old_size);
trace_alloc.onAlloc(buf, new_size);
@@ -209,6 +206,7 @@ void * Allocator::realloc(void * buf, size_t old_size,
}
else
{
+ [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size);
ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed);
}
}
@@ -231,13 +229,17 @@ void * Allocator::realloc(void * buf, size_t old_size,
if (alignment <= MALLOC_MIN_ALIGNMENT)
{
/// Resize malloc'd memory region with no special alignment requirement.
- auto trace_free = CurrentMemoryTracker::free(old_size);
+ /// Realloc can do 2 possible things:
+ /// - expand existing memory region
+ /// - allocate new memory block and free the old one
+ /// Because we don't know which option will be picked we need to make sure there is enough
+ /// memory for all options
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
- trace_free.onFree(buf, old_size);
void * new_buf = ::realloc(buf, new_size);
if (nullptr == new_buf)
{
+ [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size);
throw DB::ErrnoException(
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY,
"Allocator: Cannot realloc from {} to {}",
@@ -246,6 +248,8 @@ void * Allocator::realloc(void * buf, size_t old_size,
}
buf = new_buf;
+ auto trace_free = CurrentMemoryTracker::free(old_size);
+ trace_free.onFree(buf, old_size);
trace_alloc.onAlloc(buf, new_size);
if constexpr (clear_memory)
diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp
index cfb273b9058..6264eb03106 100644
--- a/src/Common/AsyncLoader.cpp
+++ b/src/Common/AsyncLoader.cpp
@@ -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(
init.metric_threads,
init.metric_active_threads,
@@ -56,17 +57,16 @@ AsyncLoader::Pool::Pool(const AsyncLoader::PoolInitializer & init)
/* max_threads = */ std::numeric_limits::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)
diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h
index 42707a4ee91..05b809aceae 100644
--- a/src/Common/AsyncLoader.h
+++ b/src/Common/AsyncLoader.h
@@ -365,11 +365,11 @@ private:
{
const String name;
const Priority priority;
- std::unique_ptr thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools.
std::map 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 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 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;
diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp
index d36c7fd08aa..02bde0d80b7 100644
--- a/src/Common/CgroupsMemoryUsageObserver.cpp
+++ b/src/Common/CgroupsMemoryUsageObserver.cpp
@@ -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 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 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 getCgroupsV2Path()
std::optional getCgroupsV1Path()
{
auto path = default_cgroups_mount / "memory/memory.stat";
- if (!std::filesystem::exists(path))
+ if (!fs::exists(path))
return {};
return {default_cgroups_mount / "memory"};
}
diff --git a/src/Common/CollectionOfDerived.h b/src/Common/CollectionOfDerived.h
new file mode 100644
index 00000000000..97c0c3fbc06
--- /dev/null
+++ b/src/Common/CollectionOfDerived.h
@@ -0,0 +1,184 @@
+#pragma once
+
+#include
+
+#include
+
+#include
+#include
+#include
+#include
+#include
+
+
+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 CollectionOfDerivedItems
+{
+public:
+ using Self = CollectionOfDerivedItems;
+ using ItemPtr = std::shared_ptr;
+
+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;
+
+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
+ void add(std::shared_ptr info)
+ {
+ static_assert(std::is_base_of_v, "Template parameter must inherit items base class");
+ return addImpl(std::type_index(typeid(T)), std::move(info));
+ }
+
+ template
+ std::shared_ptr get() const
+ {
+ static_assert(std::is_base_of_v, "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(it->ptr);
+ chassert(cast);
+ return cast;
+ }
+
+ template
+ std::shared_ptr extract()
+ {
+ static_assert(std::is_base_of_v, "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(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;
+};
+
+}
diff --git a/src/Common/IFactoryWithAliases.h b/src/Common/IFactoryWithAliases.h
index 74d4b6e3bcb..431e5c7b733 100644
--- a/src/Common/IFactoryWithAliases.h
+++ b/src/Common/IFactoryWithAliases.h
@@ -39,16 +39,16 @@ protected:
public:
/// For compatibility with SQL, it's possible to specify that certain function name is case insensitive.
- enum CaseSensitiveness
+ enum Case
{
- CaseSensitive,
- CaseInsensitive
+ Sensitive,
+ Insensitive
};
/** Register additional name for value
* real_name have to be already registered.
*/
- void registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive)
+ void registerAlias(const String & alias_name, const String & real_name, Case case_sensitiveness = Sensitive)
{
const auto & creator_map = getMap();
const auto & case_insensitive_creator_map = getCaseInsensitiveMap();
@@ -66,12 +66,12 @@ public:
}
/// We need sure the real_name exactly exists when call the function directly.
- void registerAliasUnchecked(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive)
+ void registerAliasUnchecked(const String & alias_name, const String & real_name, Case case_sensitiveness = Sensitive)
{
String alias_name_lowercase = Poco::toLower(alias_name);
const String factory_name = getFactoryName();
- if (case_sensitiveness == CaseInsensitive)
+ if (case_sensitiveness == Insensitive)
{
if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second)
throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", factory_name, alias_name);
diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h
index 188ec4ed764..9baa1c7ffc4 100644
--- a/src/Common/ProgressIndication.h
+++ b/src/Common/ProgressIndication.h
@@ -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
diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp
index 7e18a93e6ed..34a1add2f0e 100644
--- a/src/Common/getNumberOfPhysicalCPUCores.cpp
+++ b/src/Common/getNumberOfPhysicalCPUCores.cpp
@@ -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())
{
diff --git a/src/Common/memory.h b/src/Common/memory.h
index caa0418fa56..dbef069b408 100644
--- a/src/Common/memory.h
+++ b/src/Common/memory.h
@@ -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()))
diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp
index e8151fbe201..80e05fc4ea0 100644
--- a/src/Common/new_delete.cpp
+++ b/src/Common/new_delete.cpp
@@ -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;
}
diff --git a/src/Core/Settings.h b/src/Core/Settings.h
index 52fa28a4481..f19f2827b62 100644
--- a/src/Core/Settings.h
+++ b/src/Core/Settings.h
@@ -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) \
diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp
index b9b72209103..73fb28dc67a 100644
--- a/src/Core/SettingsChangesHistory.cpp
+++ b/src/Core/SettingsChangesHistory.cpp
@@ -77,6 +77,7 @@ static std::initializer_list()); }, DataTypeFactory::CaseInsensitive);
+ factory.registerSimpleDataType("Date", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::Case::Insensitive);
}
}
diff --git a/src/DataTypes/DataTypeDate32.cpp b/src/DataTypes/DataTypeDate32.cpp
index 343e498d303..b2b8e7c0c1c 100644
--- a/src/DataTypes/DataTypeDate32.cpp
+++ b/src/DataTypes/DataTypeDate32.cpp
@@ -24,7 +24,7 @@ Field DataTypeDate32::getDefault() const
void registerDataTypeDate32(DataTypeFactory & factory)
{
factory.registerSimpleDataType(
- "Date32", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::CaseInsensitive);
+ "Date32", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::Case::Insensitive);
}
}
diff --git a/src/DataTypes/DataTypeDomainBool.cpp b/src/DataTypes/DataTypeDomainBool.cpp
index 3d19b6262d8..30dbba2d8c0 100644
--- a/src/DataTypes/DataTypeDomainBool.cpp
+++ b/src/DataTypes/DataTypeDomainBool.cpp
@@ -15,8 +15,8 @@ void registerDataTypeDomainBool(DataTypeFactory & factory)
std::make_unique("Bool"), std::make_unique(type->getDefaultSerialization())));
});
- factory.registerAlias("bool", "Bool", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("boolean", "Bool", DataTypeFactory::CaseInsensitive);
+ factory.registerAlias("bool", "Bool", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("boolean", "Bool", DataTypeFactory::Case::Insensitive);
}
}
diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp
index a1d5e4b39b7..08e0c0d2045 100644
--- a/src/DataTypes/DataTypeEnum.cpp
+++ b/src/DataTypes/DataTypeEnum.cpp
@@ -318,7 +318,7 @@ void registerDataTypeEnum(DataTypeFactory & factory)
factory.registerDataType("Enum", create);
/// MySQL
- factory.registerAlias("ENUM", "Enum", DataTypeFactory::CaseInsensitive);
+ factory.registerAlias("ENUM", "Enum", DataTypeFactory::Case::Insensitive);
}
}
diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp
index 07dc4395bfe..af37cde2846 100644
--- a/src/DataTypes/DataTypeFactory.cpp
+++ b/src/DataTypes/DataTypeFactory.cpp
@@ -175,7 +175,7 @@ DataTypePtr DataTypeFactory::getCustom(DataTypeCustomDescPtr customization) cons
}
-void DataTypeFactory::registerDataType(const String & family_name, Value creator, CaseSensitiveness case_sensitiveness)
+void DataTypeFactory::registerDataType(const String & family_name, Value creator, Case case_sensitiveness)
{
if (creator == nullptr)
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type family {} has been provided a null constructor", family_name);
@@ -189,12 +189,12 @@ void DataTypeFactory::registerDataType(const String & family_name, Value creator
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type family name '{}' is not unique",
family_name);
- if (case_sensitiveness == CaseInsensitive
+ if (case_sensitiveness == Case::Insensitive
&& !case_insensitive_data_types.emplace(family_name_lowercase, creator).second)
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the case insensitive data type family name '{}' is not unique", family_name);
}
-void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness)
+void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, Case case_sensitiveness)
{
if (creator == nullptr)
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type {} has been provided a null constructor",
@@ -208,7 +208,7 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator
}, case_sensitiveness);
}
-void DataTypeFactory::registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, CaseSensitiveness case_sensitiveness)
+void DataTypeFactory::registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, Case case_sensitiveness)
{
registerDataType(family_name, [creator](const ASTPtr & ast)
{
@@ -219,7 +219,7 @@ void DataTypeFactory::registerDataTypeCustom(const String & family_name, Creator
}, case_sensitiveness);
}
-void DataTypeFactory::registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, CaseSensitiveness case_sensitiveness)
+void DataTypeFactory::registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, Case case_sensitiveness)
{
registerDataTypeCustom(name, [name, creator](const ASTPtr & ast)
{
diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h
index 86e0203358d..edba9886d1c 100644
--- a/src/DataTypes/DataTypeFactory.h
+++ b/src/DataTypes/DataTypeFactory.h
@@ -41,16 +41,16 @@ public:
DataTypePtr tryGet(const ASTPtr & ast) const;
/// Register a type family by its name.
- void registerDataType(const String & family_name, Value creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
+ void registerDataType(const String & family_name, Value creator, Case case_sensitiveness = Case::Sensitive);
/// Register a simple data type, that have no parameters.
- void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
+ void registerSimpleDataType(const String & name, SimpleCreator creator, Case case_sensitiveness = Case::Sensitive);
/// Register a customized type family
- void registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
+ void registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, Case case_sensitiveness = Case::Sensitive);
/// Register a simple customized data type
- void registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
+ void registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, Case case_sensitiveness = Case::Sensitive);
private:
template
diff --git a/src/DataTypes/DataTypeFixedString.cpp b/src/DataTypes/DataTypeFixedString.cpp
index 85af59e852d..080ff8826a5 100644
--- a/src/DataTypes/DataTypeFixedString.cpp
+++ b/src/DataTypes/DataTypeFixedString.cpp
@@ -64,7 +64,7 @@ void registerDataTypeFixedString(DataTypeFactory & factory)
factory.registerDataType("FixedString", create);
/// Compatibility alias.
- factory.registerAlias("BINARY", "FixedString", DataTypeFactory::CaseInsensitive);
+ factory.registerAlias("BINARY", "FixedString", DataTypeFactory::Case::Insensitive);
}
}
diff --git a/src/DataTypes/DataTypeIPv4andIPv6.cpp b/src/DataTypes/DataTypeIPv4andIPv6.cpp
index 4c0b45f472a..de11cc50107 100644
--- a/src/DataTypes/DataTypeIPv4andIPv6.cpp
+++ b/src/DataTypes/DataTypeIPv4andIPv6.cpp
@@ -9,9 +9,9 @@ namespace DB
void registerDataTypeIPv4andIPv6(DataTypeFactory & factory)
{
factory.registerSimpleDataType("IPv4", [] { return DataTypePtr(std::make_shared()); });
- factory.registerAlias("INET4", "IPv4", DataTypeFactory::CaseInsensitive);
+ factory.registerAlias("INET4", "IPv4", DataTypeFactory::Case::Insensitive);
factory.registerSimpleDataType("IPv6", [] { return DataTypePtr(std::make_shared()); });
- factory.registerAlias("INET6", "IPv6", DataTypeFactory::CaseInsensitive);
+ factory.registerAlias("INET6", "IPv6", DataTypeFactory::Case::Insensitive);
}
}
diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp
index 720436d0e0d..5636a46373f 100644
--- a/src/DataTypes/DataTypeObject.cpp
+++ b/src/DataTypes/DataTypeObject.cpp
@@ -76,7 +76,7 @@ void registerDataTypeObject(DataTypeFactory & factory)
factory.registerDataType("Object", create);
factory.registerSimpleDataType("JSON",
[] { return std::make_shared("JSON", false); },
- DataTypeFactory::CaseInsensitive);
+ DataTypeFactory::Case::Insensitive);
}
}
diff --git a/src/DataTypes/DataTypeString.cpp b/src/DataTypes/DataTypeString.cpp
index 95e49420009..ca65fb42cc8 100644
--- a/src/DataTypes/DataTypeString.cpp
+++ b/src/DataTypes/DataTypeString.cpp
@@ -62,38 +62,38 @@ void registerDataTypeString(DataTypeFactory & factory)
/// These synonims are added for compatibility.
- factory.registerAlias("CHAR", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("NCHAR", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("CHARACTER", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("VARCHAR", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("NVARCHAR", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("VARCHAR2", "String", DataTypeFactory::CaseInsensitive); /// Oracle
- factory.registerAlias("TEXT", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("TINYTEXT", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("LONGTEXT", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("BLOB", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("CLOB", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("TINYBLOB", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("LONGBLOB", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("BYTEA", "String", DataTypeFactory::CaseInsensitive); /// PostgreSQL
+ factory.registerAlias("CHAR", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("NCHAR", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("CHARACTER", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("VARCHAR", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("NVARCHAR", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("VARCHAR2", "String", DataTypeFactory::Case::Insensitive); /// Oracle
+ factory.registerAlias("TEXT", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("TINYTEXT", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("LONGTEXT", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("BLOB", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("CLOB", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("TINYBLOB", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("LONGBLOB", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("BYTEA", "String", DataTypeFactory::Case::Insensitive); /// PostgreSQL
- factory.registerAlias("CHARACTER LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("CHARACTER VARYING", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("CHAR LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("CHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("NATIONAL CHAR", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("NATIONAL CHARACTER", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("NATIONAL CHARACTER LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("NATIONAL CHARACTER VARYING", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("NATIONAL CHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("NCHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("NCHAR LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("BINARY LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("BINARY VARYING", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("VARBINARY", "String", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("GEOMETRY", "String", DataTypeFactory::CaseInsensitive); //mysql
+ factory.registerAlias("CHARACTER LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("CHARACTER VARYING", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("CHAR LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("CHAR VARYING", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("NATIONAL CHAR", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("NATIONAL CHARACTER", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("NATIONAL CHARACTER LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("NATIONAL CHARACTER VARYING", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("NATIONAL CHAR VARYING", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("NCHAR VARYING", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("NCHAR LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("BINARY LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("BINARY VARYING", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("VARBINARY", "String", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("GEOMETRY", "String", DataTypeFactory::Case::Insensitive); //mysql
}
}
diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp
index 77a7a3e7237..a427fd0717a 100644
--- a/src/DataTypes/DataTypesDecimal.cpp
+++ b/src/DataTypes/DataTypesDecimal.cpp
@@ -364,15 +364,15 @@ template class DataTypeDecimal;
void registerDataTypeDecimal(DataTypeFactory & factory)
{
- factory.registerDataType("Decimal32", createExact, DataTypeFactory::CaseInsensitive);
- factory.registerDataType("Decimal64", createExact, DataTypeFactory::CaseInsensitive);
- factory.registerDataType("Decimal128", createExact, DataTypeFactory::CaseInsensitive);
- factory.registerDataType("Decimal256", createExact, DataTypeFactory::CaseInsensitive);
+ factory.registerDataType("Decimal32", createExact, DataTypeFactory::Case::Insensitive);
+ factory.registerDataType("Decimal64", createExact, DataTypeFactory::Case::Insensitive);
+ factory.registerDataType("Decimal128", createExact, DataTypeFactory::Case::Insensitive);
+ factory.registerDataType("Decimal256", createExact, DataTypeFactory::Case::Insensitive);
- factory.registerDataType("Decimal", create, DataTypeFactory::CaseInsensitive);
- factory.registerAlias("DEC", "Decimal", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("NUMERIC", "Decimal", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("FIXED", "Decimal", DataTypeFactory::CaseInsensitive);
+ factory.registerDataType("Decimal", create, DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("DEC", "Decimal", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("NUMERIC", "Decimal", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("FIXED", "Decimal", DataTypeFactory::Case::Insensitive);
}
}
diff --git a/src/DataTypes/DataTypesNumber.cpp b/src/DataTypes/DataTypesNumber.cpp
index 99446d24eed..72020b0a5aa 100644
--- a/src/DataTypes/DataTypesNumber.cpp
+++ b/src/DataTypes/DataTypesNumber.cpp
@@ -65,41 +65,41 @@ void registerDataTypeNumbers(DataTypeFactory & factory)
/// These synonyms are added for compatibility.
- factory.registerAlias("TINYINT", "Int8", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("INT1", "Int8", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("BYTE", "Int8", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("TINYINT SIGNED", "Int8", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("INT1 SIGNED", "Int8", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("SMALLINT SIGNED", "Int16", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("INT", "Int32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("INTEGER", "Int32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("MEDIUMINT", "Int32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("MEDIUMINT SIGNED", "Int32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("INT SIGNED", "Int32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("INTEGER SIGNED", "Int32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("BIGINT", "Int64", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("SIGNED", "Int64", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("BIGINT SIGNED", "Int64", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("TIME", "Int64", DataTypeFactory::CaseInsensitive);
+ factory.registerAlias("TINYINT", "Int8", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("INT1", "Int8", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("BYTE", "Int8", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("TINYINT SIGNED", "Int8", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("INT1 SIGNED", "Int8", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("SMALLINT SIGNED", "Int16", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("INT", "Int32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("INTEGER", "Int32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("MEDIUMINT", "Int32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("MEDIUMINT SIGNED", "Int32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("INT SIGNED", "Int32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("INTEGER SIGNED", "Int32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("BIGINT", "Int64", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("SIGNED", "Int64", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("BIGINT SIGNED", "Int64", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("TIME", "Int64", DataTypeFactory::Case::Insensitive);
- factory.registerAlias("TINYINT UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("INT1 UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("SMALLINT UNSIGNED", "UInt16", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("YEAR", "UInt16", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("MEDIUMINT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("BIT", "UInt64", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("SET", "UInt64", DataTypeFactory::CaseInsensitive);
+ factory.registerAlias("TINYINT UNSIGNED", "UInt8", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("INT1 UNSIGNED", "UInt8", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("SMALLINT UNSIGNED", "UInt16", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("YEAR", "UInt16", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("MEDIUMINT UNSIGNED", "UInt32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("UNSIGNED", "UInt64", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("BIT", "UInt64", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("SET", "UInt64", DataTypeFactory::Case::Insensitive);
- factory.registerAlias("FLOAT", "Float32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("REAL", "Float32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("SINGLE", "Float32", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive);
- factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::CaseInsensitive);
+ factory.registerAlias("FLOAT", "Float32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("REAL", "Float32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("SINGLE", "Float32", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::Case::Insensitive);
+ factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::Case::Insensitive);
}
/// Explicit template instantiations.
diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp
index 802356cc108..84a52d4affb 100644
--- a/src/DataTypes/registerDataTypeDateTime.cpp
+++ b/src/DataTypes/registerDataTypeDateTime.cpp
@@ -108,11 +108,11 @@ static DataTypePtr create64(const ASTPtr & arguments)
void registerDataTypeDateTime(DataTypeFactory & factory)
{
- factory.registerDataType("DateTime", create, DataTypeFactory::CaseInsensitive);
- factory.registerDataType("DateTime32", create32, DataTypeFactory::CaseInsensitive);
- factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive);
+ factory.registerDataType("DateTime", create, DataTypeFactory::Case::Insensitive);
+ factory.registerDataType("DateTime32", create32, DataTypeFactory::Case::Insensitive);
+ factory.registerDataType("DateTime64", create64, DataTypeFactory::Case::Insensitive);
- factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive);
+ factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::Case::Insensitive);
}
}
diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp
index 9e7004e72a9..d86e29ca915 100644
--- a/src/Databases/DatabaseAtomic.cpp
+++ b/src/Databases/DatabaseAtomic.cpp
@@ -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)
diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h
index b59edd479ba..4a4ccfa2573 100644
--- a/src/Databases/DatabaseAtomic.h
+++ b/src/Databases/DatabaseAtomic.h
@@ -1,7 +1,8 @@
#pragma once
-#include
#include
+#include
+#include
namespace DB
diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp
index da942cebf8f..5017c9b25cb 100644
--- a/src/Databases/DatabaseLazy.cpp
+++ b/src/Databases/DatabaseLazy.cpp
@@ -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);
}
diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp
index 261a917c595..f419f5811a1 100644
--- a/src/Databases/DatabaseOnDisk.cpp
+++ b/src/Databases/DatabaseOnDisk.cpp
@@ -23,6 +23,7 @@
#include
#include
#include
+#include
#include
#include
#include
@@ -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)
{
diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp
index 6555c4444e2..3ab5d3fa697 100644
--- a/src/Databases/DatabaseOrdinary.cpp
+++ b/src/Databases/DatabaseOrdinary.cpp
@@ -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 unique_names;
diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h
index ef00ac8fdfa..c2c5775e5ab 100644
--- a/src/Databases/DatabaseOrdinary.h
+++ b/src/Databases/DatabaseOrdinary.h
@@ -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 load_table TSA_GUARDED_BY(mutex);
std::unordered_map startup_table TSA_GUARDED_BY(mutex);
diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp
index 6426123bb4f..fe0baf30e57 100644
--- a/src/Databases/DatabasesCommon.cpp
+++ b/src/Databases/DatabasesCommon.cpp
@@ -2,12 +2,9 @@
#include
#include
-#include
-#include
-#include
-#include
#include
#include
+#include
#include
#include
#include
@@ -16,6 +13,10 @@
#include
#include
#include
+#include
+#include
+#include
+#include
namespace DB
@@ -237,6 +238,24 @@ DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPt
return std::make_unique(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(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(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()
diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h
index 2eecf8a564f..1ca49e90c23 100644
--- a/src/Databases/DatabasesCommon.h
+++ b/src/Databases/DatabasesCommon.h
@@ -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> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override;
void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr 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;
};
diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h
index b00f2fe4baf..ce5a52b1b0f 100644
--- a/src/Databases/IDatabase.h
+++ b/src/Databases/IDatabase.h
@@ -5,20 +5,22 @@
#include
#include
#include
+#include
+#include
#include
#include
-#include
#include
+#include
#include
#include
-#include
#include
#include
+#include