-
AST Fuzzer for PR #${PR_TO_TEST} @ ${SHA_TO_TEST}
+
AST Fuzzer for PR #${PR_TO_TEST} @ ${SHA_TO_TEST}
-runlog.log
-fuzzer.log
-server.log.gz
-main.log
-${CORE_LINK}
+ run.log
+ fuzzer.log
+ server.log.gz
+ main.log
+ ${CORE_LINK}
-Test name | Test status | Description |
-AST Fuzzer | $(cat status.txt) | $(cat description.txt) |
+
+ Test name |
+ Test status |
+ Description |
+
+
+ AST Fuzzer |
+ $(cat status.txt) |
+ $(
+ clickhouse-local --input-format RawBLOB --output-format RawBLOB --query "SELECT encodeXMLComponent(*) FROM table" < description.txt || cat description.txt
+ ) |
+
+
+ $(
+ clickhouse-local --input-format RawBLOB --output-format RawBLOB --query "SELECT encodeXMLComponent(*) FROM table" < fatal.log || cat fatal.log
+ ) |
+
diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile
index 84c04dd03ec..ccfd63c8ed0 100644
--- a/docker/test/integration/runner/Dockerfile
+++ b/docker/test/integration/runner/Dockerfile
@@ -83,6 +83,7 @@ RUN python3 -m pip install \
pytest \
pytest-order==1.0.0 \
pytest-timeout \
+ pytest-random \
pytest-xdist \
pytest-repeat \
pytz \
diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py
index cb23372d31f..76e4c5ad8c1 100755
--- a/docker/test/performance-comparison/perf.py
+++ b/docker/test/performance-comparison/perf.py
@@ -297,6 +297,7 @@ if not args.use_existing_tables:
# Let's sync the data to avoid writeback affects performance
os.system("sync")
+reportStageEnd("sync")
# By default, test all queries.
queries_to_run = range(0, len(test_queries))
diff --git a/docker/test/split_build_smoke_test/Dockerfile b/docker/test/split_build_smoke_test/Dockerfile
deleted file mode 100644
index 5f84eb42216..00000000000
--- a/docker/test/split_build_smoke_test/Dockerfile
+++ /dev/null
@@ -1,9 +0,0 @@
-# rebuild in #33610
-# docker build -t clickhouse/split-build-smoke-test .
-ARG FROM_TAG=latest
-FROM clickhouse/binary-builder:$FROM_TAG
-
-COPY run.sh /run.sh
-COPY process_split_build_smoke_test_result.py /
-
-CMD /run.sh
diff --git a/docker/test/split_build_smoke_test/process_split_build_smoke_test_result.py b/docker/test/split_build_smoke_test/process_split_build_smoke_test_result.py
deleted file mode 100755
index b5bc82e6818..00000000000
--- a/docker/test/split_build_smoke_test/process_split_build_smoke_test_result.py
+++ /dev/null
@@ -1,64 +0,0 @@
-#!/usr/bin/env python3
-
-import os
-import logging
-import argparse
-import csv
-
-RESULT_LOG_NAME = "run.log"
-
-
-def process_result(result_folder):
-
- status = "success"
- description = "Server started and responded"
- summary = [("Smoke test", "OK")]
- with open(os.path.join(result_folder, RESULT_LOG_NAME), "r") as run_log:
- lines = run_log.read().split("\n")
- if not lines or lines[0].strip() != "OK":
- status = "failure"
- logging.info("Lines is not ok: %s", str("\n".join(lines)))
- summary = [("Smoke test", "FAIL")]
- description = "Server failed to respond, see result in logs"
-
- result_logs = []
- server_log_path = os.path.join(result_folder, "clickhouse-server.log")
- stderr_log_path = os.path.join(result_folder, "stderr.log")
- client_stderr_log_path = os.path.join(result_folder, "clientstderr.log")
-
- if os.path.exists(server_log_path):
- result_logs.append(server_log_path)
-
- if os.path.exists(stderr_log_path):
- result_logs.append(stderr_log_path)
-
- if os.path.exists(client_stderr_log_path):
- result_logs.append(client_stderr_log_path)
-
- return status, description, summary, result_logs
-
-
-def write_results(results_file, status_file, results, status):
- with open(results_file, "w") as f:
- out = csv.writer(f, delimiter="\t")
- out.writerows(results)
- with open(status_file, "w") as f:
- out = csv.writer(f, delimiter="\t")
- out.writerow(status)
-
-
-if __name__ == "__main__":
- logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s")
- parser = argparse.ArgumentParser(
- description="ClickHouse script for parsing results of split build smoke test"
- )
- parser.add_argument("--in-results-dir", default="/test_output/")
- parser.add_argument("--out-results-file", default="/test_output/test_results.tsv")
- parser.add_argument("--out-status-file", default="/test_output/check_status.tsv")
- args = parser.parse_args()
-
- state, description, test_results, logs = process_result(args.in_results_dir)
- logging.info("Result parsed")
- status = (state, description)
- write_results(args.out_results_file, args.out_status_file, test_results, status)
- logging.info("Result written")
diff --git a/docker/test/split_build_smoke_test/run.sh b/docker/test/split_build_smoke_test/run.sh
deleted file mode 100755
index b565d7a481e..00000000000
--- a/docker/test/split_build_smoke_test/run.sh
+++ /dev/null
@@ -1,22 +0,0 @@
-#!/bin/bash
-
-set -x
-
-install_and_run_server() {
- mkdir /unpacked
- tar -xzf /package_folder/shared_build.tgz -C /unpacked --strip 1
- LD_LIBRARY_PATH=/unpacked /unpacked/clickhouse-server --config /unpacked/config/config.xml >/test_output/stderr.log 2>&1 &
-}
-
-run_client() {
- for i in {1..100}; do
- sleep 1
- LD_LIBRARY_PATH=/unpacked /unpacked/clickhouse-client --query "select 'OK'" > /test_output/run.log 2> /test_output/clientstderr.log && break
- [[ $i == 100 ]] && echo 'FAIL'
- done
-}
-
-install_and_run_server
-run_client
-mv /var/log/clickhouse-server/clickhouse-server.log /test_output/clickhouse-server.log
-/process_split_build_smoke_test_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv
diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh
index a1de7f2d6ed..c756ce4669d 100755
--- a/docker/test/stateless/setup_minio.sh
+++ b/docker/test/stateless/setup_minio.sh
@@ -1,90 +1,151 @@
#!/bin/bash
-USAGE='Usage for local run:
+set -euxf -o pipefail
-./docker/test/stateless/setup_minio.sh { stateful | stateless } ./tests/
+export MINIO_ROOT_USER=${MINIO_ROOT_USER:-clickhouse}
+export MINIO_ROOT_PASSWORD=${MINIO_ROOT_PASSWORD:-clickhouse}
-'
+usage() {
+ echo $"Usage: $0
(default path: /usr/share/clickhouse-test)"
+ exit 1
+}
-set -e -x -a -u
-
-TEST_TYPE="$1"
-shift
-
-case $TEST_TYPE in
- stateless) QUERY_DIR=0_stateless ;;
- stateful) QUERY_DIR=1_stateful ;;
- *) echo "unknown test type $TEST_TYPE"; echo "${USAGE}"; exit 1 ;;
-esac
-
-ls -lha
-
-mkdir -p ./minio_data
-
-if [ ! -f ./minio ]; then
- MINIO_SERVER_VERSION=${MINIO_SERVER_VERSION:-2022-09-07T22-25-02Z}
- MINIO_CLIENT_VERSION=${MINIO_CLIENT_VERSION:-2022-08-28T20-08-11Z}
- case $(uname -m) in
- x86_64) BIN_ARCH=amd64 ;;
- aarch64) BIN_ARCH=arm64 ;;
- *) echo "unknown architecture $(uname -m)"; exit 1 ;;
- esac
- echo 'MinIO binary not found, downloading...'
-
- BINARY_TYPE=$(uname -s | tr '[:upper:]' '[:lower:]')
-
- wget "https://dl.min.io/server/minio/release/${BINARY_TYPE}-${BIN_ARCH}/archive/minio.RELEASE.${MINIO_SERVER_VERSION}" -O ./minio \
- && wget "https://dl.min.io/client/mc/release/${BINARY_TYPE}-${BIN_ARCH}/archive/mc.RELEASE.${MINIO_CLIENT_VERSION}" -O ./mc \
- && chmod +x ./mc ./minio
-fi
-
-MINIO_ROOT_USER=${MINIO_ROOT_USER:-clickhouse}
-MINIO_ROOT_PASSWORD=${MINIO_ROOT_PASSWORD:-clickhouse}
-
-./minio --version
-./minio server --address ":11111" ./minio_data &
-
-i=0
-while ! curl -v --silent http://localhost:11111 2>&1 | grep AccessDenied
-do
- if [[ $i == 60 ]]; then
- echo "Failed to setup minio"
- exit 0
+check_arg() {
+ local query_dir
+ if [ ! $# -eq 1 ]; then
+ if [ ! $# -eq 2 ]; then
+ echo "ERROR: need either one or two arguments, (default path: /usr/share/clickhouse-test)"
+ usage
+ fi
fi
- echo "Trying to connect to minio"
- sleep 1
- i=$((i + 1))
-done
+ case "$1" in
+ stateless)
+ query_dir="0_stateless"
+ ;;
+ stateful)
+ query_dir="1_stateful"
+ ;;
+ *)
+ echo "unknown test type ${test_type}"
+ usage
+ ;;
+ esac
+ echo ${query_dir}
+}
-lsof -i :11111
+find_arch() {
+ local arch
+ case $(uname -m) in
+ x86_64)
+ arch="amd64"
+ ;;
+ aarch64)
+ arch="arm64"
+ ;;
+ *)
+ echo "unknown architecture $(uname -m)";
+ exit 1
+ ;;
+ esac
+ echo ${arch}
+}
-sleep 5
+find_os() {
+ local os
+ os=$(uname -s | tr '[:upper:]' '[:lower:]')
+ echo "${os}"
+}
-./mc alias set clickminio http://localhost:11111 clickhouse clickhouse
-./mc admin user add clickminio test testtest
-./mc admin policy set clickminio readwrite user=test
-./mc mb clickminio/test
-if [ "$TEST_TYPE" = "stateless" ]; then
- ./mc policy set public clickminio/test
-fi
+download_minio() {
+ local os
+ local arch
+ local minio_server_version=${MINIO_SERVER_VERSION:-2022-09-07T22-25-02Z}
+ local minio_client_version=${MINIO_CLIENT_VERSION:-2022-08-28T20-08-11Z}
+ os=$(find_os)
+ arch=$(find_arch)
+ wget "https://dl.min.io/server/minio/release/${os}-${arch}/archive/minio.RELEASE.${minio_server_version}" -O ./minio
+ wget "https://dl.min.io/client/mc/release/${os}-${arch}/archive/mc.RELEASE.${minio_client_version}" -O ./mc
+ chmod +x ./mc ./minio
+}
-# Upload data to Minio. By default after unpacking all tests will in
-# /usr/share/clickhouse-test/queries
+start_minio() {
+ mkdir -p ./minio_data
+ ./minio --version
+ ./minio server --address ":11111" ./minio_data &
+ wait_for_it
+ lsof -i :11111
+ sleep 5
+}
-TEST_PATH=${1:-/usr/share/clickhouse-test}
-MINIO_DATA_PATH=${TEST_PATH}/queries/${QUERY_DIR}/data_minio
+setup_minio() {
+ local test_type=$1
+ ./mc alias set clickminio http://localhost:11111 clickhouse clickhouse
+ ./mc admin user add clickminio test testtest
+ ./mc admin policy set clickminio readwrite user=test
+ ./mc mb clickminio/test
+ if [ "$test_type" = "stateless" ]; then
+ ./mc policy set public clickminio/test
+ fi
+}
-# Iterating over globs will cause redudant FILE variale to be a path to a file, not a filename
-# shellcheck disable=SC2045
-for FILE in $(ls "${MINIO_DATA_PATH}"); do
- echo "$FILE";
- ./mc cp "${MINIO_DATA_PATH}"/"$FILE" clickminio/test/"$FILE";
-done
+# uploads data to minio, by default after unpacking all tests
+# will be in /usr/share/clickhouse-test/queries
+upload_data() {
+ local query_dir=$1
+ local test_path=$2
+ local data_path=${test_path}/queries/${query_dir}/data_minio
-mkdir -p ~/.aws
-cat <> ~/.aws/credentials
+ # iterating over globs will cause redundant file variable to be
+ # a path to a file, not a filename
+ # shellcheck disable=SC2045
+ for file in $(ls "${data_path}"); do
+ echo "${file}";
+ ./mc cp "${data_path}"/"${file}" clickminio/test/"${file}";
+ done
+}
+
+setup_aws_credentials() {
+ local minio_root_user=${MINIO_ROOT_USER:-clickhouse}
+ local minio_root_password=${MINIO_ROOT_PASSWORD:-clickhouse}
+ mkdir -p ~/.aws
+ cat <> ~/.aws/credentials
[default]
-aws_access_key_id=${MINIO_ROOT_USER}
-aws_secret_access_key=${MINIO_ROOT_PASSWORD}
+aws_access_key_id=${minio_root_user}
+aws_secret_access_key=${minio_root_password}
EOT
+}
+
+wait_for_it() {
+ local counter=0
+ local max_counter=60
+ local url="http://localhost:11111"
+ local params=(
+ --silent
+ --verbose
+ )
+ while ! curl "${params[@]}" "${url}" 2>&1 | grep AccessDenied
+ do
+ if [[ ${counter} == "${max_counter}" ]]; then
+ echo "failed to setup minio"
+ exit 0
+ fi
+ echo "trying to connect to minio"
+ sleep 1
+ counter=$((counter + 1))
+ done
+}
+
+main() {
+ local query_dir
+ query_dir=$(check_arg "$@")
+ if [ ! -f ./minio ]; then
+ download_minio
+ fi
+ start_minio
+ setup_minio "$1"
+ upload_data "${query_dir}" "${2:-/usr/share/clickhouse-test}"
+ setup_aws_credentials
+}
+
+main "$@"
\ No newline at end of file
diff --git a/docker/test/stress/README.md b/docker/test/stress/README.md
index 96807b9f9a6..c22721fd7da 100644
--- a/docker/test/stress/README.md
+++ b/docker/test/stress/README.md
@@ -1,6 +1,6 @@
-Allow to run simple ClickHouse stress test in Docker from debian packages.
+Allows to run simple ClickHouse stress test in Docker from debian packages.
Actually it runs multiple copies of clickhouse-test (functional tests).
-This allows to find problems like segmentation fault which cause shutdown of server.
+This allows to find problems like failed assertions and memory safety issues.
Usage:
```
diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh
index 01e0f5b4897..dc2b184dc5a 100644
--- a/docker/test/stress/run.sh
+++ b/docker/test/stress/run.sh
@@ -11,31 +11,6 @@ set -x
# core.COMM.PID-TID
sysctl kernel.core_pattern='core.%e.%p-%P'
-# Thread Fuzzer allows to check more permutations of possible thread scheduling
-# and find more potential issues.
-# Temporarily disable ThreadFuzzer with tsan because of https://github.com/google/sanitizers/issues/1540
-is_tsan_build=$(clickhouse local -q "select value like '% -fsanitize=thread %' from system.build_options where name='CXX_FLAGS'")
-if [ "$is_tsan_build" -eq "0" ]; then
- export THREAD_FUZZER_CPU_TIME_PERIOD_US=1000
- export THREAD_FUZZER_SLEEP_PROBABILITY=0.1
- export THREAD_FUZZER_SLEEP_TIME_US=100000
-
- export THREAD_FUZZER_pthread_mutex_lock_BEFORE_MIGRATE_PROBABILITY=1
- export THREAD_FUZZER_pthread_mutex_lock_AFTER_MIGRATE_PROBABILITY=1
- export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_MIGRATE_PROBABILITY=1
- export THREAD_FUZZER_pthread_mutex_unlock_AFTER_MIGRATE_PROBABILITY=1
-
- export THREAD_FUZZER_pthread_mutex_lock_BEFORE_SLEEP_PROBABILITY=0.001
- export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_PROBABILITY=0.001
- export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_PROBABILITY=0.001
- export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_PROBABILITY=0.001
- export THREAD_FUZZER_pthread_mutex_lock_BEFORE_SLEEP_TIME_US=10000
-
- export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_TIME_US=10000
- export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_TIME_US=10000
- export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_TIME_US=10000
-fi
-
function install_packages()
{
@@ -54,7 +29,7 @@ function configure()
# we mount tests folder from repo to /usr/share
ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
- ln -s /usr/share/clickhouse-test/ci/download_release_packets.py /usr/bin/download_release_packets
+ ln -s /usr/share/clickhouse-test/ci/download_release_packages.py /usr/bin/download_release_packages
ln -s /usr/share/clickhouse-test/ci/get_previous_release_tag.py /usr/bin/get_previous_release_tag
# avoid too slow startup
@@ -78,6 +53,7 @@ function configure()
local total_mem
total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB
total_mem=$(( total_mem*1024 )) # bytes
+
# Set maximum memory usage as half of total memory (less chance of OOM).
#
# But not via max_server_memory_usage but via max_memory_usage_for_user,
@@ -90,16 +66,17 @@ function configure()
# max_server_memory_usage will be hard limit, and queries that should be
# executed regardless memory limits will use max_memory_usage_for_user=0,
# instead of relying on max_untracked_memory
- local max_server_mem
- max_server_mem=$((total_mem*75/100)) # 75%
- echo "Setting max_server_memory_usage=$max_server_mem"
+
+ max_server_memory_usage_to_ram_ratio=0.5
+ echo "Setting max_server_memory_usage_to_ram_ratio to ${max_server_memory_usage_to_ram_ratio}"
cat > /etc/clickhouse-server/config.d/max_server_memory_usage.xml <
- ${max_server_mem}
+ ${max_server_memory_usage_to_ram_ratio}
EOL
+
local max_users_mem
- max_users_mem=$((total_mem*50/100)) # 50%
+ max_users_mem=$((total_mem*30/100)) # 30%
echo "Setting max_memory_usage_for_user=$max_users_mem"
cat > /etc/clickhouse-server/users.d/max_memory_usage_for_user.xml <
@@ -123,6 +100,29 @@ EOL
$PWD
EOL
+
+ # Let OOM killer terminate other processes before clickhouse-server:
+ cat > /etc/clickhouse-server/config.d/oom_score.xml <
+ -1000
+
+EOL
+
+ # Analyzer is not yet ready for testing
+ cat > /etc/clickhouse-server/users.d/no_analyzer.xml <
+
+
+
+
+
+
+
+
+
+
+EOL
+
}
function stop()
@@ -210,6 +210,31 @@ quit
install_packages package_folder
+# Thread Fuzzer allows to check more permutations of possible thread scheduling
+# and find more potential issues.
+# Temporarily disable ThreadFuzzer with tsan because of https://github.com/google/sanitizers/issues/1540
+is_tsan_build=$(clickhouse local -q "select value like '% -fsanitize=thread %' from system.build_options where name='CXX_FLAGS'")
+if [ "$is_tsan_build" -eq "0" ]; then
+ export THREAD_FUZZER_CPU_TIME_PERIOD_US=1000
+ export THREAD_FUZZER_SLEEP_PROBABILITY=0.1
+ export THREAD_FUZZER_SLEEP_TIME_US=100000
+
+ export THREAD_FUZZER_pthread_mutex_lock_BEFORE_MIGRATE_PROBABILITY=1
+ export THREAD_FUZZER_pthread_mutex_lock_AFTER_MIGRATE_PROBABILITY=1
+ export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_MIGRATE_PROBABILITY=1
+ export THREAD_FUZZER_pthread_mutex_unlock_AFTER_MIGRATE_PROBABILITY=1
+
+ export THREAD_FUZZER_pthread_mutex_lock_BEFORE_SLEEP_PROBABILITY=0.001
+ export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_PROBABILITY=0.001
+ export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_PROBABILITY=0.001
+ export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_PROBABILITY=0.001
+ export THREAD_FUZZER_pthread_mutex_lock_BEFORE_SLEEP_TIME_US=10000
+
+ export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_TIME_US=10000
+ export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_TIME_US=10000
+ export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_TIME_US=10000
+fi
+
export ZOOKEEPER_FAULT_INJECTION=1
configure
@@ -334,219 +359,228 @@ zgrep -Fa "########################################" /test_output/* > /dev/null
zgrep -Fa " received signal " /test_output/gdb.log > /dev/null \
&& echo -e 'Found signal in gdb.log\tFAIL' >> /test_output/test_results.tsv
-echo -e "Backward compatibility check\n"
+if [ "$DISABLE_BC_CHECK" -ne "1" ]; then
+ echo -e "Backward compatibility check\n"
-echo "Get previous release tag"
-previous_release_tag=$(clickhouse-client --version | grep -o "[0-9]*\.[0-9]*\.[0-9]*\.[0-9]*" | get_previous_release_tag)
-echo $previous_release_tag
+ echo "Get previous release tag"
+ previous_release_tag=$(clickhouse-client --version | grep -o "[0-9]*\.[0-9]*\.[0-9]*\.[0-9]*" | get_previous_release_tag)
+ echo $previous_release_tag
-echo "Clone previous release repository"
-git clone https://github.com/ClickHouse/ClickHouse.git --no-tags --progress --branch=$previous_release_tag --no-recurse-submodules --depth=1 previous_release_repository
+ echo "Clone previous release repository"
+ git clone https://github.com/ClickHouse/ClickHouse.git --no-tags --progress --branch=$previous_release_tag --no-recurse-submodules --depth=1 previous_release_repository
-echo "Download previous release server"
-mkdir previous_release_package_folder
+ echo "Download clickhouse-server from the previous release"
+ mkdir previous_release_package_folder
-echo $previous_release_tag | download_release_packets && echo -e 'Download script exit code\tOK' >> /test_output/test_results.tsv \
- || echo -e 'Download script failed\tFAIL' >> /test_output/test_results.tsv
+ echo $previous_release_tag | download_release_packages && echo -e 'Download script exit code\tOK' >> /test_output/test_results.tsv \
+ || echo -e 'Download script failed\tFAIL' >> /test_output/test_results.tsv
-mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.clean.log
-for table in query_log trace_log
-do
- clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.tsv.gz ||:
-done
-
-tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:
-
-# Check if we cloned previous release repository successfully
-if ! [ "$(ls -A previous_release_repository/tests/queries)" ]
-then
- echo -e "Backward compatibility check: Failed to clone previous release tests\tFAIL" >> /test_output/test_results.tsv
-elif ! [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ]
-then
- echo -e "Backward compatibility check: Failed to download previous release packets\tFAIL" >> /test_output/test_results.tsv
-else
- echo -e "Successfully cloned previous release tests\tOK" >> /test_output/test_results.tsv
- echo -e "Successfully downloaded previous release packets\tOK" >> /test_output/test_results.tsv
-
- # Uninstall current packages
- dpkg --remove clickhouse-client
- dpkg --remove clickhouse-server
- dpkg --remove clickhouse-common-static-dbg
- dpkg --remove clickhouse-common-static
-
- rm -rf /var/lib/clickhouse/*
-
- # Make BC check more funny by forcing Ordinary engine for system database
- mkdir /var/lib/clickhouse/metadata
- echo "ATTACH DATABASE system ENGINE=Ordinary" > /var/lib/clickhouse/metadata/system.sql
-
- # Install previous release packages
- install_packages previous_release_package_folder
-
- # Start server from previous release
- # Previous version may not be ready for fault injections
- export ZOOKEEPER_FAULT_INJECTION=0
- configure
-
- # Avoid "Setting s3_check_objects_after_upload is neither a builtin setting..."
- rm -f /etc/clickhouse-server/users.d/enable_blobs_check.xml ||:
- rm -f /etc/clickhouse-server/users.d/marks.xml ||:
-
- # Remove s3 related configs to avoid "there is no disk type `cache`"
- rm -f /etc/clickhouse-server/config.d/storage_conf.xml ||:
- rm -f /etc/clickhouse-server/config.d/azure_storage_conf.xml ||:
-
- # Turn on after 22.12
- rm -f /etc/clickhouse-server/config.d/compressed_marks_and_index.xml ||:
- # it uses recently introduced settings which previous versions may not have
- rm -f /etc/clickhouse-server/users.d/insert_keeper_retries.xml ||:
-
- start
-
- clickhouse-client --query="SELECT 'Server version: ', version()"
-
- # Install new package before running stress test because we should use new
- # clickhouse-client and new clickhouse-test.
- #
- # But we should leave old binary in /usr/bin/ and debug symbols in
- # /usr/lib/debug/usr/bin (if any) for gdb and internal DWARF parser, so it
- # will print sane stacktraces and also to avoid possible crashes.
- #
- # FIXME: those files can be extracted directly from debian package, but
- # actually better solution will be to use different PATH instead of playing
- # games with files from packages.
- mv /usr/bin/clickhouse previous_release_package_folder/
- mv /usr/lib/debug/usr/bin/clickhouse.debug previous_release_package_folder/
- install_packages package_folder
- mv /usr/bin/clickhouse package_folder/
- mv /usr/lib/debug/usr/bin/clickhouse.debug package_folder/
- mv previous_release_package_folder/clickhouse /usr/bin/
- mv previous_release_package_folder/clickhouse.debug /usr/lib/debug/usr/bin/clickhouse.debug
-
- mkdir tmp_stress_output
-
- ./stress --test-cmd="/usr/bin/clickhouse-test --queries=\"previous_release_repository/tests/queries\"" --backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1200 \
- && echo -e 'Backward compatibility check: Test script exit code\tOK' >> /test_output/test_results.tsv \
- || echo -e 'Backward compatibility check: Test script failed\tFAIL' >> /test_output/test_results.tsv
- rm -rf tmp_stress_output
-
- clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables"
-
- stop 1
- mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.stress.log
-
- # Start new server
- mv package_folder/clickhouse /usr/bin/
- mv package_folder/clickhouse.debug /usr/lib/debug/usr/bin/clickhouse.debug
- export ZOOKEEPER_FAULT_INJECTION=1
- configure
- start 500
- clickhouse-client --query "SELECT 'Backward compatibility check: Server successfully started', 'OK'" >> /test_output/test_results.tsv \
- || (echo -e 'Backward compatibility check: Server failed to start\tFAIL' >> /test_output/test_results.tsv \
- && grep -a ".*Application" /var/log/clickhouse-server/clickhouse-server.log >> /test_output/bc_check_application_errors.txt)
-
- clickhouse-client --query="SELECT 'Server version: ', version()"
-
- # Let the server run for a while before checking log.
- sleep 60
-
- stop
- mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.clean.log
-
- # Error messages (we should ignore some errors)
- # FIXME https://github.com/ClickHouse/ClickHouse/issues/38643 ("Unknown index: idx.")
- # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 ("Cannot parse string 'Hello' as UInt64")
- # FIXME Not sure if it's expected, but some tests from BC check may not be finished yet when we restarting server.
- # Let's just ignore all errors from queries ("} TCPHandler: Code:", "} executeQuery: Code:")
- # FIXME https://github.com/ClickHouse/ClickHouse/issues/39197 ("Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'")
- # NOTE Incompatibility was introduced in https://github.com/ClickHouse/ClickHouse/pull/39263, it's expected
- # ("This engine is deprecated and is not supported in transactions", "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part")
- # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 - bad mutation does not indicate backward incompatibility
- echo "Check for Error messages in server log:"
- zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \
- -e "Code: 236. DB::Exception: Cancelled mutating parts" \
- -e "REPLICA_IS_ALREADY_ACTIVE" \
- -e "REPLICA_ALREADY_EXISTS" \
- -e "ALL_REPLICAS_LOST" \
- -e "DDLWorker: Cannot parse DDL task query" \
- -e "RaftInstance: failed to accept a rpc connection due to error 125" \
- -e "UNKNOWN_DATABASE" \
- -e "NETWORK_ERROR" \
- -e "UNKNOWN_TABLE" \
- -e "ZooKeeperClient" \
- -e "KEEPER_EXCEPTION" \
- -e "DirectoryMonitor" \
- -e "TABLE_IS_READ_ONLY" \
- -e "Code: 1000, e.code() = 111, Connection refused" \
- -e "UNFINISHED" \
- -e "NETLINK_ERROR" \
- -e "Renaming unexpected part" \
- -e "PART_IS_TEMPORARILY_LOCKED" \
- -e "and a merge is impossible: we didn't find" \
- -e "found in queue and some source parts for it was lost" \
- -e "is lost forever." \
- -e "Unknown index: idx." \
- -e "Cannot parse string 'Hello' as UInt64" \
- -e "} TCPHandler: Code:" \
- -e "} executeQuery: Code:" \
- -e "Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'" \
- -e "This engine is deprecated and is not supported in transactions" \
- -e "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part" \
- -e "The set of parts restored in place of" \
- -e "(ReplicatedMergeTreeAttachThread): Initialization failed. Error" \
- -e "Code: 269. DB::Exception: Destination table is myself" \
- -e "Coordination::Exception: Connection loss" \
- -e "MutateFromLogEntryTask" \
- -e "No connection to ZooKeeper, cannot get shared table ID" \
- -e "Session expired" \
- /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \
- && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \
- || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
-
- # Remove file bc_check_error_messages.txt if it's empty
- [ -s /test_output/bc_check_error_messages.txt ] || rm /test_output/bc_check_error_messages.txt
-
- # Sanitizer asserts
- zgrep -Fa "==================" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
- zgrep -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
- zgrep -Fav -e "ASan doesn't fully support makecontext/swapcontext functions" -e "DB::Exception" /test_output/tmp > /dev/null \
- && echo -e 'Backward compatibility check: Sanitizer assert (in stderr.log)\tFAIL' >> /test_output/test_results.tsv \
- || echo -e 'Backward compatibility check: No sanitizer asserts\tOK' >> /test_output/test_results.tsv
- rm -f /test_output/tmp
-
- # OOM
- zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \
- && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \
- || echo -e 'Backward compatibility check: No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
-
- # Logical errors
- echo "Check for Logical errors in server log:"
- zgrep -Fa -A20 "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_logical_errors.txt \
- && echo -e 'Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)\tFAIL' >> /test_output/test_results.tsv \
- || echo -e 'Backward compatibility check: No logical errors\tOK' >> /test_output/test_results.tsv
-
- # Remove file bc_check_logical_errors.txt if it's empty
- [ -s /test_output/bc_check_logical_errors.txt ] || rm /test_output/bc_check_logical_errors.txt
-
- # Crash
- zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \
- && echo -e 'Backward compatibility check: Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \
- || echo -e 'Backward compatibility check: Not crashed\tOK' >> /test_output/test_results.tsv
-
- # It also checks for crash without stacktrace (printed by watchdog)
- echo "Check for Fatal message in server log:"
- zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_fatal_messages.txt \
- && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \
- || echo -e 'Backward compatibility check: No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
-
- # Remove file bc_check_fatal_messages.txt if it's empty
- [ -s /test_output/bc_check_fatal_messages.txt ] || rm /test_output/bc_check_fatal_messages.txt
-
- tar -chf /test_output/coordination.backward.tar /var/lib/clickhouse/coordination ||:
+ mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.clean.log
for table in query_log trace_log
do
- clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.backward.tsv.gz ||:
+ clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.tsv.gz ||:
done
+
+ tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:
+
+ # Check if we cloned previous release repository successfully
+ if ! [ "$(ls -A previous_release_repository/tests/queries)" ]
+ then
+ echo -e "Backward compatibility check: Failed to clone previous release tests\tFAIL" >> /test_output/test_results.tsv
+ elif ! [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ]
+ then
+ echo -e "Backward compatibility check: Failed to download previous release packages\tFAIL" >> /test_output/test_results.tsv
+ else
+ echo -e "Successfully cloned previous release tests\tOK" >> /test_output/test_results.tsv
+ echo -e "Successfully downloaded previous release packages\tOK" >> /test_output/test_results.tsv
+
+ # Uninstall current packages
+ dpkg --remove clickhouse-client
+ dpkg --remove clickhouse-server
+ dpkg --remove clickhouse-common-static-dbg
+ dpkg --remove clickhouse-common-static
+
+ rm -rf /var/lib/clickhouse/*
+
+ # Make BC check more funny by forcing Ordinary engine for system database
+ mkdir /var/lib/clickhouse/metadata
+ echo "ATTACH DATABASE system ENGINE=Ordinary" > /var/lib/clickhouse/metadata/system.sql
+
+ # Install previous release packages
+ install_packages previous_release_package_folder
+
+ # Start server from previous release
+ # Previous version may not be ready for fault injections
+ export ZOOKEEPER_FAULT_INJECTION=0
+ configure
+
+ # Avoid "Setting s3_check_objects_after_upload is neither a builtin setting..."
+ rm -f /etc/clickhouse-server/users.d/enable_blobs_check.xml ||:
+ rm -f /etc/clickhouse-server/users.d/marks.xml ||:
+
+ # Remove s3 related configs to avoid "there is no disk type `cache`"
+ rm -f /etc/clickhouse-server/config.d/storage_conf.xml ||:
+ rm -f /etc/clickhouse-server/config.d/azure_storage_conf.xml ||:
+
+ # Turn on after 22.12
+ rm -f /etc/clickhouse-server/config.d/compressed_marks_and_index.xml ||:
+ # it uses recently introduced settings which previous versions may not have
+ rm -f /etc/clickhouse-server/users.d/insert_keeper_retries.xml ||:
+
+ start
+
+ clickhouse-client --query="SELECT 'Server version: ', version()"
+
+ # Install new package before running stress test because we should use new
+ # clickhouse-client and new clickhouse-test.
+ #
+ # But we should leave old binary in /usr/bin/ and debug symbols in
+ # /usr/lib/debug/usr/bin (if any) for gdb and internal DWARF parser, so it
+ # will print sane stacktraces and also to avoid possible crashes.
+ #
+ # FIXME: those files can be extracted directly from debian package, but
+ # actually better solution will be to use different PATH instead of playing
+ # games with files from packages.
+ mv /usr/bin/clickhouse previous_release_package_folder/
+ mv /usr/lib/debug/usr/bin/clickhouse.debug previous_release_package_folder/
+ install_packages package_folder
+ mv /usr/bin/clickhouse package_folder/
+ mv /usr/lib/debug/usr/bin/clickhouse.debug package_folder/
+ mv previous_release_package_folder/clickhouse /usr/bin/
+ mv previous_release_package_folder/clickhouse.debug /usr/lib/debug/usr/bin/clickhouse.debug
+
+ mkdir tmp_stress_output
+
+ ./stress --test-cmd="/usr/bin/clickhouse-test --queries=\"previous_release_repository/tests/queries\"" --backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1200 \
+ && echo -e 'Backward compatibility check: Test script exit code\tOK' >> /test_output/test_results.tsv \
+ || echo -e 'Backward compatibility check: Test script failed\tFAIL' >> /test_output/test_results.tsv
+ rm -rf tmp_stress_output
+
+ # We experienced deadlocks in this command in very rare cases. Let's debug it:
+ timeout 10m clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables" ||
+ (
+ echo "thread apply all backtrace (on select tables count)" >> /test_output/gdb.log
+ timeout 30m gdb -batch -ex 'thread apply all backtrace' -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log
+ clickhouse stop --force
+ )
+
+ stop 1
+ mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.stress.log
+
+ # Start new server
+ mv package_folder/clickhouse /usr/bin/
+ mv package_folder/clickhouse.debug /usr/lib/debug/usr/bin/clickhouse.debug
+ # Disable fault injections on start (we don't test them here, and it can lead to tons of requests in case of huge number of tables).
+ export ZOOKEEPER_FAULT_INJECTION=0
+ configure
+ start 500
+ clickhouse-client --query "SELECT 'Backward compatibility check: Server successfully started', 'OK'" >> /test_output/test_results.tsv \
+ || (echo -e 'Backward compatibility check: Server failed to start\tFAIL' >> /test_output/test_results.tsv \
+ && grep -a ".*Application" /var/log/clickhouse-server/clickhouse-server.log >> /test_output/bc_check_application_errors.txt)
+
+ clickhouse-client --query="SELECT 'Server version: ', version()"
+
+ # Let the server run for a while before checking log.
+ sleep 60
+
+ stop
+ mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.dirty.log
+
+ # Error messages (we should ignore some errors)
+ # FIXME https://github.com/ClickHouse/ClickHouse/issues/38643 ("Unknown index: idx.")
+ # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 ("Cannot parse string 'Hello' as UInt64")
+ # FIXME Not sure if it's expected, but some tests from BC check may not be finished yet when we restarting server.
+ # Let's just ignore all errors from queries ("} TCPHandler: Code:", "} executeQuery: Code:")
+ # FIXME https://github.com/ClickHouse/ClickHouse/issues/39197 ("Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'")
+ # NOTE Incompatibility was introduced in https://github.com/ClickHouse/ClickHouse/pull/39263, it's expected
+ # ("This engine is deprecated and is not supported in transactions", "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part")
+ # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 - bad mutation does not indicate backward incompatibility
+ echo "Check for Error messages in server log:"
+ zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \
+ -e "Code: 236. DB::Exception: Cancelled mutating parts" \
+ -e "REPLICA_IS_ALREADY_ACTIVE" \
+ -e "REPLICA_ALREADY_EXISTS" \
+ -e "ALL_REPLICAS_LOST" \
+ -e "DDLWorker: Cannot parse DDL task query" \
+ -e "RaftInstance: failed to accept a rpc connection due to error 125" \
+ -e "UNKNOWN_DATABASE" \
+ -e "NETWORK_ERROR" \
+ -e "UNKNOWN_TABLE" \
+ -e "ZooKeeperClient" \
+ -e "KEEPER_EXCEPTION" \
+ -e "DirectoryMonitor" \
+ -e "TABLE_IS_READ_ONLY" \
+ -e "Code: 1000, e.code() = 111, Connection refused" \
+ -e "UNFINISHED" \
+ -e "NETLINK_ERROR" \
+ -e "Renaming unexpected part" \
+ -e "PART_IS_TEMPORARILY_LOCKED" \
+ -e "and a merge is impossible: we didn't find" \
+ -e "found in queue and some source parts for it was lost" \
+ -e "is lost forever." \
+ -e "Unknown index: idx." \
+ -e "Cannot parse string 'Hello' as UInt64" \
+ -e "} TCPHandler: Code:" \
+ -e "} executeQuery: Code:" \
+ -e "Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'" \
+ -e "This engine is deprecated and is not supported in transactions" \
+ -e "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part" \
+ -e "The set of parts restored in place of" \
+ -e "(ReplicatedMergeTreeAttachThread): Initialization failed. Error" \
+ -e "Code: 269. DB::Exception: Destination table is myself" \
+ -e "Coordination::Exception: Connection loss" \
+ -e "MutateFromLogEntryTask" \
+ -e "No connection to ZooKeeper, cannot get shared table ID" \
+ -e "Session expired" \
+ /var/log/clickhouse-server/clickhouse-server.backward.dirty.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \
+ && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \
+ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
+
+ # Remove file bc_check_error_messages.txt if it's empty
+ [ -s /test_output/bc_check_error_messages.txt ] || rm /test_output/bc_check_error_messages.txt
+
+ # Sanitizer asserts
+ zgrep -Fa "==================" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
+ zgrep -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
+ zgrep -Fav -e "ASan doesn't fully support makecontext/swapcontext functions" -e "DB::Exception" /test_output/tmp > /dev/null \
+ && echo -e 'Backward compatibility check: Sanitizer assert (in stderr.log)\tFAIL' >> /test_output/test_results.tsv \
+ || echo -e 'Backward compatibility check: No sanitizer asserts\tOK' >> /test_output/test_results.tsv
+ rm -f /test_output/tmp
+
+ # OOM
+ zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \
+ && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \
+ || echo -e 'Backward compatibility check: No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
+
+ # Logical errors
+ echo "Check for Logical errors in server log:"
+ zgrep -Fa -A20 "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_logical_errors.txt \
+ && echo -e 'Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)\tFAIL' >> /test_output/test_results.tsv \
+ || echo -e 'Backward compatibility check: No logical errors\tOK' >> /test_output/test_results.tsv
+
+ # Remove file bc_check_logical_errors.txt if it's empty
+ [ -s /test_output/bc_check_logical_errors.txt ] || rm /test_output/bc_check_logical_errors.txt
+
+ # Crash
+ zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \
+ && echo -e 'Backward compatibility check: Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \
+ || echo -e 'Backward compatibility check: Not crashed\tOK' >> /test_output/test_results.tsv
+
+ # It also checks for crash without stacktrace (printed by watchdog)
+ echo "Check for Fatal message in server log:"
+ zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_fatal_messages.txt \
+ && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \
+ || echo -e 'Backward compatibility check: No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
+
+ # Remove file bc_check_fatal_messages.txt if it's empty
+ [ -s /test_output/bc_check_fatal_messages.txt ] || rm /test_output/bc_check_fatal_messages.txt
+
+ tar -chf /test_output/coordination.backward.tar /var/lib/clickhouse/coordination ||:
+ for table in query_log trace_log
+ do
+ clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.backward.tsv.gz ||:
+ done
+ fi
fi
dmesg -T > /test_output/dmesg.log
diff --git a/docker/test/stress/stress b/docker/test/stress/stress
index a0ec86f7fbe..d1860e9e14b 100755
--- a/docker/test/stress/stress
+++ b/docker/test/stress/stress
@@ -14,9 +14,6 @@ def get_options(i, backward_compatibility_check):
if 0 < i:
options.append("--order=random")
- if i % 3 == 1:
- options.append("--db-engine=Ordinary")
-
if i % 3 == 2 and not backward_compatibility_check:
options.append(
'''--db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i)
diff --git a/docker/test/testflows/runner/Dockerfile b/docker/test/testflows/runner/Dockerfile
deleted file mode 100644
index bfc3ed5e39f..00000000000
--- a/docker/test/testflows/runner/Dockerfile
+++ /dev/null
@@ -1,82 +0,0 @@
-# docker build -t clickhouse/testflows-runner .
-FROM ubuntu:20.04
-
-# ARG for quick switch to a given ubuntu mirror
-ARG apt_archive="http://archive.ubuntu.com"
-RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list
-
-RUN apt-get update \
- && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
- ca-certificates \
- bash \
- btrfs-progs \
- e2fsprogs \
- iptables \
- xfsprogs \
- tar \
- pigz \
- wget \
- git \
- iproute2 \
- cgroupfs-mount \
- python3-pip \
- tzdata \
- libicu-dev \
- bsdutils \
- curl \
- liblua5.1-dev \
- luajit \
- libssl-dev \
- libcurl4-openssl-dev \
- gdb \
- && rm -rf \
- /var/lib/apt/lists/* \
- /var/cache/debconf \
- /tmp/* \
- && apt-get clean
-
-ENV TZ=Europe/Moscow
-RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
-
-RUN pip3 install urllib3 testflows==1.7.20 docker-compose==1.29.2 docker==5.0.0 dicttoxml kazoo tzlocal==2.1 pytz python-dateutil numpy
-
-ENV DOCKER_CHANNEL stable
-ENV DOCKER_VERSION 20.10.6
-
-# Architecture of the image when BuildKit/buildx is used
-ARG TARGETARCH
-
-# Install docker
-RUN arch=${TARGETARCH:-amd64} \
- && case $arch in \
- amd64) rarch=x86_64 ;; \
- arm64) rarch=aarch64 ;; \
- esac \
- && set -eux \
- && if ! wget -nv -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/${rarch}/docker-${DOCKER_VERSION}.tgz"; then \
- echo >&2 "error: failed to download 'docker-${DOCKER_VERSION}' from '${DOCKER_CHANNEL}' for '${rarch}'" \
- && exit 1; \
- fi \
- && tar --extract \
- --file docker.tgz \
- --strip-components 1 \
- --directory /usr/local/bin/ \
- && rm docker.tgz \
- && dockerd --version \
- && docker --version
-
-COPY modprobe.sh /usr/local/bin/modprobe
-COPY dockerd-entrypoint.sh /usr/local/bin/
-COPY process_testflows_result.py /usr/local/bin/
-
-RUN set -x \
- && addgroup --system dockremap \
- && adduser --system dockremap \
- && adduser dockremap dockremap \
- && echo 'dockremap:165536:65536' >> /etc/subuid \
- && echo 'dockremap:165536:65536' >> /etc/subgid
-
-VOLUME /var/lib/docker
-EXPOSE 2375
-ENTRYPOINT ["dockerd-entrypoint.sh"]
-CMD ["sh", "-c", "python3 regression.py --no-color -o new-fails --local --clickhouse-binary-path ${CLICKHOUSE_TESTS_SERVER_BIN_PATH} --log test.log ${TESTFLOWS_OPTS}; cat test.log | tfs report results --format json > results.json; /usr/local/bin/process_testflows_result.py || echo -e 'failure\tCannot parse results' > check_status.tsv; find * -type f | grep _instances | grep clickhouse-server | xargs -n1 tar -rvf clickhouse_logs.tar; gzip -9 clickhouse_logs.tar"]
diff --git a/docker/test/testflows/runner/dockerd-entrypoint.sh b/docker/test/testflows/runner/dockerd-entrypoint.sh
deleted file mode 100755
index d310ee583bf..00000000000
--- a/docker/test/testflows/runner/dockerd-entrypoint.sh
+++ /dev/null
@@ -1,39 +0,0 @@
-#!/bin/bash
-set -e
-
-echo "Configure to use Yandex dockerhub-proxy"
-mkdir -p /etc/docker/
-cat > /etc/docker/daemon.json << EOF
-{
- "insecure-registries" : ["dockerhub-proxy.dockerhub-proxy-zone:5000"],
- "registry-mirrors" : ["http://dockerhub-proxy.dockerhub-proxy-zone:5000"]
-}
-EOF
-
-# In case of test hung it is convenient to use pytest --pdb to debug it,
-# and on hung you can simply press Ctrl-C and it will spawn a python pdb,
-# but on SIGINT dockerd will exit, so ignore it to preserve the daemon.
-trap '' INT
-dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 &>/var/log/somefile &
-
-set +e
-reties=0
-while true; do
- docker info &>/dev/null && break
- reties=$((reties+1))
- if [[ $reties -ge 100 ]]; then # 10 sec max
- echo "Can't start docker daemon, timeout exceeded." >&2
- exit 1;
- fi
- sleep 0.1
-done
-set -e
-
-echo "Start tests"
-export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse
-export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse
-export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config
-export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge
-
-cd /ClickHouse/tests/testflows
-exec "$@"
diff --git a/docker/test/testflows/runner/modprobe.sh b/docker/test/testflows/runner/modprobe.sh
deleted file mode 100755
index cb6a527736b..00000000000
--- a/docker/test/testflows/runner/modprobe.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/bin/sh
-set -eu
-
-# "modprobe" without modprobe
-# https://twitter.com/lucabruno/status/902934379835662336
-
-# this isn't 100% fool-proof, but it'll have a much higher success rate than simply using the "real" modprobe
-
-# Docker often uses "modprobe -va foo bar baz"
-# so we ignore modules that start with "-"
-for module; do
- if [ "${module#-}" = "$module" ]; then
- ip link show "$module" || true
- lsmod | grep "$module" || true
- fi
-done
-
-# remove /usr/local/... from PATH so we can exec the real modprobe as a last resort
-export PATH='/usr/sbin:/usr/bin:/sbin:/bin'
-exec modprobe "$@"
diff --git a/docker/test/testflows/runner/process_testflows_result.py b/docker/test/testflows/runner/process_testflows_result.py
deleted file mode 100755
index 8bfc4ac0b0f..00000000000
--- a/docker/test/testflows/runner/process_testflows_result.py
+++ /dev/null
@@ -1,71 +0,0 @@
-#!/usr/bin/env python3
-
-import os
-import logging
-import argparse
-import csv
-import json
-
-
-def process_result(result_folder):
- json_path = os.path.join(result_folder, "results.json")
- if not os.path.exists(json_path):
- return "success", "No testflows in branch", None, []
-
- test_binary_log = os.path.join(result_folder, "test.log")
- with open(json_path) as source:
- results = json.loads(source.read())
-
- total_tests = 0
- total_ok = 0
- total_fail = 0
- total_other = 0
- test_results = []
- for test in results["tests"]:
- test_name = test["test"]["test_name"]
- test_result = test["result"]["result_type"].upper()
- test_time = str(test["result"]["message_rtime"])
- total_tests += 1
- if test_result == "OK":
- total_ok += 1
- elif test_result == "FAIL" or test_result == "ERROR":
- total_fail += 1
- else:
- total_other += 1
-
- test_results.append((test_name, test_result, test_time))
- if total_fail != 0:
- status = "failure"
- else:
- status = "success"
-
- description = "failed: {}, passed: {}, other: {}".format(
- total_fail, total_ok, total_other
- )
- return status, description, test_results, [json_path, test_binary_log]
-
-
-def write_results(results_file, status_file, results, status):
- with open(results_file, "w") as f:
- out = csv.writer(f, delimiter="\t")
- out.writerows(results)
- with open(status_file, "w") as f:
- out = csv.writer(f, delimiter="\t")
- out.writerow(status)
-
-
-if __name__ == "__main__":
- logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s")
- parser = argparse.ArgumentParser(
- description="ClickHouse script for parsing results of Testflows tests"
- )
- parser.add_argument("--in-results-dir", default="./")
- parser.add_argument("--out-results-file", default="./test_results.tsv")
- parser.add_argument("--out-status-file", default="./check_status.tsv")
- args = parser.parse_args()
-
- state, description, test_results, logs = process_result(args.in_results_dir)
- logging.info("Result parsed")
- status = (state, description)
- write_results(args.out_results_file, args.out_status_file, test_results, status)
- logging.info("Result written")
diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md
index 677fb81efdd..ef3efa75d66 100644
--- a/docs/en/development/continuous-integration.md
+++ b/docs/en/development/continuous-integration.md
@@ -118,7 +118,6 @@ Builds ClickHouse in various configurations for use in further steps. You have t
- **Compiler**: `gcc-9` or `clang-10` (or `clang-10-xx` for other architectures e.g. `clang-10-freebsd`).
- **Build type**: `Debug` or `RelWithDebInfo` (cmake).
- **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan).
-- **Split** `splitted` is a [split build](../development/build.md#split-build)
- **Status**: `success` or `fail`
- **Build log**: link to the building and files copying log, useful when build failed.
- **Build time**.
@@ -130,7 +129,6 @@ Builds ClickHouse in various configurations for use in further steps. You have t
- `clickhouse`: Main built binary.
- `clickhouse-odbc-bridge`
- `unit_tests_dbms`: GoogleTest binary with ClickHouse unit tests.
- - `shared_build.tgz`: build with shared libraries.
- `performance.tgz`: Special package for performance tests.
@@ -169,16 +167,6 @@ concurrency-related errors. If it fails:
of error.
-## Split Build Smoke Test
-
-Checks that the server build in [split build](../development/developer-instruction.md#split-build)
-configuration can start and run simple queries. If it fails:
-
- * Fix other test errors first;
- * Build the server in [split build](../development/developer-instruction.md#split-build) configuration
- locally and check whether it can start and run `select 1`.
-
-
## Compatibility Check
Checks that `clickhouse` binary runs on distributions with old libc versions. If it fails, ask a maintainer for help.
diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md
index 88a0d08ebbd..7de856716fb 100644
--- a/docs/en/engines/table-engines/integrations/kafka.md
+++ b/docs/en/engines/table-engines/integrations/kafka.md
@@ -34,7 +34,14 @@ SETTINGS
[kafka_max_block_size = 0,]
[kafka_skip_broken_messages = N,]
[kafka_commit_every_batch = 0,]
- [kafka_thread_per_consumer = 0]
+ [kafka_client_id = '',]
+ [kafka_poll_timeout_ms = 0,]
+ [kafka_poll_max_batch_size = 0,]
+ [kafka_flush_interval_ms = 0,]
+ [kafka_thread_per_consumer = 0,]
+ [kafka_handle_error_mode = 'default',]
+ [kafka_commit_on_select = false,]
+ [kafka_max_rows_per_message = 1];
```
Required parameters:
@@ -46,13 +53,20 @@ Required parameters:
Optional parameters:
-- `kafka_row_delimiter` — Delimiter character, which ends the message.
+- `kafka_row_delimiter` — Delimiter character, which ends the message. **This setting is deprecated and is no longer used, not left for compatibility reasons.**
- `kafka_schema` — Parameter that must be used if the format requires a schema definition. For example, [Cap’n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object.
-- `kafka_num_consumers` — The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. The total number of consumers should not exceed the number of partitions in the topic, since only one consumer can be assigned per partition, and must not be greater than the number of physical cores on the server where ClickHouse is deployed.
-- `kafka_max_block_size` — The maximum batch size (in messages) for poll (default: `max_block_size`).
-- `kafka_skip_broken_messages` — Kafka message parser tolerance to schema-incompatible messages per block. Default: `0`. If `kafka_skip_broken_messages = N` then the engine skips *N* Kafka messages that cannot be parsed (a message equals a row of data).
-- `kafka_commit_every_batch` — Commit every consumed and handled batch instead of a single commit after writing a whole block (default: `0`).
-- `kafka_thread_per_consumer` — Provide independent thread for each consumer (default: `0`). When enabled, every consumer flush the data independently, in parallel (otherwise — rows from several consumers squashed to form one block).
+- `kafka_num_consumers` — The number of consumers per table. Specify more consumers if the throughput of one consumer is insufficient. The total number of consumers should not exceed the number of partitions in the topic, since only one consumer can be assigned per partition, and must not be greater than the number of physical cores on the server where ClickHouse is deployed. Default: `1`.
+- `kafka_max_block_size` — The maximum batch size (in messages) for poll. Default: [max_insert_block_size](../../../operations/settings/settings.md#setting-max_insert_block_size).
+- `kafka_skip_broken_messages` — Kafka message parser tolerance to schema-incompatible messages per block. If `kafka_skip_broken_messages = N` then the engine skips *N* Kafka messages that cannot be parsed (a message equals a row of data). Default: `0`.
+- `kafka_commit_every_batch` — Commit every consumed and handled batch instead of a single commit after writing a whole block. Default: `0`.
+- `kafka_client_id` — Client identifier. Empty by default.
+- `kafka_poll_timeout_ms` — Timeout for single poll from Kafka. Default: [stream_poll_timeout_ms](../../../operations/settings/settings.md#stream_poll_timeout_ms).
+- `kafka_poll_max_batch_size` — Maximum amount of messages to be polled in a single Kafka poll. Default: [max_block_size](../../../operations/settings/settings.md#setting-max_block_size).
+- `kafka_flush_interval_ms` — Timeout for flushing data from Kafka. Default: [stream_flush_interval_ms](../../../operations/settings/settings.md#stream-flush-interval-ms).
+- `kafka_thread_per_consumer` — Provide independent thread for each consumer. When enabled, every consumer flush the data independently, in parallel (otherwise — rows from several consumers squashed to form one block). Default: `0`.
+- `kafka_handle_error_mode` — How to handle errors for Kafka engine. Possible values: default, stream.
+- `kafka_commit_on_select` — Commit messages when select query is made. Default: `false`.
+- `kafka_max_rows_per_message` — The maximum number of rows written in one kafka message for row-based formats. Default : `1`.
Examples:
@@ -94,7 +108,7 @@ Do not use this method in new projects. If possible, switch old projects to the
``` sql
Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format
- [, kafka_row_delimiter, kafka_schema, kafka_num_consumers, kafka_skip_broken_messages])
+ [, kafka_row_delimiter, kafka_schema, kafka_num_consumers, kafka_max_block_size, kafka_skip_broken_messages, kafka_commit_every_batch, kafka_client_id, kafka_poll_timeout_ms, kafka_poll_max_batch_size, kafka_flush_interval_ms, kafka_thread_per_consumer, kafka_handle_error_mode, kafka_commit_on_select, kafka_max_rows_per_message]);
```
@@ -193,6 +207,14 @@ Example:
- `_headers.name` — Array of message's headers keys.
- `_headers.value` — Array of message's headers values.
+## Data formats support {#data-formats-support}
+
+Kafka engine supports all [formats](../../../interfaces/formats.md) supported in ClickHouse.
+The number of rows in one Kafka message depends on whether the format is row-based or block-based:
+
+- For row-based formats the number of rows in one Kafka message can be controlled by setting `kafka_max_rows_per_message`.
+- For block-based formats we cannot divide block into smaller parts, but the number of rows in one block can be controlled by general setting [max_block_size](../../../operations/settings/settings.md#setting-max_block_size).
+
**See Also**
- [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns)
diff --git a/docs/en/engines/table-engines/integrations/nats.md b/docs/en/engines/table-engines/integrations/nats.md
index 90b30dc8295..35ad9068be8 100644
--- a/docs/en/engines/table-engines/integrations/nats.md
+++ b/docs/en/engines/table-engines/integrations/nats.md
@@ -37,8 +37,10 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
[nats_max_block_size = N,]
[nats_flush_interval_ms = N,]
[nats_username = 'user',]
- [nats_password = 'password']
- [redis_password = 'clickhouse']
+ [nats_password = 'password',]
+ [nats_token = 'clickhouse',]
+ [nats_startup_connect_tries = '5']
+ [nats_max_rows_per_message = 1]
```
Required parameters:
@@ -49,7 +51,7 @@ Required parameters:
Optional parameters:
-- `nats_row_delimiter` – Delimiter character, which ends the message.
+- `nats_row_delimiter` – Delimiter character, which ends the message. **This setting is deprecated and is no longer used, not left for compatibility reasons.**
- `nats_schema` – Parameter that must be used if the format requires a schema definition. For example, [Cap’n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object.
- `nats_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient.
- `nats_queue_group` – Name for queue group of NATS subscribers. Default is the table name.
@@ -57,11 +59,13 @@ Optional parameters:
- `nats_reconnect_wait` – Amount of time in milliseconds to sleep between each reconnect attempt. Default: `5000`.
- `nats_server_list` - Server list for connection. Can be specified to connect to NATS cluster.
- `nats_skip_broken_messages` - NATS message parser tolerance to schema-incompatible messages per block. Default: `0`. If `nats_skip_broken_messages = N` then the engine skips *N* RabbitMQ messages that cannot be parsed (a message equals a row of data).
-- `nats_max_block_size` - Number of row collected by poll(s) for flushing data from NATS.
-- `nats_flush_interval_ms` - Timeout for flushing data read from NATS.
+- `nats_max_block_size` - Number of row collected by poll(s) for flushing data from NATS. Default: [max_insert_block_size](../../../operations/settings/settings.md#setting-max_insert_block_size).
+- `nats_flush_interval_ms` - Timeout for flushing data read from NATS. Default: [stream_flush_interval_ms](../../../operations/settings/settings.md#stream-flush-interval-ms).
- `nats_username` - NATS username.
- `nats_password` - NATS password.
- `nats_token` - NATS auth token.
+- `nats_startup_connect_tries` - Number of connect tries at startup. Default: `5`.
+- `nats_max_rows_per_message` — The maximum number of rows written in one NATS message for row-based formats. (default : `1`).
SSL connection:
@@ -159,6 +163,14 @@ If you want to change the target table by using `ALTER`, we recommend disabling
## Virtual Columns {#virtual-columns}
-- `_subject` - NATS message subject.
+- `_subject` - NATS message subject.
+
+## Data formats support {#data-formats-support}
+
+NATS engine supports all [formats](../../../interfaces/formats.md) supported in ClickHouse.
+The number of rows in one NATS message depends on whether the format is row-based or block-based:
+
+- For row-based formats the number of rows in one NATS message can be controlled by setting `nats_max_rows_per_message`.
+- For block-based formats we cannot divide block into smaller parts, but the number of rows in one block can be controlled by general setting [max_block_size](../../../operations/settings/settings.md#setting-max_block_size).
[Original article](https://clickhouse.com/docs/en/engines/table-engines/integrations/nats/)
diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md
index 9227e5cdbfd..2e5a45931f8 100644
--- a/docs/en/engines/table-engines/integrations/rabbitmq.md
+++ b/docs/en/engines/table-engines/integrations/rabbitmq.md
@@ -37,8 +37,16 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
[rabbitmq_persistent = 0,]
[rabbitmq_skip_broken_messages = N,]
[rabbitmq_max_block_size = N,]
- [rabbitmq_flush_interval_ms = N]
- [rabbitmq_queue_settings_list = 'x-dead-letter-exchange=my-dlx,x-max-length=10,x-overflow=reject-publish']
+ [rabbitmq_flush_interval_ms = N,]
+ [rabbitmq_queue_settings_list = 'x-dead-letter-exchange=my-dlx,x-max-length=10,x-overflow=reject-publish',]
+ [rabbitmq_queue_consume = false,]
+ [rabbitmq_address = '',]
+ [rabbitmq_vhost = '/',]
+ [rabbitmq_queue_consume = false,]
+ [rabbitmq_username = '',]
+ [rabbitmq_password = '',]
+ [rabbitmq_commit_on_select = false,]
+ [rabbitmq_max_rows_per_message = 1]
```
Required parameters:
@@ -49,19 +57,27 @@ Required parameters:
Optional parameters:
-- `rabbitmq_exchange_type` – The type of RabbitMQ exchange: `direct`, `fanout`, `topic`, `headers`, `consistent_hash`. Default: `fanout`.
-- `rabbitmq_routing_key_list` – A comma-separated list of routing keys.
-- `rabbitmq_row_delimiter` – Delimiter character, which ends the message.
-- `rabbitmq_schema` – Parameter that must be used if the format requires a schema definition. For example, [Cap’n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object.
-- `rabbitmq_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient.
-- `rabbitmq_num_queues` – Total number of queues. Default: `1`. Increasing this number can significantly improve performance.
-- `rabbitmq_queue_base` - Specify a hint for queue names. Use cases of this setting are described below.
-- `rabbitmq_deadletter_exchange` - Specify name for a [dead letter exchange](https://www.rabbitmq.com/dlx.html). You can create another table with this exchange name and collect messages in cases when they are republished to dead letter exchange. By default dead letter exchange is not specified.
-- `rabbitmq_persistent` - If set to 1 (true), in insert query delivery mode will be set to 2 (marks messages as 'persistent'). Default: `0`.
-- `rabbitmq_skip_broken_messages` – RabbitMQ message parser tolerance to schema-incompatible messages per block. Default: `0`. If `rabbitmq_skip_broken_messages = N` then the engine skips *N* RabbitMQ messages that cannot be parsed (a message equals a row of data).
-- `rabbitmq_max_block_size`
-- `rabbitmq_flush_interval_ms`
-- `rabbitmq_queue_settings_list` - allows to set RabbitMQ settings when creating a queue. Available settings: `x-max-length`, `x-max-length-bytes`, `x-message-ttl`, `x-expires`, `x-priority`, `x-max-priority`, `x-overflow`, `x-dead-letter-exchange`, `x-queue-type`. The `durable` setting is enabled automatically for the queue.
+- `rabbitmq_exchange_type` – The type of RabbitMQ exchange: `direct`, `fanout`, `topic`, `headers`, `consistent_hash`. Default: `fanout`.
+- `rabbitmq_routing_key_list` – A comma-separated list of routing keys.
+- `rabbitmq_row_delimiter` – Delimiter character, which ends the message. **This setting is deprecated and is no longer used, not left for compatibility reasons.**
+- `rabbitmq_schema` – Parameter that must be used if the format requires a schema definition. For example, [Cap’n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object.
+- `rabbitmq_num_consumers` – The number of consumers per table. Specify more consumers if the throughput of one consumer is insufficient. Default: `1`
+- `rabbitmq_num_queues` – Total number of queues. Increasing this number can significantly improve performance. Default: `1`.
+- `rabbitmq_queue_base` - Specify a hint for queue names. Use cases of this setting are described below.
+- `rabbitmq_deadletter_exchange` - Specify name for a [dead letter exchange](https://www.rabbitmq.com/dlx.html). You can create another table with this exchange name and collect messages in cases when they are republished to dead letter exchange. By default dead letter exchange is not specified.
+- `rabbitmq_persistent` - If set to 1 (true), in insert query delivery mode will be set to 2 (marks messages as 'persistent'). Default: `0`.
+- `rabbitmq_skip_broken_messages` – RabbitMQ message parser tolerance to schema-incompatible messages per block. If `rabbitmq_skip_broken_messages = N` then the engine skips *N* RabbitMQ messages that cannot be parsed (a message equals a row of data). Default: `0`.
+- `rabbitmq_max_block_size` - Number of row collected before flushing data from RabbitMQ. Default: [max_insert_block_size](../../../operations/settings/settings.md#setting-max_insert_block_size).
+- `rabbitmq_flush_interval_ms` - Timeout for flushing data from RabbitMQ. Default: [stream_flush_interval_ms](../../../operations/settings/settings.md#stream-flush-interval-ms).
+- `rabbitmq_queue_settings_list` - allows to set RabbitMQ settings when creating a queue. Available settings: `x-max-length`, `x-max-length-bytes`, `x-message-ttl`, `x-expires`, `x-priority`, `x-max-priority`, `x-overflow`, `x-dead-letter-exchange`, `x-queue-type`. The `durable` setting is enabled automatically for the queue.
+- `rabbitmq_address` - Address for connection. Use ether this setting or `rabbitmq_host_port`.
+- `rabbitmq_vhost` - RabbitMQ vhost. Default: `'\'`.
+- `rabbitmq_queue_consume` - Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings. Default: `false`.
+- `rabbitmq_username` - RabbitMQ username.
+- `rabbitmq_password` - RabbitMQ password.
+- `rabbitmq_commit_on_select` - Commit messages when select query is made. Default: `false`.
+- `rabbitmq_max_rows_per_message` — The maximum number of rows written in one RabbitMQ message for row-based formats. Default : `1`.
+
SSL connection:
@@ -166,11 +182,20 @@ Example:
## Virtual Columns {#virtual-columns}
-- `_exchange_name` - RabbitMQ exchange name.
-- `_channel_id` - ChannelID, on which consumer, who received the message, was declared.
-- `_delivery_tag` - DeliveryTag of the received message. Scoped per channel.
-- `_redelivered` - `redelivered` flag of the message.
-- `_message_id` - messageID of the received message; non-empty if was set, when message was published.
-- `_timestamp` - timestamp of the received message; non-empty if was set, when message was published.
+- `_exchange_name` - RabbitMQ exchange name.
+- `_channel_id` - ChannelID, on which consumer, who received the message, was declared.
+- `_delivery_tag` - DeliveryTag of the received message. Scoped per channel.
+- `_redelivered` - `redelivered` flag of the message.
+- `_message_id` - messageID of the received message; non-empty if was set, when message was published.
+- `_timestamp` - timestamp of the received message; non-empty if was set, when message was published.
+
+## Data formats support {#data-formats-support}
+
+RabbitMQ engine supports all [formats](../../../interfaces/formats.md) supported in ClickHouse.
+The number of rows in one RabbitMQ message depends on whether the format is row-based or block-based:
+
+- For row-based formats the number of rows in one RabbitMQ message can be controlled by setting `rabbitmq_max_rows_per_message`.
+- For block-based formats we cannot divide block into smaller parts, but the number of rows in one block can be controlled by general setting [max_block_size](../../../operations/settings/settings.md#setting-max_block_size).
+
[Original article](https://clickhouse.com/docs/en/engines/table-engines/integrations/rabbitmq/)
diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md
index 4f07f99fb26..e3b40d83efe 100644
--- a/docs/en/interfaces/cli.md
+++ b/docs/en/interfaces/cli.md
@@ -2,11 +2,10 @@
slug: /en/interfaces/cli
sidebar_position: 17
sidebar_label: Command-Line Client
+title: Command-Line Client
---
import ConnectionDetails from '@site/docs/en/_snippets/_gather_your_details_native.md';
-# Command-line Client
-
## clickhouse-client
ClickHouse provides a native command-line client: `clickhouse-client`. The client supports command-line options and configuration files. For more information, see [Configuring](#interfaces_cli_configuration).
diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md
index b7ef859f974..3fe26fa8eff 100644
--- a/docs/en/interfaces/formats.md
+++ b/docs/en/interfaces/formats.md
@@ -85,7 +85,7 @@ The supported formats are:
| [MySQLDump](#mysqldump) | ✔ | ✗ |
-You can control some format processing parameters with the ClickHouse settings. For more information read the [Settings](../operations/settings/settings.md) section.
+You can control some format processing parameters with the ClickHouse settings. For more information read the [Settings](/docs/en/operations/settings/settings.md) section.
## TabSeparated {#tabseparated}
@@ -148,12 +148,12 @@ Only a small set of symbols are escaped. You can easily stumble onto a string va
Arrays are written as a list of comma-separated values in square brackets. Number items in the array are formatted as normally. `Date` and `DateTime` types are written in single quotes. Strings are written in single quotes with the same escaping rules as above.
-[NULL](../sql-reference/syntax.md) is formatted according to setting [format_tsv_null_representation](../operations/settings/settings.md#format_tsv_null_representation) (default value is `\N`).
+[NULL](/docs/en/sql-reference/syntax.md) is formatted according to setting [format_tsv_null_representation](/docs/en/operations/settings/settings.md/#format_tsv_null_representation) (default value is `\N`).
In input data, ENUM values can be represented as names or as ids. First, we try to match the input value to the ENUM name. If we fail and the input value is a number, we try to match this number to ENUM id.
-If input data contains only ENUM ids, it's recommended to enable the setting [input_format_tsv_enum_as_number](../operations/settings/settings.md#input_format_tsv_enum_as_number) to optimize ENUM parsing.
+If input data contains only ENUM ids, it's recommended to enable the setting [input_format_tsv_enum_as_number](/docs/en/operations/settings/settings.md/#input_format_tsv_enum_as_number) to optimize ENUM parsing.
-Each element of [Nested](../sql-reference/data-types/nested-data-structures/nested.md) structures is represented as array.
+Each element of [Nested](/docs/en/sql-reference/data-types/nested-data-structures/nested.md) structures is represented as an array.
For example:
@@ -183,12 +183,12 @@ SELECT * FROM nestedt FORMAT TSV
### TabSeparated format settings {#tabseparated-format-settings}
-- [format_tsv_null_representation](../operations/settings/settings.md#format_tsv_null_representation) - custom NULL representation in TSV format. Default value - `\N`.
-- [input_format_tsv_empty_as_default](../operations/settings/settings.md#input_format_tsv_empty_as_default) - treat empty fields in TSV input as default values. Default value - `false`. For complex default expressions [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#input_format_defaults_for_omitted_fields) must be enabled too.
-- [input_format_tsv_enum_as_number](../operations/settings/settings.md#input_format_tsv_enum_as_number) - treat inserted enum values in TSV formats as enum indices. Default value - `false`.
-- [input_format_tsv_use_best_effort_in_schema_inference](../operations/settings/settings.md#input_format_tsv_use_best_effort_in_schema_inference) - use some tweaks and heuristics to infer schema in TSV format. If disabled, all fields will be inferred as Strings. Default value - `true`.
-- [output_format_tsv_crlf_end_of_line](../operations/settings/settings.md#output_format_tsv_crlf_end_of_line) - if it is set true, end of line in TSV output format will be `\r\n` instead of `\n`. Default value - `false`.
-- [input_format_tsv_skip_first_lines](../operations/settings/settings.md#input_format_tsv_skip_first_lines) - skip specified number of lines at the beginning of data. Default value - `0`.
+- [format_tsv_null_representation](/docs/en/operations/settings/settings.md/#format_tsv_null_representation) - custom NULL representation in TSV format. Default value - `\N`.
+- [input_format_tsv_empty_as_default](/docs/en/operations/settings/settings.md/#input_format_tsv_empty_as_default) - treat empty fields in TSV input as default values. Default value - `false`. For complex default expressions [input_format_defaults_for_omitted_fields](/docs/en/operations/settings/settings.md/#input_format_defaults_for_omitted_fields) must be enabled too.
+- [input_format_tsv_enum_as_number](/docs/en/operations/settings/settings.md/#input_format_tsv_enum_as_number) - treat inserted enum values in TSV formats as enum indices. Default value - `false`.
+- [input_format_tsv_use_best_effort_in_schema_inference](/docs/en/operations/settings/settings.md/#input_format_tsv_use_best_effort_in_schema_inference) - use some tweaks and heuristics to infer schema in TSV format. If disabled, all fields will be inferred as Strings. Default value - `true`.
+- [output_format_tsv_crlf_end_of_line](/docs/en/operations/settings/settings.md/#output_format_tsv_crlf_end_of_line) - if it is set true, end of line in TSV output format will be `\r\n` instead of `\n`. Default value - `false`.
+- [input_format_tsv_skip_first_lines](/docs/en/operations/settings/settings.md/#input_format_tsv_skip_first_lines) - skip specified number of lines at the beginning of data. Default value - `0`.
## TabSeparatedRaw {#tabseparatedraw}
@@ -204,8 +204,8 @@ Differs from the `TabSeparated` format in that the column names are written in t
During parsing, the first row is expected to contain the column names. You can use column names to determine their position and to check their correctness.
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from the input data will be mapped to the columns of the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
:::
@@ -216,10 +216,10 @@ This format is also available under the name `TSVWithNames`.
Differs from the `TabSeparated` format in that the column names are written to the first row, while the column types are in the second row.
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from the input data will be mapped to the columns in the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
-If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1,
+If setting [input_format_with_types_use_header](/docs/en/operations/settings/settings.md/#input_format_with_types_use_header) is set to 1,
the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped.
:::
@@ -245,7 +245,7 @@ This format allows specifying a custom format string with placeholders for value
It uses settings `format_template_resultset`, `format_template_row`, `format_template_rows_between_delimiter` and some settings of other formats (e.g. `output_format_json_quote_64bit_integers` when using `JSON` escaping, see further)
-Setting `format_template_row` specifies path to file, which contains format string for rows with the following syntax:
+Setting `format_template_row` specifies the path to the file containing format strings for rows with the following syntax:
`delimiter_1${column_1:serializeAs_1}delimiter_2${column_2:serializeAs_2} ... delimiter_N`,
@@ -253,10 +253,10 @@ where `delimiter_i` is a delimiter between values (`$` symbol can be escaped as
`column_i` is a name or index of a column whose values are to be selected or inserted (if empty, then column will be skipped),
`serializeAs_i` is an escaping rule for the column values. The following escaping rules are supported:
-- `CSV`, `JSON`, `XML` (similarly to the formats of the same names)
-- `Escaped` (similarly to `TSV`)
-- `Quoted` (similarly to `Values`)
-- `Raw` (without escaping, similarly to `TSVRaw`)
+- `CSV`, `JSON`, `XML` (similar to the formats of the same names)
+- `Escaped` (similar to `TSV`)
+- `Quoted` (similar to `Values`)
+- `Raw` (without escaping, similar to `TSVRaw`)
- `None` (no escaping rule, see further)
If an escaping rule is omitted, then `None` will be used. `XML` is suitable only for output.
@@ -269,9 +269,9 @@ the values of `SearchPhrase`, `c` and `price` columns, which are escaped as `Quo
`Search phrase: 'bathroom interior design', count: 2166, ad price: $3;`
-The `format_template_rows_between_delimiter` setting specifies delimiter between rows, which is printed (or expected) after every row except the last one (`\n` by default)
+The `format_template_rows_between_delimiter` setting specifies the delimiter between rows, which is printed (or expected) after every row except the last one (`\n` by default)
-Setting `format_template_resultset` specifies the path to file, which contains a format string for resultset. Format string for resultset has the same syntax as a format string for row and allows to specify a prefix, a suffix and a way to print some additional information. It contains the following placeholders instead of column names:
+Setting `format_template_resultset` specifies the path to the file, which contains a format string for resultset. Format string for resultset has the same syntax as a format string for row and allows to specify a prefix, a suffix and a way to print some additional information. It contains the following placeholders instead of column names:
- `data` is the rows with data in `format_template_row` format, separated by `format_template_rows_between_delimiter`. This placeholder must be the first placeholder in the format string.
- `totals` is the row with total values in `format_template_row` format (when using WITH TOTALS)
@@ -284,8 +284,8 @@ Setting `format_template_resultset` specifies the path to file, which contains a
- `bytes_read` is the number of bytes (uncompressed) has been read
The placeholders `data`, `totals`, `min` and `max` must not have escaping rule specified (or `None` must be specified explicitly). The remaining placeholders may have any escaping rule specified.
-If the `format_template_resultset` setting is an empty string, `${data}` is used as default value.
-For insert queries format allows skipping some columns or some fields if prefix or suffix (see example).
+If the `format_template_resultset` setting is an empty string, `${data}` is used as the default value.
+For insert queries format allows skipping some columns or fields if prefix or suffix (see example).
Select example:
@@ -373,8 +373,8 @@ All delimiters in the input data must be strictly equal to delimiters in specifi
## TemplateIgnoreSpaces {#templateignorespaces}
This format is suitable only for input.
-Similar to `Template`, but skips whitespace characters between delimiters and values in the input stream. However, if format strings contain whitespace characters, these characters will be expected in the input stream. Also allows to specify empty placeholders (`${}` or `${:None}`) to split some delimiter into separate parts to ignore spaces between them. Such placeholders are used only for skipping whitespace characters.
-It’s possible to read `JSON` using this format, if values of columns have the same order in all rows. For example, the following request can be used for inserting data from output example of format [JSON](#json):
+Similar to `Template`, but skips whitespace characters between delimiters and values in the input stream. However, if format strings contain whitespace characters, these characters will be expected in the input stream. Also allows specifying empty placeholders (`${}` or `${:None}`) to split some delimiter into separate parts to ignore spaces between them. Such placeholders are used only for skipping whitespace characters.
+It’s possible to read `JSON` using this format if the values of columns have the same order in all rows. For example, the following request can be used for inserting data from its output example of format [JSON](#json):
``` sql
INSERT INTO table_name SETTINGS
@@ -411,7 +411,7 @@ SearchPhrase=curtain designs count()=1064
SearchPhrase=baku count()=1000
```
-[NULL](../sql-reference/syntax.md) is formatted as `\N`.
+[NULL](/docs/en/sql-reference/syntax.md) is formatted as `\N`.
``` sql
SELECT * FROM t_null FORMAT TSKV
@@ -427,49 +427,49 @@ Both data output and parsing are supported in this format. For parsing, any orde
Parsing allows the presence of the additional field `tskv` without the equal sign or a value. This field is ignored.
-During import, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+During import, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
## CSV {#csv}
Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)).
-When formatting, rows are enclosed in double-quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double-quotes. Numbers are output without quotes. Values are separated by a delimiter character, which is `,` by default. The delimiter character is defined in the setting [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter). Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first, the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double-quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost).
+When formatting, rows are enclosed in double quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double quotes. Numbers are output without quotes. Values are separated by a delimiter character, which is `,` by default. The delimiter character is defined in the setting [format_csv_delimiter](/docs/en/operations/settings/settings.md/#format_csv_delimiter). Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first, the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost).
``` bash
$ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv
```
-\*By default, the delimiter is `,`. See the [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter) setting for more information.
+\*By default, the delimiter is `,`. See the [format_csv_delimiter](/docs/en/operations/settings/settings.md/#format_csv_delimiter) setting for more information.
When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to the delimiter character or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) types are all supported.
-`NULL` is formatted according to setting [format_csv_null_representation](../operations/settings/settings.md#format_csv_null_representation) (default value is `\N`).
+`NULL` is formatted according to setting [format_csv_null_representation](/docs/en/operations/settings/settings.md/#format_csv_null_representation) (default value is `\N`).
-In input data, ENUM values can be represented as names or as ids. First, we try to match the input value to the ENUM name. If we fail and the input value is a number, we try to match this number to ENUM id.
-If input data contains only ENUM ids, it's recommended to enable the setting [input_format_csv_enum_as_number](../operations/settings/settings.md#input_format_csv_enum_as_number) to optimize ENUM parsing.
+In input data, ENUM values can be represented as names or as ids. First, we try to match the input value to the ENUM name. If we fail and the input value is a number, we try to match this number to the ENUM id.
+If input data contains only ENUM ids, it's recommended to enable the setting [input_format_csv_enum_as_number](/docs/en/operations/settings/settings.md/#input_format_csv_enum_as_number) to optimize ENUM parsing.
The CSV format supports the output of totals and extremes the same way as `TabSeparated`.
### CSV format settings {#csv-format-settings}
-- [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter) - the character to be considered as a delimiter in CSV data. Default value - `,`.
-- [format_csv_allow_single_quotes](../operations/settings/settings.md#format_csv_allow_single_quotes) - allow strings in single quotes. Default value - `true`.
-- [format_csv_allow_double_quotes](../operations/settings/settings.md#format_csv_allow_double_quotes) - allow strings in double quotes. Default value - `true`.
-- [format_csv_null_representation](../operations/settings/settings.md#format_tsv_null_representation) - custom NULL representation in CSV format. Default value - `\N`.
-- [input_format_csv_empty_as_default](../operations/settings/settings.md#input_format_csv_empty_as_default) - treat empty fields in CSV input as default values. Default value - `true`. For complex default expressions [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#input_format_defaults_for_omitted_fields) must be enabled too.
-- [input_format_csv_enum_as_number](../operations/settings/settings.md#input_format_csv_enum_as_number) - treat inserted enum values in CSV formats as enum indices. Default value - `false`.
-- [input_format_csv_use_best_effort_in_schema_inference](../operations/settings/settings.md#input_format_csv_use_best_effort_in_schema_inference) - use some tweaks and heuristics to infer schema in CSV format. If disabled, all fields will be inferred as Strings. Default value - `true`.
-- [input_format_csv_arrays_as_nested_csv](../operations/settings/settings.md#input_format_csv_arrays_as_nested_csv) - when reading Array from CSV, expect that its elements were serialized in nested CSV and then put into string. Default value - `false`.
-- [output_format_csv_crlf_end_of_line](../operations/settings/settings.md#output_format_csv_crlf_end_of_line) - if it is set true, end of line in CSV output format will be `\r\n` instead of `\n`. Default value - `false`.
-- [input_format_csv_skip_first_lines](../operations/settings/settings.md#input_format_csv_skip_first_lines) - skip specified number of lines at the beginning of data. Default value - `0`.
+- [format_csv_delimiter](/docs/en/operations/settings/settings.md/#format_csv_delimiter) - the character to be considered as a delimiter in CSV data. Default value - `,`.
+- [format_csv_allow_single_quotes](/docs/en/operations/settings/settings.md/#format_csv_allow_single_quotes) - allow strings in single quotes. Default value - `true`.
+- [format_csv_allow_double_quotes](/docs/en/operations/settings/settings.md/#format_csv_allow_double_quotes) - allow strings in double quotes. Default value - `true`.
+- [format_csv_null_representation](/docs/en/operations/settings/settings.md/#format_tsv_null_representation) - custom NULL representation in CSV format. Default value - `\N`.
+- [input_format_csv_empty_as_default](/docs/en/operations/settings/settings.md/#input_format_csv_empty_as_default) - treat empty fields in CSV input as default values. Default value - `true`. For complex default expressions, [input_format_defaults_for_omitted_fields](/docs/en/operations/settings/settings.md/#input_format_defaults_for_omitted_fields) must be enabled too.
+- [input_format_csv_enum_as_number](/docs/en/operations/settings/settings.md/#input_format_csv_enum_as_number) - treat inserted enum values in CSV formats as enum indices. Default value - `false`.
+- [input_format_csv_use_best_effort_in_schema_inference](/docs/en/operations/settings/settings.md/#input_format_csv_use_best_effort_in_schema_inference) - use some tweaks and heuristics to infer schema in CSV format. If disabled, all fields will be inferred as Strings. Default value - `true`.
+- [input_format_csv_arrays_as_nested_csv](/docs/en/operations/settings/settings.md/#input_format_csv_arrays_as_nested_csv) - when reading Array from CSV, expect that its elements were serialized in nested CSV and then put into string. Default value - `false`.
+- [output_format_csv_crlf_end_of_line](/docs/en/operations/settings/settings.md/#output_format_csv_crlf_end_of_line) - if it is set to true, end of line in CSV output format will be `\r\n` instead of `\n`. Default value - `false`.
+- [input_format_csv_skip_first_lines](/docs/en/operations/settings/settings.md/#input_format_csv_skip_first_lines) - skip the specified number of lines at the beginning of data. Default value - `0`.
## CSVWithNames {#csvwithnames}
Also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames).
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
:::
@@ -478,16 +478,16 @@ Otherwise, the first row will be skipped.
Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes).
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
-If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1,
+If setting [input_format_with_types_use_header](/docs/en/operations/settings/settings.md/#input_format_with_types_use_header) is set to 1,
the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped.
:::
## CustomSeparated {#format-customseparated}
-Similar to [Template](#format-template), but it prints or reads all names and types of columns and uses escaping rule from [format_custom_escaping_rule](../operations/settings/settings.md#format_custom_escaping_rule) setting and delimiters from [format_custom_field_delimiter](../operations/settings/settings.md#format_custom_field_delimiter), [format_custom_row_before_delimiter](../operations/settings/settings.md#format_custom_row_before_delimiter), [format_custom_row_after_delimiter](../operations/settings/settings.md#format_custom_row_after_delimiter), [format_custom_row_between_delimiter](../operations/settings/settings.md#format_custom_row_between_delimiter), [format_custom_result_before_delimiter](../operations/settings/settings.md#format_custom_result_before_delimiter) and [format_custom_result_after_delimiter](../operations/settings/settings.md#format_custom_result_after_delimiter) settings, not from format strings.
+Similar to [Template](#format-template), but it prints or reads all names and types of columns and uses escaping rule from [format_custom_escaping_rule](/docs/en/operations/settings/settings.md/#format_custom_escaping_rule) setting and delimiters from [format_custom_field_delimiter](/docs/en/operations/settings/settings.md/#format_custom_field_delimiter), [format_custom_row_before_delimiter](/docs/en/operations/settings/settings.md/#format_custom_row_before_delimiter), [format_custom_row_after_delimiter](/docs/en/operations/settings/settings.md/#format_custom_row_after_delimiter), [format_custom_row_between_delimiter](/docs/en/operations/settings/settings.md/#format_custom_row_between_delimiter), [format_custom_result_before_delimiter](/docs/en/operations/settings/settings.md/#format_custom_result_before_delimiter) and [format_custom_result_after_delimiter](/docs/en/operations/settings/settings.md/#format_custom_result_after_delimiter) settings, not from format strings.
There is also `CustomSeparatedIgnoreSpaces` format, which is similar to [TemplateIgnoreSpaces](#templateignorespaces).
@@ -496,8 +496,8 @@ There is also `CustomSeparatedIgnoreSpaces` format, which is similar to [Templat
Also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames).
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
:::
@@ -506,10 +506,10 @@ Otherwise, the first row will be skipped.
Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes).
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
-If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1,
+If setting [input_format_with_types_use_header](/docs/en/operations/settings/settings.md/#input_format_with_types_use_header) is set to 1,
the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped.
:::
@@ -531,15 +531,15 @@ INSERT INTO table (x, y, z) VALUES (6, 7, 'Hello'), (7, 8, 'Hello');
INSERT INTO table (x, y, z) VALUES (8, 9, 'Hello'), (9, 10, 'Hello');
```
-To read data output by this format ypu can use [MySQLDump](#mysqldump) input format.
+To read data output by this format you can use [MySQLDump](#mysqldump) input format.
### SQLInsert format settings {#sqlinsert-format-settings}
-- [output_format_sql_insert_max_batch_size](../operations/settings/settings.md#output_format_sql_insert_max_batch_size) - The maximum number of rows in one INSERT statement. Default value - `65505`.
-- [output_format_sql_insert_table_name](../operations/settings/settings.md#output_format_sql_insert_table_name) - The name of table in the output INSERT query. Default value - `'table'`.
-- [output_format_sql_insert_include_column_names](../operations/settings/settings.md#output_format_sql_insert_include_column_names) - Include column names in INSERT query. Default value - `true`.
-- [output_format_sql_insert_use_replace](../operations/settings/settings.md#output_format_sql_insert_use_replace) - Use REPLACE statement instead of INSERT. Default value - `false`.
-- [output_format_sql_insert_quote_names](../operations/settings/settings.md#output_format_sql_insert_quote_names) - Quote column names with "\`" characters . Default value - `true`.
+- [output_format_sql_insert_max_batch_size](/docs/en/operations/settings/settings.md/#output_format_sql_insert_max_batch_size) - The maximum number of rows in one INSERT statement. Default value - `65505`.
+- [output_format_sql_insert_table_name](/docs/en/operations/settings/settings.md/#output_format_sql_insert_table_name) - The name of the table in the output INSERT query. Default value - `'table'`.
+- [output_format_sql_insert_include_column_names](/docs/en/operations/settings/settings.md/#output_format_sql_insert_include_column_names) - Include column names in INSERT query. Default value - `true`.
+- [output_format_sql_insert_use_replace](/docs/en/operations/settings/settings.md/#output_format_sql_insert_use_replace) - Use REPLACE statement instead of INSERT. Default value - `false`.
+- [output_format_sql_insert_quote_names](/docs/en/operations/settings/settings.md/#output_format_sql_insert_quote_names) - Quote column names with "\`" characters. Default value - `true`.
## JSON {#json}
@@ -599,7 +599,7 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA
}
```
-The JSON is compatible with JavaScript. To ensure this, some characters are additionally escaped: the slash `/` is escaped as `\/`; alternative line breaks `U+2028` and `U+2029`, which break some browsers, are escaped as `\uXXXX`. ASCII control characters are escaped: backspace, form feed, line feed, carriage return, and horizontal tab are replaced with `\b`, `\f`, `\n`, `\r`, `\t` , as well as the remaining bytes in the 00-1F range using `\uXXXX` sequences. Invalid UTF-8 sequences are changed to the replacement character � so the output text will consist of valid UTF-8 sequences. For compatibility with JavaScript, Int64 and UInt64 integers are enclosed in double-quotes by default. To remove the quotes, you can set the configuration parameter [output_format_json_quote_64bit_integers](../operations/settings/settings.md#output_format_json_quote_64bit_integers) to 0.
+The JSON is compatible with JavaScript. To ensure this, some characters are additionally escaped: the slash `/` is escaped as `\/`; alternative line breaks `U+2028` and `U+2029`, which break some browsers, are escaped as `\uXXXX`. ASCII control characters are escaped: backspace, form feed, line feed, carriage return, and horizontal tab are replaced with `\b`, `\f`, `\n`, `\r`, `\t` , as well as the remaining bytes in the 00-1F range using `\uXXXX` sequences. Invalid UTF-8 sequences are changed to the replacement character � so the output text will consist of valid UTF-8 sequences. For compatibility with JavaScript, Int64 and UInt64 integers are enclosed in double quotes by default. To remove the quotes, you can set the configuration parameter [output_format_json_quote_64bit_integers](/docs/en/operations/settings/settings.md/#output_format_json_quote_64bit_integers) to 0.
`rows` – The total number of output rows.
@@ -610,14 +610,14 @@ If the query contains GROUP BY, rows_before_limit_at_least is the exact number o
`extremes` – Extreme values (when extremes are set to 1).
-ClickHouse supports [NULL](../sql-reference/syntax.md), which is displayed as `null` in the JSON output. To enable `+nan`, `-nan`, `+inf`, `-inf` values in output, set the [output_format_json_quote_denormals](../operations/settings/settings.md#output_format_json_quote_denormals) to 1.
+ClickHouse supports [NULL](/docs/en/sql-reference/syntax.md), which is displayed as `null` in the JSON output. To enable `+nan`, `-nan`, `+inf`, `-inf` values in output, set the [output_format_json_quote_denormals](/docs/en/operations/settings/settings.md/#output_format_json_quote_denormals) to 1.
**See Also**
- [JSONEachRow](#jsoneachrow) format
-- [output_format_json_array_of_rows](../operations/settings/settings.md#output_format_json_array_of_rows) setting
+- [output_format_json_array_of_rows](/docs/en/operations/settings/settings.md/#output_format_json_array_of_rows) setting
-For JSON input format, if setting [input_format_json_validate_types_from_metadata](../operations/settings/settings.md#input_format_json_validate_types_from_metadata) is set to 1,
+For JSON input format, if setting [input_format_json_validate_types_from_metadata](/docs/en/operations/settings/settings.md/#input_format_json_validate_types_from_metadata) is set to 1,
the types from metadata in input data will be compared with the types of the corresponding columns from the table.
## JSONStrings {#jsonstrings}
@@ -690,8 +690,8 @@ Example:
}
```
-During import, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
-Columns that are not present in the block will be filled with default values (you can use [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#input_format_defaults_for_omitted_fields) setting here)
+During import, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
+Columns that are not present in the block will be filled with default values (you can use the [input_format_defaults_for_omitted_fields](/docs/en/operations/settings/settings.md/#input_format_defaults_for_omitted_fields) setting here)
## JSONColumnsWithMetadata {#jsoncolumnsmonoblock}
@@ -739,14 +739,14 @@ Example:
}
```
-For JSONColumnsWithMetadata input format, if setting [input_format_json_validate_types_from_metadata](../operations/settings/settings.md#input_format_json_validate_types_from_metadata) is set to 1,
+For JSONColumnsWithMetadata input format, if setting [input_format_json_validate_types_from_metadata](/docs/en/operations/settings/settings.md/#input_format_json_validate_types_from_metadata) is set to 1,
the types from metadata in input data will be compared with the types of the corresponding columns from the table.
## JSONAsString {#jsonasstring}
In this format, a single JSON object is interpreted as a single value. If the input has several JSON objects (comma separated), they are interpreted as separate rows. If the input data is enclosed in square brackets, it is interpreted as an array of JSONs.
-This format can only be parsed for table with a single field of type [String](../sql-reference/data-types/string.md). The remaining columns must be set to [DEFAULT](../sql-reference/statements/create/table.md#default) or [MATERIALIZED](../sql-reference/statements/create/table.md#materialized), or omitted. Once you collect whole JSON object to string you can use [JSON functions](../sql-reference/functions/json-functions.md) to process it.
+This format can only be parsed for a table with a single field of type [String](/docs/en/sql-reference/data-types/string.md). The remaining columns must be set to [DEFAULT](/docs/en/sql-reference/statements/create/table.md/#default) or [MATERIALIZED](/docs/en/sql-reference/statements/create/table.md/#materialized), or omitted. Once you collect the whole JSON object to string you can use [JSON functions](/docs/en/sql-reference/functions/json-functions.md) to process it.
**Examples**
@@ -891,7 +891,7 @@ Example:
]
```
-Columns that are not present in the block will be filled with default values (you can use [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#input_format_defaults_for_omitted_fields) setting here)
+Columns that are not present in the block will be filled with default values (you can use [input_format_defaults_for_omitted_fields](/docs/en/operations/settings/settings.md/#input_format_defaults_for_omitted_fields) setting here)
## JSONEachRow {#jsoneachrow}
@@ -905,7 +905,7 @@ Example:
{"num":44,"str":"hello","arr":[0,1,2,3]}
```
-While importing data columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+While importing data columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
## JSONStringsEachRow {#jsonstringseachrow}
@@ -960,8 +960,8 @@ Differs from `JSONEachRow`/`JSONStringsEachRow` in that ClickHouse will also yie
Differs from `JSONCompactEachRow` format in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames).
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
:::
@@ -970,10 +970,10 @@ Otherwise, the first row will be skipped.
Differs from `JSONCompactEachRow` format in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes).
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
-If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1,
+If setting [input_format_with_types_use_header](/docs/en/operations/settings/settings.md/#input_format_with_types_use_header) is set to 1,
the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped.
:::
@@ -982,8 +982,8 @@ the types from input data will be compared with the types of the corresponding c
Differs from `JSONCompactStringsEachRow` in that in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames).
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
:::
@@ -992,10 +992,10 @@ Otherwise, the first row will be skipped.
Differs from `JSONCompactStringsEachRow` in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes).
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
-If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1,
+If setting [input_format_with_types_use_header](/docs/en/operations/settings/settings.md/#input_format_with_types_use_header) is set to 1,
the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped.
:::
@@ -1009,7 +1009,7 @@ the types from input data will be compared with the types of the corresponding c
## JSONObjectEachRow {#jsonobjecteachrow}
-In this format, all data is represented as a single JSON Object, each row is represented as separate field of this object similar to JSONEachRow format.
+In this format, all data is represented as a single JSON Object, each row is represented as a separate field of this object similar to JSONEachRow format.
Example:
@@ -1021,12 +1021,12 @@ Example:
}
```
-To use object name as column value you can use special setting [format_json_object_each_row_column_for_object_name](../operations/settings/settings.md#format_json_object_each_row_column_for_object_name). Value of this setting is set to the name of a column, that is used as JSON key for a row in resulting object.
+To use an object name as a column value you can use the special setting [format_json_object_each_row_column_for_object_name](/docs/en/operations/settings/settings.md/#format_json_object_each_row_column_for_object_name). The value of this setting is set to the name of a column, that is used as JSON key for a row in the resulting object.
Examples:
For output:
-Let's say we have table `test` with two columns:
+Let's say we have the table `test` with two columns:
```
┌─object_name─┬─number─┐
│ first_obj │ 1 │
@@ -1051,7 +1051,7 @@ The output:
For input:
-Let's say we stored output from previous example in a file with name `data.json`:
+Let's say we stored output from the previous example in a file named `data.json`:
```sql
select * from file('data.json', JSONObjectEachRow, 'object_name String, number UInt64') settings format_json_object_each_row_column_for_object_name='object_name'
```
@@ -1093,9 +1093,9 @@ ClickHouse ignores spaces between elements and commas after the objects. You can
**Omitted values processing**
-ClickHouse substitutes omitted values with the default values for the corresponding [data types](../sql-reference/data-types/index.md).
+ClickHouse substitutes omitted values with the default values for the corresponding [data types](/docs/en/sql-reference/data-types/index.md).
-If `DEFAULT expr` is specified, ClickHouse uses different substitution rules depending on the [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#input_format_defaults_for_omitted_fields) setting.
+If `DEFAULT expr` is specified, ClickHouse uses different substitution rules depending on the [input_format_defaults_for_omitted_fields](/docs/en/operations/settings/settings.md/#input_format_defaults_for_omitted_fields) setting.
Consider the following table:
@@ -1140,7 +1140,7 @@ Any set of bytes can be output in the strings. Use the `JSONEachRow` format if y
### Usage of Nested Structures {#jsoneachrow-nested}
-If you have a table with [Nested](../sql-reference/data-types/nested-data-structures/nested.md) data type columns, you can insert JSON data with the same structure. Enable this feature with the [input_format_import_nested_json](../operations/settings/settings.md#input_format_import_nested_json) setting.
+If you have a table with [Nested](/docs/en/sql-reference/data-types/nested-data-structures/nested.md) data type columns, you can insert JSON data with the same structure. Enable this feature with the [input_format_import_nested_json](/docs/en/operations/settings/settings.md/#input_format_import_nested_json) setting.
For example, consider the following table:
@@ -1154,7 +1154,7 @@ As you can see in the `Nested` data type description, ClickHouse treats each com
INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n.s": ["abc", "def"], "n.i": [1, 23]}
```
-To insert data as a hierarchical JSON object, set [input_format_import_nested_json=1](../operations/settings/settings.md#input_format_import_nested_json).
+To insert data as a hierarchical JSON object, set [input_format_import_nested_json=1](/docs/en/operations/settings/settings.md/#input_format_import_nested_json).
``` json
{
@@ -1199,18 +1199,18 @@ SELECT * FROM json_each_row_nested
### JSON formats settings {#json-formats-settings}
-- [input_format_import_nested_json](../operations/settings/settings.md#input_format_import_nested_json) - map nested JSON data to nested tables (it works for JSONEachRow format). Default value - `false`.
-- [input_format_json_read_bools_as_numbers](../operations/settings/settings.md#input_format_json_read_bools_as_numbers) - allow to parse bools as numbers in JSON input formats. Default value - `true`.
-- [input_format_json_read_numbers_as_strings](../operations/settings/settings.md#input_format_json_read_numbers_as_strings) - allow to parse numbers as strings in JSON input formats. Default value - `false`.
-- [input_format_json_read_objects_as_strings](../operations/settings/settings.md#input_format_json_read_objects_as_strings) - allow to parse JSON objects as strings in JSON input formats. Default value - `false`.
-- [output_format_json_quote_64bit_integers](../operations/settings/settings.md#output_format_json_quote_64bit_integers) - controls quoting of 64-bit integers in JSON output format. Default value - `true`.
-- [output_format_json_quote_64bit_floats](../operations/settings/settings.md#output_format_json_quote_64bit_floats) - controls quoting of 64-bit floats in JSON output format. Default value - `false`.
-- [output_format_json_quote_denormals](../operations/settings/settings.md#output_format_json_quote_denormals) - enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format. Default value - `false`.
-- [output_format_json_quote_decimals](../operations/settings/settings.md#output_format_json_quote_decimals) - controls quoting of decimals in JSON output format. Default value - `false`.
-- [output_format_json_escape_forward_slashes](../operations/settings/settings.md#output_format_json_escape_forward_slashes) - controls escaping forward slashes for string outputs in JSON output format. Default value - `true`.
-- [output_format_json_named_tuples_as_objects](../operations/settings/settings.md#output_format_json_named_tuples_as_objects) - serialize named tuple columns as JSON objects. Default value - `false`.
-- [output_format_json_array_of_rows](../operations/settings/settings.md#output_format_json_array_of_rows) - output a JSON array of all rows in JSONEachRow(Compact) format. Default value - `false`.
-- [output_format_json_validate_utf8](../operations/settings/settings.md#output_format_json_validate_utf8) - enables validation of UTF-8 sequences in JSON output formats (note that it doesn't impact formats JSON/JSONCompact/JSONColumnsWithMetadata, they always validate utf8). Default value - `false`.
+- [input_format_import_nested_json](/docs/en/operations/settings/settings.md/#input_format_import_nested_json) - map nested JSON data to nested tables (it works for JSONEachRow format). Default value - `false`.
+- [input_format_json_read_bools_as_numbers](/docs/en/operations/settings/settings.md/#input_format_json_read_bools_as_numbers) - allow to parse bools as numbers in JSON input formats. Default value - `true`.
+- [input_format_json_read_numbers_as_strings](/docs/en/operations/settings/settings.md/#input_format_json_read_numbers_as_strings) - allow to parse numbers as strings in JSON input formats. Default value - `false`.
+- [input_format_json_read_objects_as_strings](/docs/en/operations/settings/settings.md/#input_format_json_read_objects_as_strings) - allow to parse JSON objects as strings in JSON input formats. Default value - `false`.
+- [output_format_json_quote_64bit_integers](/docs/en/operations/settings/settings.md/#output_format_json_quote_64bit_integers) - controls quoting of 64-bit integers in JSON output format. Default value - `true`.
+- [output_format_json_quote_64bit_floats](/docs/en/operations/settings/settings.md/#output_format_json_quote_64bit_floats) - controls quoting of 64-bit floats in JSON output format. Default value - `false`.
+- [output_format_json_quote_denormals](/docs/en/operations/settings/settings.md/#output_format_json_quote_denormals) - enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format. Default value - `false`.
+- [output_format_json_quote_decimals](/docs/en/operations/settings/settings.md/#output_format_json_quote_decimals) - controls quoting of decimals in JSON output format. Default value - `false`.
+- [output_format_json_escape_forward_slashes](/docs/en/operations/settings/settings.md/#output_format_json_escape_forward_slashes) - controls escaping forward slashes for string outputs in JSON output format. Default value - `true`.
+- [output_format_json_named_tuples_as_objects](/docs/en/operations/settings/settings.md/#output_format_json_named_tuples_as_objects) - serialize named tuple columns as JSON objects. Default value - `false`.
+- [output_format_json_array_of_rows](/docs/en/operations/settings/settings.md/#output_format_json_array_of_rows) - output a JSON array of all rows in JSONEachRow(Compact) format. Default value - `false`.
+- [output_format_json_validate_utf8](/docs/en/operations/settings/settings.md/#output_format_json_validate_utf8) - enables validation of UTF-8 sequences in JSON output formats (note that it doesn't impact formats JSON/JSONCompact/JSONColumnsWithMetadata, they always validate utf8). Default value - `false`.
## BSONEachRow {#bsoneachrow}
@@ -1221,49 +1221,49 @@ For output it uses the following correspondence between ClickHouse types and BSO
| ClickHouse type | BSON Type |
|-----------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------|
-| [Bool](../sql-reference/data-types/boolean.md) | `\x08` boolean |
-| [Int8/UInt8](../sql-reference/data-types/int-uint.md) | `\x10` int32 |
-| [Int16UInt16](../sql-reference/data-types/int-uint.md) | `\x10` int32 |
-| [Int32](../sql-reference/data-types/int-uint.md) | `\x10` int32 |
-| [UInt32](../sql-reference/data-types/int-uint.md) | `\x12` int64 |
-| [Int64/UInt64](../sql-reference/data-types/int-uint.md) | `\x12` int64 |
-| [Float32/Float64](../sql-reference/data-types/float.md) | `\x01` double |
-| [Date](../sql-reference/data-types/date.md)/[Date32](../sql-reference/data-types/date32.md) | `\x10` int32 |
-| [DateTime](../sql-reference/data-types/datetime.md) | `\x12` int64 |
-| [DateTime64](../sql-reference/data-types/datetime64.md) | `\x09` datetime |
-| [Decimal32](../sql-reference/data-types/decimal.md) | `\x10` int32 |
-| [Decimal64](../sql-reference/data-types/decimal.md) | `\x12` int64 |
-| [Decimal128](../sql-reference/data-types/decimal.md) | `\x05` binary, `\x00` binary subtype, size = 16 |
-| [Decimal256](../sql-reference/data-types/decimal.md) | `\x05` binary, `\x00` binary subtype, size = 32 |
-| [Int128/UInt128](../sql-reference/data-types/int-uint.md) | `\x05` binary, `\x00` binary subtype, size = 16 |
-| [Int256/UInt256](../sql-reference/data-types/int-uint.md) | `\x05` binary, `\x00` binary subtype, size = 32 |
-| [String](../sql-reference/data-types/string.md)/[FixedString](../sql-reference/data-types/fixedstring.md) | `\x05` binary, `\x00` binary subtype or \x02 string if setting output_format_bson_string_as_string is enabled |
-| [UUID](../sql-reference/data-types/uuid.md) | `\x05` binary, `\x04` uuid subtype, size = 16 |
-| [Array](../sql-reference/data-types/array.md) | `\x04` array |
-| [Tuple](../sql-reference/data-types/tuple.md) | `\x04` array |
-| [Named Tuple](../sql-reference/data-types/tuple.md) | `\x03` document |
-| [Map](../sql-reference/data-types/map.md) (with String keys) | `\x03` document |
+| [Bool](/docs/en/sql-reference/data-types/boolean.md) | `\x08` boolean |
+| [Int8/UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `\x10` int32 |
+| [Int16UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `\x10` int32 |
+| [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `\x10` int32 |
+| [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `\x12` int64 |
+| [Int64/UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `\x12` int64 |
+| [Float32/Float64](/docs/en/sql-reference/data-types/float.md) | `\x01` double |
+| [Date](/docs/en/sql-reference/data-types/date.md)/[Date32](/docs/en/sql-reference/data-types/date32.md) | `\x10` int32 |
+| [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `\x12` int64 |
+| [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `\x09` datetime |
+| [Decimal32](/docs/en/sql-reference/data-types/decimal.md) | `\x10` int32 |
+| [Decimal64](/docs/en/sql-reference/data-types/decimal.md) | `\x12` int64 |
+| [Decimal128](/docs/en/sql-reference/data-types/decimal.md) | `\x05` binary, `\x00` binary subtype, size = 16 |
+| [Decimal256](/docs/en/sql-reference/data-types/decimal.md) | `\x05` binary, `\x00` binary subtype, size = 32 |
+| [Int128/UInt128](/docs/en/sql-reference/data-types/int-uint.md) | `\x05` binary, `\x00` binary subtype, size = 16 |
+| [Int256/UInt256](/docs/en/sql-reference/data-types/int-uint.md) | `\x05` binary, `\x00` binary subtype, size = 32 |
+| [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `\x05` binary, `\x00` binary subtype or \x02 string if setting output_format_bson_string_as_string is enabled |
+| [UUID](/docs/en/sql-reference/data-types/uuid.md) | `\x05` binary, `\x04` uuid subtype, size = 16 |
+| [Array](/docs/en/sql-reference/data-types/array.md) | `\x04` array |
+| [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `\x04` array |
+| [Named Tuple](/docs/en/sql-reference/data-types/tuple.md) | `\x03` document |
+| [Map](/docs/en/sql-reference/data-types/map.md) (with String keys) | `\x03` document |
For input it uses the following correspondence between BSON types and ClickHouse types:
| BSON Type | ClickHouse Type |
|------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| `\x01` double | [Float32/Float64](../sql-reference/data-types/float.md) |
-| `\x02` string | [String](../sql-reference/data-types/string.md)/[FixedString](../sql-reference/data-types/fixedstring.md) |
-| `\x03` document | [Map](../sql-reference/data-types/map.md)/[Named Tuple](../sql-reference/data-types/tuple.md) |
-| `\x04` array | [Array](../sql-reference/data-types/array.md)/[Tuple](../sql-reference/data-types/tuple.md) |
-| `\x05` binary, `\x00` binary subtype | [String](../sql-reference/data-types/string.md)/[FixedString](../sql-reference/data-types/fixedstring.md) |
-| `\x05` binary, `\x02` old binary subtype | [String](../sql-reference/data-types/string.md)/[FixedString](../sql-reference/data-types/fixedstring.md) |
-| `\x05` binary, `\x03` old uuid subtype | [UUID](../sql-reference/data-types/uuid.md) |
-| `\x05` binary, `\x04` uuid subtype | [UUID](../sql-reference/data-types/uuid.md) |
-| `\x07` ObjectId | [String](../sql-reference/data-types/string.md)/[FixedString](../sql-reference/data-types/fixedstring.md) |
-| `\x08` boolean | [Bool](../sql-reference/data-types/boolean.md) |
-| `\x09` datetime | [DateTime64](../sql-reference/data-types/datetime64.md) |
-| `\x0A` null value | [NULL](../sql-reference/data-types/nullable.md) |
-| `\x0D` JavaScript code | [String](../sql-reference/data-types/string.md)/[FixedString](../sql-reference/data-types/fixedstring.md) |
-| `\x0E` symbol | [String](../sql-reference/data-types/string.md)/[FixedString](../sql-reference/data-types/fixedstring.md) |
-| `\x10` int32 | [Int32/UInt32](../sql-reference/data-types/int-uint.md)/[Decimal32](../sql-reference/data-types/decimal.md) |
-| `\x12` int64 | [Int64/UInt64](../sql-reference/data-types/int-uint.md)/[Decimal64](../sql-reference/data-types/decimal.md)/[DateTime64](../sql-reference/data-types/datetime64.md) |
+| `\x01` double | [Float32/Float64](/docs/en/sql-reference/data-types/float.md) |
+| `\x02` string | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) |
+| `\x03` document | [Map](/docs/en/sql-reference/data-types/map.md)/[Named Tuple](/docs/en/sql-reference/data-types/tuple.md) |
+| `\x04` array | [Array](/docs/en/sql-reference/data-types/array.md)/[Tuple](/docs/en/sql-reference/data-types/tuple.md) |
+| `\x05` binary, `\x00` binary subtype | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) |
+| `\x05` binary, `\x02` old binary subtype | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) |
+| `\x05` binary, `\x03` old uuid subtype | [UUID](/docs/en/sql-reference/data-types/uuid.md) |
+| `\x05` binary, `\x04` uuid subtype | [UUID](/docs/en/sql-reference/data-types/uuid.md) |
+| `\x07` ObjectId | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) |
+| `\x08` boolean | [Bool](/docs/en/sql-reference/data-types/boolean.md) |
+| `\x09` datetime | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) |
+| `\x0A` null value | [NULL](/docs/en/sql-reference/data-types/nullable.md) |
+| `\x0D` JavaScript code | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) |
+| `\x0E` symbol | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) |
+| `\x10` int32 | [Int32/UInt32](/docs/en/sql-reference/data-types/int-uint.md)/[Decimal32](/docs/en/sql-reference/data-types/decimal.md) |
+| `\x12` int64 | [Int64/UInt64](/docs/en/sql-reference/data-types/int-uint.md)/[Decimal64](/docs/en/sql-reference/data-types/decimal.md)/[DateTime64](/docs/en/sql-reference/data-types/datetime64.md) |
Other BSON types are not supported. Also, it performs conversion between different integer types (for example, you can insert BSON int32 value into ClickHouse UInt8).
Big integers and decimals (Int128/UInt128/Int256/UInt256/Decimal128/Decimal256) can be parsed from BSON Binary value with `\x00` binary subtype. In this case this format will validate that the size of binary data equals the size of expected value.
@@ -1272,8 +1272,8 @@ Note: this format don't work properly on Big-Endian platforms.
### BSON format settings {#bson-format-settings}
-- [output_format_bson_string_as_string](../operations/settings/settings.md#output_format_bson_string_as_string) - use BSON String type instead of Binary for String columns. Default value - `false`.
-- [input_format_bson_skip_fields_with_unsupported_types_in_schema_inference](../operations/settings/settings.md#input_format_bson_skip_fields_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for format BSONEachRow. Default value - `false`.
+- [output_format_bson_string_as_string](/docs/en/operations/settings/settings.md/#output_format_bson_string_as_string) - use BSON String type instead of Binary for String columns. Default value - `false`.
+- [input_format_bson_skip_fields_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings.md/#input_format_bson_skip_fields_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for format BSONEachRow. Default value - `false`.
## Native {#native}
@@ -1292,7 +1292,7 @@ Outputs data as Unicode-art tables, also using ANSI-escape sequences for setting
A full grid of the table is drawn, and each row occupies two lines in the terminal.
Each result block is output as a separate table. This is necessary so that blocks can be output without buffering results (buffering would be necessary in order to pre-calculate the visible width of all the values).
-[NULL](../sql-reference/syntax.md) is output as `ᴺᵁᴸᴸ`.
+[NULL](/docs/en/sql-reference/syntax.md) is output as `ᴺᵁᴸᴸ`.
Example (shown for the [PrettyCompact](#prettycompact) format):
@@ -1406,12 +1406,12 @@ Differs from [PrettySpaceNoEscapes](#prettyspacenoescapes) in that up to 10,000
## Pretty formats settings {#pretty-formats-settings}
-- [output_format_pretty_max_rows](../operations/settings/settings.md#output_format_pretty_max_rows) - rows limit for Pretty formats. Default value - `10000`.
-- [output_format_pretty_max_column_pad_width](../operations/settings/settings.md#output_format_pretty_max_column_pad_width) - maximum width to pad all values in a column in Pretty formats. Default value - `250`.
-- [output_format_pretty_max_value_width](../operations/settings/settings.md#output_format_pretty_max_value_width) - Maximum width of value to display in Pretty formats. If greater - it will be cut. Default value - `10000`.
-- [output_format_pretty_color](../operations/settings/settings.md#output_format_pretty_color) - use ANSI escape sequences to paint colors in Pretty formats. Default value - `true`.
-- [output_format_pretty_grid_charset](../operations/settings/settings.md#output_format_pretty_grid_charset) - Charset for printing grid borders. Available charsets: ASCII, UTF-8. Default value - `UTF-8`.
-- [output_format_pretty_row_numbers](../operations/settings/settings.md#output_format_pretty_row_numbers) - Add row numbers before each row for pretty output format. Default value - `false`.
+- [output_format_pretty_max_rows](/docs/en/operations/settings/settings.md/#output_format_pretty_max_rows) - rows limit for Pretty formats. Default value - `10000`.
+- [output_format_pretty_max_column_pad_width](/docs/en/operations/settings/settings.md/#output_format_pretty_max_column_pad_width) - maximum width to pad all values in a column in Pretty formats. Default value - `250`.
+- [output_format_pretty_max_value_width](/docs/en/operations/settings/settings.md/#output_format_pretty_max_value_width) - Maximum width of value to display in Pretty formats. If greater - it will be cut. Default value - `10000`.
+- [output_format_pretty_color](/docs/en/operations/settings/settings.md/#output_format_pretty_color) - use ANSI escape sequences to paint colors in Pretty formats. Default value - `true`.
+- [output_format_pretty_grid_charset](/docs/en/operations/settings/settings.md/#output_format_pretty_grid_charset) - Charset for printing grid borders. Available charsets: ASCII, UTF-8. Default value - `UTF-8`.
+- [output_format_pretty_row_numbers](/docs/en/operations/settings/settings.md/#output_format_pretty_row_numbers) - Add row numbers before each row for pretty output format. Default value - `false`.
## RowBinary {#rowbinary}
@@ -1426,7 +1426,7 @@ FixedString is represented simply as a sequence of bytes.
Array is represented as a varint length (unsigned [LEB128](https://en.wikipedia.org/wiki/LEB128)), followed by successive elements of the array.
-For [NULL](../sql-reference/syntax.md#null-literal) support, an additional byte containing 1 or 0 is added before each [Nullable](../sql-reference/data-types/nullable.md) value. If 1, then the value is `NULL` and this byte is interpreted as a separate value. If 0, the value after the byte is not `NULL`.
+For [NULL](/docs/en/sql-reference/syntax.md/#null-literal) support, an additional byte containing 1 or 0 is added before each [Nullable](/docs/en/sql-reference/data-types/nullable.md) value. If 1, then the value is `NULL` and this byte is interpreted as a separate value. If 0, the value after the byte is not `NULL`.
## RowBinaryWithNames {#rowbinarywithnames}
@@ -1436,8 +1436,8 @@ Similar to [RowBinary](#rowbinary), but with added header:
- N `String`s specifying column names
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
:::
@@ -1450,20 +1450,20 @@ Similar to [RowBinary](#rowbinary), but with added header:
- N `String`s specifying column types
:::warning
-If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1,
-the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings.md/#input_format_with_names_use_header) is set to 1,
+the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
Otherwise, the first row will be skipped.
-If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1,
+If setting [input_format_with_types_use_header](/docs/en/operations/settings/settings.md/#input_format_with_types_use_header) is set to 1,
the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped.
:::
## RowBinary format settings {#row-binary-format-settings}
-- [format_binary_max_string_size](../operations/settings/settings.md#format_binary_max_string_size) - The maximum allowed size for String in RowBinary format. Default value - `1GiB`.
+- [format_binary_max_string_size](/docs/en/operations/settings/settings.md/#format_binary_max_string_size) - The maximum allowed size for String in RowBinary format. Default value - `1GiB`.
## Values {#data-format-values}
-Prints every row in brackets. Rows are separated by commas. There is no comma after the last row. The values inside the brackets are also comma-separated. Numbers are output in a decimal format without quotes. Arrays are output in square brackets. Strings, dates, and dates with times are output in quotes. Escaping rules and parsing are similar to the [TabSeparated](#tabseparated) format. During formatting, extra spaces aren’t inserted, but during parsing, they are allowed and skipped (except for spaces inside array values, which are not allowed). [NULL](../sql-reference/syntax.md) is represented as `NULL`.
+Prints every row in brackets. Rows are separated by commas. There is no comma after the last row. The values inside the brackets are also comma-separated. Numbers are output in a decimal format without quotes. Arrays are output in square brackets. Strings, dates, and dates with times are output in quotes. Escaping rules and parsing are similar to the [TabSeparated](#tabseparated) format. During formatting, extra spaces aren’t inserted, but during parsing, they are allowed and skipped (except for spaces inside array values, which are not allowed). [NULL](/docs/en/sql-reference/syntax.md) is represented as `NULL`.
The minimum set of characters that you need to escape when passing data in Values format: single quotes and backslashes.
@@ -1471,16 +1471,16 @@ This is the format that is used in `INSERT INTO t VALUES ...`, but you can also
## Values format settings {#values-format-settings}
-- [input_format_values_interpret_expressions](../operations/settings/settings.md#input_format_values_interpret_expressions) - if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression. Default value - `true`.
-- [input_format_values_deduce_templates_of_expressions](../operations/settings/settings.md#input_format_values_deduce_templates_of_expressions) -if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows. Default value - `true`.
-- [input_format_values_accurate_types_of_literals](../operations/settings/settings.md#input_format_values_accurate_types_of_literals) - when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues. Default value - `true`.
+- [input_format_values_interpret_expressions](/docs/en/operations/settings/settings.md/#input_format_values_interpret_expressions) - if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression. Default value - `true`.
+- [input_format_values_deduce_templates_of_expressions](/docs/en/operations/settings/settings.md/#input_format_values_deduce_templates_of_expressions) -if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows. Default value - `true`.
+- [input_format_values_accurate_types_of_literals](/docs/en/operations/settings/settings.md/#input_format_values_accurate_types_of_literals) - when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues. Default value - `true`.
## Vertical {#vertical}
Prints each value on a separate line with the column name specified. This format is convenient for printing just one or a few rows if each row consists of a large number of columns.
-[NULL](../sql-reference/syntax.md) is output as `ᴺᵁᴸᴸ`.
+[NULL](/docs/en/sql-reference/syntax.md) is output as `ᴺᵁᴸᴸ`.
Example:
@@ -1593,27 +1593,27 @@ See also [Format Schema](#formatschema).
### Data Types Matching {#data_types-matching-capnproto}
-The table below shows supported data types and how they match ClickHouse [data types](../sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries.
+The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries.
| CapnProto data type (`INSERT`) | ClickHouse data type | CapnProto data type (`SELECT`) |
|--------------------------------|-----------------------------------------------------------|--------------------------------|
-| `UINT8`, `BOOL` | [UInt8](../sql-reference/data-types/int-uint.md) | `UINT8` |
-| `INT8` | [Int8](../sql-reference/data-types/int-uint.md) | `INT8` |
-| `UINT16` | [UInt16](../sql-reference/data-types/int-uint.md), [Date](../sql-reference/data-types/date.md) | `UINT16` |
-| `INT16` | [Int16](../sql-reference/data-types/int-uint.md) | `INT16` |
-| `UINT32` | [UInt32](../sql-reference/data-types/int-uint.md), [DateTime](../sql-reference/data-types/datetime.md) | `UINT32` |
-| `INT32` | [Int32](../sql-reference/data-types/int-uint.md) | `INT32` |
-| `UINT64` | [UInt64](../sql-reference/data-types/int-uint.md) | `UINT64` |
-| `INT64` | [Int64](../sql-reference/data-types/int-uint.md), [DateTime64](../sql-reference/data-types/datetime.md) | `INT64` |
-| `FLOAT32` | [Float32](../sql-reference/data-types/float.md) | `FLOAT32` |
-| `FLOAT64` | [Float64](../sql-reference/data-types/float.md) | `FLOAT64` |
-| `TEXT, DATA` | [String](../sql-reference/data-types/string.md), [FixedString](../sql-reference/data-types/fixedstring.md) | `TEXT, DATA` |
-| `union(T, Void), union(Void, T)` | [Nullable(T)](../sql-reference/data-types/date.md) | `union(T, Void), union(Void, T)` |
-| `ENUM` | [Enum(8\|16)](../sql-reference/data-types/enum.md) | `ENUM` |
-| `LIST` | [Array](../sql-reference/data-types/array.md) | `LIST` |
-| `STRUCT` | [Tuple](../sql-reference/data-types/tuple.md) | `STRUCT` |
+| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` |
+| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` |
+| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) | `UINT16` |
+| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` |
+| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md), [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` |
+| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` |
+| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` |
+| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md), [DateTime64](/docs/en/sql-reference/data-types/datetime.md) | `INT64` |
+| `FLOAT32` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT32` |
+| `FLOAT64` | [Float64](/docs/en/sql-reference/data-types/float.md) | `FLOAT64` |
+| `TEXT, DATA` | [String](/docs/en/sql-reference/data-types/string.md), [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `TEXT, DATA` |
+| `union(T, Void), union(Void, T)` | [Nullable(T)](/docs/en/sql-reference/data-types/date.md) | `union(T, Void), union(Void, T)` |
+| `ENUM` | [Enum(8\|16)](/docs/en/sql-reference/data-types/enum.md) | `ENUM` |
+| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` |
+| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` |
-For working with `Enum` in CapnProto format use the [format_capn_proto_enum_comparising_mode](../operations/settings/settings.md#format_capn_proto_enum_comparising_mode) setting.
+For working with `Enum` in CapnProto format use the [format_capn_proto_enum_comparising_mode](/docs/en/operations/settings/settings.md/#format_capn_proto_enum_comparising_mode) setting.
Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` type also can be nested.
@@ -1644,10 +1644,10 @@ $ clickhouse-client --query = "SELECT * FROM test.hits FORMAT CapnProto SETTINGS
Expose metrics in [Prometheus text-based exposition format](https://prometheus.io/docs/instrumenting/exposition_formats/#text-based-format).
The output table should have a proper structure.
-Columns `name` ([String](../sql-reference/data-types/string.md)) and `value` (number) are required.
-Rows may optionally contain `help` ([String](../sql-reference/data-types/string.md)) and `timestamp` (number).
-Column `type` ([String](../sql-reference/data-types/string.md)) is either `counter`, `gauge`, `histogram`, `summary`, `untyped` or empty.
-Each metric value may also have some `labels` ([Map(String, String)](../sql-reference/data-types/map.md)).
+Columns `name` ([String](/docs/en/sql-reference/data-types/string.md)) and `value` (number) are required.
+Rows may optionally contain `help` ([String](/docs/en/sql-reference/data-types/string.md)) and `timestamp` (number).
+Column `type` ([String](/docs/en/sql-reference/data-types/string.md)) is either `counter`, `gauge`, `histogram`, `summary`, `untyped` or empty.
+Each metric value may also have some `labels` ([Map(String, String)](/docs/en/sql-reference/data-types/map.md)).
Several consequent rows may refer to the one metric with different labels. The table should be sorted by metric name (e.g., with `ORDER BY name`).
There's special requirements for labels for `histogram` and `summary`, see [Prometheus doc](https://prometheus.io/docs/instrumenting/exposition_formats/#histograms-and-summaries) for the details. Special rules applied to row with labels `{'count':''}` and `{'sum':''}`, they'll be converted to `_count` and `_sum` respectively.
@@ -1759,7 +1759,7 @@ message MessageType {
```
ClickHouse tries to find a column named `x.y.z` (or `x_y_z` or `X.y_Z` and so on).
-Nested messages are suitable to input or output a [nested data structures](../sql-reference/data-types/nested-data-structures/nested.md).
+Nested messages are suitable to input or output a [nested data structures](/docs/en/sql-reference/data-types/nested-data-structures/nested.md).
Default values defined in a protobuf schema like this
@@ -1771,7 +1771,7 @@ message MessageType {
}
```
-are not applied; the [table defaults](../sql-reference/statements/create/table.md#create-default-values) are used instead of them.
+are not applied; the [table defaults](/docs/en/sql-reference/statements/create/table.md/#create-default-values) are used instead of them.
ClickHouse inputs and outputs protobuf messages in the `length-delimited` format.
It means before every message should be written its length as a [varint](https://developers.google.com/protocol-buffers/docs/encoding#varints).
@@ -1789,25 +1789,25 @@ ClickHouse Avro format supports reading and writing [Avro data files](https://av
### Data Types Matching {#data_types-matching}
-The table below shows supported data types and how they match ClickHouse [data types](../sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries.
+The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries.
-| Avro data type `INSERT` | ClickHouse data type | Avro data type `SELECT` |
-|---------------------------------------------|-----------------------------------------------------------------------------------------------------------------------|------------------------------|
-| `boolean`, `int`, `long`, `float`, `double` | [Int(8\|16\|32)](../sql-reference/data-types/int-uint.md), [UInt(8\|16\|32)](../sql-reference/data-types/int-uint.md) | `int` |
-| `boolean`, `int`, `long`, `float`, `double` | [Int64](../sql-reference/data-types/int-uint.md), [UInt64](../sql-reference/data-types/int-uint.md) | `long` |
-| `boolean`, `int`, `long`, `float`, `double` | [Float32](../sql-reference/data-types/float.md) | `float` |
-| `boolean`, `int`, `long`, `float`, `double` | [Float64](../sql-reference/data-types/float.md) | `double` |
-| `bytes`, `string`, `fixed`, `enum` | [String](../sql-reference/data-types/string.md) | `bytes` or `string` \* |
-| `bytes`, `string`, `fixed` | [FixedString(N)](../sql-reference/data-types/fixedstring.md) | `fixed(N)` |
-| `enum` | [Enum(8\|16)](../sql-reference/data-types/enum.md) | `enum` |
-| `array(T)` | [Array(T)](../sql-reference/data-types/array.md) | `array(T)` |
-| `union(null, T)`, `union(T, null)` | [Nullable(T)](../sql-reference/data-types/date.md) | `union(null, T)` |
-| `null` | [Nullable(Nothing)](../sql-reference/data-types/special-data-types/nothing.md) | `null` |
-| `int (date)` \** | [Date](../sql-reference/data-types/date.md) | `int (date)` \** |
-| `long (timestamp-millis)` \** | [DateTime64(3)](../sql-reference/data-types/datetime.md) | `long (timestamp-millis)` \* |
-| `long (timestamp-micros)` \** | [DateTime64(6)](../sql-reference/data-types/datetime.md) | `long (timestamp-micros)` \* |
+| Avro data type `INSERT` | ClickHouse data type | Avro data type `SELECT` |
+|---------------------------------------------|----------------------------------------------------------------------------------------------------|------------------------------|
+| `boolean`, `int`, `long`, `float`, `double` | [Int(8\|16\|32)](/docs/en/sql-reference/data-types/int-uint.md), [UInt(8\|16\|32)](/docs/en/sql-reference/data-types/int-uint.md) | `int` |
+| `boolean`, `int`, `long`, `float`, `double` | [Int64](/docs/en/sql-reference/data-types/int-uint.md), [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `long` |
+| `boolean`, `int`, `long`, `float`, `double` | [Float32](/docs/en/sql-reference/data-types/float.md) | `float` |
+| `boolean`, `int`, `long`, `float`, `double` | [Float64](/docs/en/sql-reference/data-types/float.md) | `double` |
+| `bytes`, `string`, `fixed`, `enum` | [String](/docs/en/sql-reference/data-types/string.md) | `bytes` or `string` \* |
+| `bytes`, `string`, `fixed` | [FixedString(N)](/docs/en/sql-reference/data-types/fixedstring.md) | `fixed(N)` |
+| `enum` | [Enum(8\|16)](/docs/en/sql-reference/data-types/enum.md) | `enum` |
+| `array(T)` | [Array(T)](/docs/en/sql-reference/data-types/array.md) | `array(T)` |
+| `union(null, T)`, `union(T, null)` | [Nullable(T)](/docs/en/sql-reference/data-types/date.md) | `union(null, T)` |
+| `null` | [Nullable(Nothing)](/docs/en/sql-reference/data-types/special-data-types/nothing.md) | `null` |
+| `int (date)` \** | [Date](/docs/en/sql-reference/data-types/date.md) | `int (date)` \** |
+| `long (timestamp-millis)` \** | [DateTime64(3)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-millis)` \* |
+| `long (timestamp-micros)` \** | [DateTime64(6)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-micros)` \* |
-\* `bytes` is default, controlled by [output_format_avro_string_column_pattern](../operations/settings/settings.md#output_format_avro_string_column_pattern)
+\* `bytes` is default, controlled by [output_format_avro_string_column_pattern](/docs/en/operations/settings/settings.md/#output_format_avro_string_column_pattern)
\** [Avro logical types](https://avro.apache.org/docs/current/spec.html#Logical+Types)
Unsupported Avro data types: `record` (non-root), `map`
@@ -1827,9 +1827,9 @@ The root schema of input Avro file must be of `record` type.
To find the correspondence between table columns and fields of Avro schema ClickHouse compares their names. This comparison is case-sensitive.
Unused fields are skipped.
-Data types of ClickHouse table columns can differ from the corresponding fields of the Avro data inserted. When inserting data, ClickHouse interprets data types according to the table above and then [casts](../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) the data to corresponding column type.
+Data types of ClickHouse table columns can differ from the corresponding fields of the Avro data inserted. When inserting data, ClickHouse interprets data types according to the table above and then [casts](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_function-cast) the data to corresponding column type.
-While importing data, when field is not found in schema and setting [input_format_avro_allow_missing_fields](../operations/settings/settings.md#input_format_avro_allow_missing_fields) is enabled, default value will be used instead of error.
+While importing data, when field is not found in schema and setting [input_format_avro_allow_missing_fields](/docs/en/operations/settings/settings.md/#input_format_avro_allow_missing_fields) is enabled, default value will be used instead of error.
### Selecting Data {#selecting-data-1}
@@ -1844,7 +1844,7 @@ Column names must:
- start with `[A-Za-z_]`
- subsequently contain only `[A-Za-z0-9_]`
-Output Avro file compression and sync interval can be configured with [output_format_avro_codec](../operations/settings/settings.md#output_format_avro_codec) and [output_format_avro_sync_interval](../operations/settings/settings.md#output_format_avro_sync_interval) respectively.
+Output Avro file compression and sync interval can be configured with [output_format_avro_codec](/docs/en/operations/settings/settings.md/#output_format_avro_codec) and [output_format_avro_sync_interval](/docs/en/operations/settings/settings.md/#output_format_avro_sync_interval) respectively.
## AvroConfluent {#data-format-avro-confluent}
@@ -1854,7 +1854,7 @@ Each Avro message embeds a schema id that can be resolved to the actual schema w
Schemas are cached once resolved.
-Schema Registry URL is configured with [format_avro_schema_registry_url](../operations/settings/settings.md#format_avro_schema_registry_url).
+Schema Registry URL is configured with [format_avro_schema_registry_url](/docs/en/operations/settings/settings.md/#format_avro_schema_registry_url).
### Data Types Matching {#data_types-matching-1}
@@ -1862,7 +1862,7 @@ Same as [Avro](#data-format-avro).
### Usage {#usage}
-To quickly verify schema resolution you can use [kafkacat](https://github.com/edenhill/kafkacat) with [clickhouse-local](../operations/utilities/clickhouse-local.md):
+To quickly verify schema resolution you can use [kafkacat](https://github.com/edenhill/kafkacat) with [clickhouse-local](/docs/en/operations/utilities/clickhouse-local.md):
``` bash
$ kafkacat -b kafka-broker -C -t topic1 -o beginning -f '%s' -c 3 | clickhouse-local --input-format AvroConfluent --format_avro_schema_registry_url 'http://schema-registry' -S "field1 Int64, field2 String" -q 'select * from table'
@@ -1871,7 +1871,7 @@ $ kafkacat -b kafka-broker -C -t topic1 -o beginning -f '%s' -c 3 | clickhouse-
3 c
```
-To use `AvroConfluent` with [Kafka](../engines/table-engines/integrations/kafka.md):
+To use `AvroConfluent` with [Kafka](/docs/en/engines/table-engines/integrations/kafka.md):
``` sql
CREATE TABLE topic1_stream
@@ -1903,36 +1903,36 @@ Setting `format_avro_schema_registry_url` needs to be configured in `users.xml`
### Data Types Matching {#data-types-matching-parquet}
-The table below shows supported data types and how they match ClickHouse [data types](../sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries.
+The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries.
| Parquet data type (`INSERT`) | ClickHouse data type | Parquet data type (`SELECT`) |
-|------------------------------|-----------------------------------------------------------|------------------------------|
-| `UINT8`, `BOOL` | [UInt8](../sql-reference/data-types/int-uint.md) | `UINT8` |
-| `INT8` | [Int8](../sql-reference/data-types/int-uint.md) | `INT8` |
-| `UINT16` | [UInt16](../sql-reference/data-types/int-uint.md) | `UINT16` |
-| `INT16` | [Int16](../sql-reference/data-types/int-uint.md) | `INT16` |
-| `UINT32` | [UInt32](../sql-reference/data-types/int-uint.md) | `UINT32` |
-| `INT32` | [Int32](../sql-reference/data-types/int-uint.md) | `INT32` |
-| `UINT64` | [UInt64](../sql-reference/data-types/int-uint.md) | `UINT64` |
-| `INT64` | [Int64](../sql-reference/data-types/int-uint.md) | `INT64` |
-| `FLOAT`, `HALF_FLOAT` | [Float32](../sql-reference/data-types/float.md) | `FLOAT` |
-| `DOUBLE` | [Float64](../sql-reference/data-types/float.md) | `DOUBLE` |
-| `DATE32` | [Date](../sql-reference/data-types/date.md) | `UINT16` |
-| `DATE64`, `TIMESTAMP` | [DateTime](../sql-reference/data-types/datetime.md) | `UINT32` |
-| `STRING`, `BINARY` | [String](../sql-reference/data-types/string.md) | `BINARY` |
-| — | [FixedString](../sql-reference/data-types/fixedstring.md) | `BINARY` |
-| `DECIMAL` | [Decimal](../sql-reference/data-types/decimal.md) | `DECIMAL` |
-| `LIST` | [Array](../sql-reference/data-types/array.md) | `LIST` |
-| `STRUCT` | [Tuple](../sql-reference/data-types/tuple.md) | `STRUCT` |
-| `MAP` | [Map](../sql-reference/data-types/map.md) | `MAP` |
+|------------------------------|-----------------------------------------------------------|----------------------------|
+| `BOOL` | [Bool](/docs/en/sql-reference/data-types/boolean.md) | `BOOL` |
+| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` |
+| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` |
+| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `UINT16` |
+| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` |
+| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `UINT32` |
+| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` |
+| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` |
+| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | `INT64` |
+| `FLOAT` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT` |
+| `DOUBLE` | [Float64](/docs/en/sql-reference/data-types/float.md) | `DOUBLE` |
+| `DATE` | [Date32](/docs/en/sql-reference/data-types/date.md) | `DATE` |
+| `TIME (ms)` | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` |
+| `TIMESTAMP`, `TIME (us, ns)` | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `TIMESTAMP` |
+| `STRING`, `BINARY` | [String](/docs/en/sql-reference/data-types/string.md) | `BINARY` |
+| — | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `BINARY` |
+| `DECIMAL` | [Decimal](/docs/en/sql-reference/data-types/decimal.md) | `DECIMAL` |
+| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` |
+| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` |
+| `MAP` | [Map](/docs/en/sql-reference/data-types/map.md) | `MAP` |
Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` and `Map` types also can be nested.
-ClickHouse supports configurable precision of `Decimal` type. The `INSERT` query treats the Parquet `DECIMAL` type as the ClickHouse `Decimal128` type.
+Unsupported Parquet data types: `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`.
-Unsupported Parquet data types: `TIME32`, `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`.
-
-Data types of ClickHouse table columns can differ from the corresponding fields of the Parquet data inserted. When inserting data, ClickHouse interprets data types according to the table above and then [cast](../sql-reference/functions/type-conversion-functions/#type_conversion_function-cast) the data to that data type which is set for the ClickHouse table column.
+Data types of ClickHouse table columns can differ from the corresponding fields of the Parquet data inserted. When inserting data, ClickHouse interprets data types according to the table above and then [cast](/docs/en/sql-reference/functions/type-conversion-functions/#type_conversion_function-cast) the data to that data type which is set for the ClickHouse table column.
### Inserting and Selecting Data {#inserting-and-selecting-data-parquet}
@@ -1948,16 +1948,16 @@ You can select data from a ClickHouse table and save them into some file in the
$ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_file.pq}
```
-To exchange data with Hadoop, you can use [HDFS table engine](../engines/table-engines/integrations/hdfs.md).
+To exchange data with Hadoop, you can use [HDFS table engine](/docs/en/engines/table-engines/integrations/hdfs.md).
### Parquet format settings {#parquet-format-settings}
-- [output_format_parquet_row_group_size](../operations/settings/settings.md#output_format_parquet_row_group_size) - row group size in rows while data output. Default value - `1000000`.
-- [output_format_parquet_string_as_string](../operations/settings/settings.md#output_format_parquet_string_as_string) - use Parquet String type instead of Binary for String columns. Default value - `false`.
-- [input_format_parquet_import_nested](../operations/settings/settings.md#input_format_parquet_import_nested) - allow inserting array of structs into [Nested](../sql-reference/data-types/nested-data-structures/nested.md) table in Parquet input format. Default value - `false`.
-- [input_format_parquet_case_insensitive_column_matching](../operations/settings/settings.md#input_format_parquet_case_insensitive_column_matching) - ignore case when matching Parquet columns with ClickHouse columns. Default value - `false`.
-- [input_format_parquet_allow_missing_columns](../operations/settings/settings.md#input_format_parquet_allow_missing_columns) - allow missing columns while reading Parquet data. Default value - `false`.
-- [input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference](../operations/settings/settings.md#input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Parquet format. Default value - `false`.
+- [output_format_parquet_row_group_size](/docs/en/operations/settings/settings.md/#output_format_parquet_row_group_size) - row group size in rows while data output. Default value - `1000000`.
+- [output_format_parquet_string_as_string](/docs/en/operations/settings/settings.md/#output_format_parquet_string_as_string) - use Parquet String type instead of Binary for String columns. Default value - `false`.
+- [input_format_parquet_import_nested](/docs/en/operations/settings/settings.md/#input_format_parquet_import_nested) - allow inserting array of structs into [Nested](/docs/en/sql-reference/data-types/nested-data-structures/nested.md) table in Parquet input format. Default value - `false`.
+- [input_format_parquet_case_insensitive_column_matching](/docs/en/operations/settings/settings.md/#input_format_parquet_case_insensitive_column_matching) - ignore case when matching Parquet columns with ClickHouse columns. Default value - `false`.
+- [input_format_parquet_allow_missing_columns](/docs/en/operations/settings/settings.md/#input_format_parquet_allow_missing_columns) - allow missing columns while reading Parquet data. Default value - `false`.
+- [input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings.md/#input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Parquet format. Default value - `false`.
## Arrow {#data-format-arrow}
@@ -1967,39 +1967,39 @@ To exchange data with Hadoop, you can use [HDFS table engine](../engines/table-e
### Data Types Matching {#data-types-matching-arrow}
-The table below shows supported data types and how they match ClickHouse [data types](../sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries.
+The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries.
-| Arrow data type (`INSERT`) | ClickHouse data type | Arrow data type (`SELECT`) |
-|----------------------------|-----------------------------------------------------|----------------------------|
-| `UINT8`, `BOOL` | [UInt8](../sql-reference/data-types/int-uint.md) | `UINT8` |
-| `INT8` | [Int8](../sql-reference/data-types/int-uint.md) | `INT8` |
-| `UINT16` | [UInt16](../sql-reference/data-types/int-uint.md) | `UINT16` |
-| `INT16` | [Int16](../sql-reference/data-types/int-uint.md) | `INT16` |
-| `UINT32` | [UInt32](../sql-reference/data-types/int-uint.md) | `UINT32` |
-| `INT32` | [Int32](../sql-reference/data-types/int-uint.md) | `INT32` |
-| `UINT64` | [UInt64](../sql-reference/data-types/int-uint.md) | `UINT64` |
-| `INT64` | [Int64](../sql-reference/data-types/int-uint.md) | `INT64` |
-| `FLOAT`, `HALF_FLOAT` | [Float32](../sql-reference/data-types/float.md) | `FLOAT32` |
-| `DOUBLE` | [Float64](../sql-reference/data-types/float.md) | `FLOAT64` |
-| `DATE32` | [Date](../sql-reference/data-types/date.md) | `UINT16` |
-| `DATE64`, `TIMESTAMP` | [DateTime](../sql-reference/data-types/datetime.md) | `UINT32` |
-| `STRING`, `BINARY` | [String](../sql-reference/data-types/string.md) | `BINARY` |
-| `STRING`, `BINARY` | [FixedString](../sql-reference/data-types/fixedstring.md) | `BINARY` |
-| `DECIMAL` | [Decimal](../sql-reference/data-types/decimal.md) | `DECIMAL` |
-| `DECIMAL256` | [Decimal256](../sql-reference/data-types/decimal.md)| `DECIMAL256` |
-| `LIST` | [Array](../sql-reference/data-types/array.md) | `LIST` |
-| `STRUCT` | [Tuple](../sql-reference/data-types/tuple.md) | `STRUCT` |
-| `MAP` | [Map](../sql-reference/data-types/map.md) | `MAP` |
+| Arrow data type (`INSERT`) | ClickHouse data type | Arrow data type (`SELECT`) |
+|---------------------------------|-----------------------------------------------------------|----------------------------|
+| `BOOL` | [Bool](/docs/en/sql-reference/data-types/boolean.md) | `BOOL` |
+| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` |
+| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` |
+| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `UINT16` |
+| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` |
+| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `UINT32` |
+| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` |
+| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` |
+| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | `INT64` |
+| `FLOAT`, `HALF_FLOAT` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT32` |
+| `DOUBLE` | [Float64](/docs/en/sql-reference/data-types/float.md) | `FLOAT64` |
+| `DATE32` | [Date32](/docs/en/sql-reference/data-types/date32.md) | `UINT16` |
+| `DATE64` | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` |
+| `TIMESTAMP`, `TIME32`, `TIME64` | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `UINT32` |
+| `STRING`, `BINARY` | [String](/docs/en/sql-reference/data-types/string.md) | `BINARY` |
+| `STRING`, `BINARY` | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `BINARY` |
+| `DECIMAL` | [Decimal](/docs/en/sql-reference/data-types/decimal.md) | `DECIMAL` |
+| `DECIMAL256` | [Decimal256](/docs/en/sql-reference/data-types/decimal.md) | `DECIMAL256` |
+| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` |
+| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` |
+| `MAP` | [Map](/docs/en/sql-reference/data-types/map.md) | `MAP` |
Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` and `Map` types also can be nested.
-The `DICTIONARY` type is supported for `INSERT` queries, and for `SELECT` queries there is an [output_format_arrow_low_cardinality_as_dictionary](../operations/settings/settings.md#output-format-arrow-low-cardinality-as-dictionary) setting that allows to output [LowCardinality](../sql-reference/data-types/lowcardinality.md) type as a `DICTIONARY` type.
+The `DICTIONARY` type is supported for `INSERT` queries, and for `SELECT` queries there is an [output_format_arrow_low_cardinality_as_dictionary](/docs/en/operations/settings/settings.md/#output-format-arrow-low-cardinality-as-dictionary) setting that allows to output [LowCardinality](/docs/en/sql-reference/data-types/lowcardinality.md) type as a `DICTIONARY` type.
-ClickHouse supports configurable precision of the `Decimal` type. The `INSERT` query treats the Arrow `DECIMAL` type as the ClickHouse `Decimal128` type.
+Unsupported Arrow data types: `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`.
-Unsupported Arrow data types: `TIME32`, `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`.
-
-The data types of ClickHouse table columns do not have to match the corresponding Arrow data fields. When inserting data, ClickHouse interprets data types according to the table above and then [casts](../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) the data to the data type set for the ClickHouse table column.
+The data types of ClickHouse table columns do not have to match the corresponding Arrow data fields. When inserting data, ClickHouse interprets data types according to the table above and then [casts](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_function-cast) the data to the data type set for the ClickHouse table column.
### Inserting Data {#inserting-data-arrow}
@@ -2019,12 +2019,12 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Arrow" > {filenam
### Arrow format settings {#parquet-format-settings}
-- [output_format_arrow_low_cardinality_as_dictionary](../operations/settings/settings.md#output_format_arrow_low_cardinality_as_dictionary) - enable output ClickHouse LowCardinality type as Dictionary Arrow type. Default value - `false`.
-- [output_format_arrow_string_as_string](../operations/settings/settings.md#output_format_arrow_string_as_string) - use Arrow String type instead of Binary for String columns. Default value - `false`.
-- [input_format_arrow_import_nested](../operations/settings/settings.md#input_format_arrow_import_nested) - allow inserting array of structs into Nested table in Arrow input format. Default value - `false`.
-- [input_format_arrow_case_insensitive_column_matching](../operations/settings/settings.md#input_format_arrow_case_insensitive_column_matching) - ignore case when matching Arrow columns with ClickHouse columns. Default value - `false`.
-- [input_format_arrow_allow_missing_columns](../operations/settings/settings.md#input_format_arrow_allow_missing_columns) - allow missing columns while reading Arrow data. Default value - `false`.
-- [input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference](../operations/settings/settings.md#input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Arrow format. Default value - `false`.
+- [output_format_arrow_low_cardinality_as_dictionary](/docs/en/operations/settings/settings.md/#output_format_arrow_low_cardinality_as_dictionary) - enable output ClickHouse LowCardinality type as Dictionary Arrow type. Default value - `false`.
+- [output_format_arrow_string_as_string](/docs/en/operations/settings/settings.md/#output_format_arrow_string_as_string) - use Arrow String type instead of Binary for String columns. Default value - `false`.
+- [input_format_arrow_import_nested](/docs/en/operations/settings/settings.md/#input_format_arrow_import_nested) - allow inserting array of structs into Nested table in Arrow input format. Default value - `false`.
+- [input_format_arrow_case_insensitive_column_matching](/docs/en/operations/settings/settings.md/#input_format_arrow_case_insensitive_column_matching) - ignore case when matching Arrow columns with ClickHouse columns. Default value - `false`.
+- [input_format_arrow_allow_missing_columns](/docs/en/operations/settings/settings.md/#input_format_arrow_allow_missing_columns) - allow missing columns while reading Arrow data. Default value - `false`.
+- [input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings.md/#input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Arrow format. Default value - `false`.
## ArrowStream {#data-format-arrow-stream}
@@ -2036,35 +2036,30 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Arrow" > {filenam
### Data Types Matching {#data-types-matching-orc}
-The table below shows supported data types and how they match ClickHouse [data types](../sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries.
+The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries.
-| ORC data type (`INSERT`) | ClickHouse data type | ORC data type (`SELECT`) |
-|--------------------------|-----------------------------------------------------|--------------------------|
-| `UINT8`, `BOOL` | [UInt8](../sql-reference/data-types/int-uint.md) | `UINT8` |
-| `INT8` | [Int8](../sql-reference/data-types/int-uint.md) | `INT8` |
-| `UINT16` | [UInt16](../sql-reference/data-types/int-uint.md) | `UINT16` |
-| `INT16` | [Int16](../sql-reference/data-types/int-uint.md) | `INT16` |
-| `UINT32` | [UInt32](../sql-reference/data-types/int-uint.md) | `UINT32` |
-| `INT32` | [Int32](../sql-reference/data-types/int-uint.md) | `INT32` |
-| `UINT64` | [UInt64](../sql-reference/data-types/int-uint.md) | `UINT64` |
-| `INT64` | [Int64](../sql-reference/data-types/int-uint.md) | `INT64` |
-| `FLOAT`, `HALF_FLOAT` | [Float32](../sql-reference/data-types/float.md) | `FLOAT` |
-| `DOUBLE` | [Float64](../sql-reference/data-types/float.md) | `DOUBLE` |
-| `DATE32` | [Date](../sql-reference/data-types/date.md) | `DATE32` |
-| `DATE64`, `TIMESTAMP` | [DateTime](../sql-reference/data-types/datetime.md) | `TIMESTAMP` |
-| `STRING`, `BINARY` | [String](../sql-reference/data-types/string.md) | `BINARY` |
-| `DECIMAL` | [Decimal](../sql-reference/data-types/decimal.md) | `DECIMAL` |
-| `LIST` | [Array](../sql-reference/data-types/array.md) | `LIST` |
-| `STRUCT` | [Tuple](../sql-reference/data-types/tuple.md) | `STRUCT` |
-| `MAP` | [Map](../sql-reference/data-types/map.md) | `MAP` |
+| ORC data type (`INSERT`) | ClickHouse data type | ORC data type (`SELECT`) |
+|---------------------------------------|---------------------------------------------------------|--------------------------|
+| `Boolean` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `Boolean` |
+| `Tinyint` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `Tinyint` |
+| `Smallint` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `Smallint` |
+| `Int` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `Int` |
+| `Bigint` | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | `Bigint` |
+| `Float` | [Float32](/docs/en/sql-reference/data-types/float.md) | `Float` |
+| `Double` | [Float64](/docs/en/sql-reference/data-types/float.md) | `Double` |
+| `Decimal` | [Decimal](/docs/en/sql-reference/data-types/decimal.md) | `Decimal` |
+| `Date` | [Date32](/docs/en/sql-reference/data-types/date32.md) | `Date` |
+| `Timestamp` | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `Timestamp` |
+| `String`, `Char`, `Varchar`, `Binary` | [String](/docs/en/sql-reference/data-types/string.md) | `Binary` |
+| `List` | [Array](/docs/en/sql-reference/data-types/array.md) | `List` |
+| `Struct` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `Struct` |
+| `Map` | [Map](/docs/en/sql-reference/data-types/map.md) | `Map` |
+
+Other types are not supported.
Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` and `Map` types also can be nested.
-ClickHouse supports configurable precision of the `Decimal` type. The `INSERT` query treats the ORC `DECIMAL` type as the ClickHouse `Decimal128` type.
-
-Unsupported ORC data types: `TIME32`, `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`.
-
-The data types of ClickHouse table columns do not have to match the corresponding ORC data fields. When inserting data, ClickHouse interprets data types according to the table above and then [casts](../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) the data to the data type set for the ClickHouse table column.
+The data types of ClickHouse table columns do not have to match the corresponding ORC data fields. When inserting data, ClickHouse interprets data types according to the table above and then [casts](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_function-cast) the data to the data type set for the ClickHouse table column.
### Inserting Data {#inserting-data-orc}
@@ -2084,18 +2079,18 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT ORC" > {filename.
### Arrow format settings {#parquet-format-settings}
-- [output_format_arrow_string_as_string](../operations/settings/settings.md#output_format_arrow_string_as_string) - use Arrow String type instead of Binary for String columns. Default value - `false`.
-- [input_format_arrow_import_nested](../operations/settings/settings.md#input_format_arrow_import_nested) - allow inserting array of structs into Nested table in Arrow input format. Default value - `false`.
-- [input_format_arrow_case_insensitive_column_matching](../operations/settings/settings.md#input_format_arrow_case_insensitive_column_matching) - ignore case when matching Arrow columns with ClickHouse columns. Default value - `false`.
-- [input_format_arrow_allow_missing_columns](../operations/settings/settings.md#input_format_arrow_allow_missing_columns) - allow missing columns while reading Arrow data. Default value - `false`.
-- [input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference](../operations/settings/settings.md#input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Arrow format. Default value - `false`.
+- [output_format_arrow_string_as_string](/docs/en/operations/settings/settings.md/#output_format_arrow_string_as_string) - use Arrow String type instead of Binary for String columns. Default value - `false`.
+- [input_format_arrow_import_nested](/docs/en/operations/settings/settings.md/#input_format_arrow_import_nested) - allow inserting array of structs into Nested table in Arrow input format. Default value - `false`.
+- [input_format_arrow_case_insensitive_column_matching](/docs/en/operations/settings/settings.md/#input_format_arrow_case_insensitive_column_matching) - ignore case when matching Arrow columns with ClickHouse columns. Default value - `false`.
+- [input_format_arrow_allow_missing_columns](/docs/en/operations/settings/settings.md/#input_format_arrow_allow_missing_columns) - allow missing columns while reading Arrow data. Default value - `false`.
+- [input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings.md/#input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Arrow format. Default value - `false`.
-To exchange data with Hadoop, you can use [HDFS table engine](../engines/table-engines/integrations/hdfs.md).
+To exchange data with Hadoop, you can use [HDFS table engine](/docs/en/engines/table-engines/integrations/hdfs.md).
## LineAsString {#lineasstring}
-In this format, every line of input data is interpreted as a single string value. This format can only be parsed for table with a single field of type [String](../sql-reference/data-types/string.md). The remaining columns must be set to [DEFAULT](../sql-reference/statements/create/table.md#default) or [MATERIALIZED](../sql-reference/statements/create/table.md#materialized), or omitted.
+In this format, every line of input data is interpreted as a single string value. This format can only be parsed for table with a single field of type [String](/docs/en/sql-reference/data-types/string.md). The remaining columns must be set to [DEFAULT](/docs/en/sql-reference/statements/create/table.md/#default) or [MATERIALIZED](/docs/en/sql-reference/statements/create/table.md/#materialized), or omitted.
**Example**
@@ -2122,9 +2117,9 @@ Each line of imported data is parsed according to the regular expression.
When working with the `Regexp` format, you can use the following settings:
-- `format_regexp` — [String](../sql-reference/data-types/string.md). Contains regular expression in the [re2](https://github.com/google/re2/wiki/Syntax) format.
+- `format_regexp` — [String](/docs/en/sql-reference/data-types/string.md). Contains regular expression in the [re2](https://github.com/google/re2/wiki/Syntax) format.
-- `format_regexp_escaping_rule` — [String](../sql-reference/data-types/string.md). The following escaping rules are supported:
+- `format_regexp_escaping_rule` — [String](/docs/en/sql-reference/data-types/string.md). The following escaping rules are supported:
- CSV (similarly to [CSV](#csv))
- JSON (similarly to [JSONEachRow](#jsoneachrow))
@@ -2132,17 +2127,17 @@ When working with the `Regexp` format, you can use the following settings:
- Quoted (similarly to [Values](#data-format-values))
- Raw (extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](#tabseparatedraw))
-- `format_regexp_skip_unmatched` — [UInt8](../sql-reference/data-types/int-uint.md). Defines the need to throw an exception in case the `format_regexp` expression does not match the imported data. Can be set to `0` or `1`.
+- `format_regexp_skip_unmatched` — [UInt8](/docs/en/sql-reference/data-types/int-uint.md). Defines the need to throw an exception in case the `format_regexp` expression does not match the imported data. Can be set to `0` or `1`.
**Usage**
-The regular expression from [format_regexp](../operations/settings/settings.md#format_regexp) setting is applied to every line of imported data. The number of subpatterns in the regular expression must be equal to the number of columns in imported dataset.
+The regular expression from [format_regexp](/docs/en/operations/settings/settings.md/#format_regexp) setting is applied to every line of imported data. The number of subpatterns in the regular expression must be equal to the number of columns in imported dataset.
Lines of the imported data must be separated by newline character `'\n'` or DOS-style newline `"\r\n"`.
-The content of every matched subpattern is parsed with the method of corresponding data type, according to [format_regexp_escaping_rule](../operations/settings/settings.md#format_regexp_escaping_rule) setting.
+The content of every matched subpattern is parsed with the method of corresponding data type, according to [format_regexp_escaping_rule](/docs/en/operations/settings/settings.md/#format_regexp_escaping_rule) setting.
-If the regular expression does not match the line and [format_regexp_skip_unmatched](../operations/settings/settings.md#format_regexp_escaping_rule) is set to 1, the line is silently skipped. Otherwise, exception is thrown.
+If the regular expression does not match the line and [format_regexp_skip_unmatched](/docs/en/operations/settings/settings.md/#format_regexp_escaping_rule) is set to 1, the line is silently skipped. Otherwise, exception is thrown.
**Example**
@@ -2190,25 +2185,25 @@ e.g. `schemafile.proto:MessageType`.
If the file has the standard extension for the format (for example, `.proto` for `Protobuf`),
it can be omitted and in this case, the format schema looks like `schemafile:MessageType`.
-If you input or output data via the [client](../interfaces/cli.md) in the [interactive mode](../interfaces/cli.md#cli_usage), the file name specified in the format schema
+If you input or output data via the [client](/docs/en/interfaces/cli.md) in the [interactive mode](/docs/en/interfaces/cli.md/#cli_usage), the file name specified in the format schema
can contain an absolute path or a path relative to the current directory on the client.
-If you use the client in the [batch mode](../interfaces/cli.md#cli_usage), the path to the schema must be relative due to security reasons.
+If you use the client in the [batch mode](/docs/en/interfaces/cli.md/#cli_usage), the path to the schema must be relative due to security reasons.
-If you input or output data via the [HTTP interface](../interfaces/http.md) the file name specified in the format schema
-should be located in the directory specified in [format_schema_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path)
+If you input or output data via the [HTTP interface](/docs/en/interfaces/http.md) the file name specified in the format schema
+should be located in the directory specified in [format_schema_path](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-format_schema_path)
in the server configuration.
## Skipping Errors {#skippingerrors}
-Some formats such as `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` and `Protobuf` can skip broken row if parsing error occurred and continue parsing from the beginning of next row. See [input_format_allow_errors_num](../operations/settings/settings.md#input_format_allow_errors_num) and
-[input_format_allow_errors_ratio](../operations/settings/settings.md#input_format_allow_errors_ratio) settings.
+Some formats such as `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` and `Protobuf` can skip broken row if parsing error occurred and continue parsing from the beginning of next row. See [input_format_allow_errors_num](/docs/en/operations/settings/settings.md/#input_format_allow_errors_num) and
+[input_format_allow_errors_ratio](/docs/en/operations/settings/settings.md/#input_format_allow_errors_ratio) settings.
Limitations:
- In case of parsing error `JSONEachRow` skips all data until the new line (or EOF), so rows must be delimited by `\n` to count errors correctly.
- `Template` and `CustomSeparated` use delimiter after the last column and delimiter between rows to find the beginning of next row, so skipping errors works only if at least one of them is not empty.
## RawBLOB {#rawblob}
-In this format, all input data is read to a single value. It is possible to parse only a table with a single field of type [String](../sql-reference/data-types/string.md) or similar.
+In this format, all input data is read to a single value. It is possible to parse only a table with a single field of type [String](/docs/en/sql-reference/data-types/string.md) or similar.
The result is output in binary format without delimiters and escaping. If more than one value is output, the format is ambiguous, and it will be impossible to read the data back.
Below is a comparison of the formats `RawBLOB` and [TabSeparatedRaw](#tabseparatedraw).
@@ -2255,18 +2250,18 @@ ClickHouse supports reading and writing [MessagePack](https://msgpack.org/) data
| MessagePack data type (`INSERT`) | ClickHouse data type | MessagePack data type (`SELECT`) |
|--------------------------------------------------------------------|-----------------------------------------------------------|------------------------------------|
-| `uint N`, `positive fixint` | [UIntN](../sql-reference/data-types/int-uint.md) | `uint N` |
-| `int N` | [IntN](../sql-reference/data-types/int-uint.md) | `int N` |
-| `bool` | [UInt8](../sql-reference/data-types/int-uint.md) | `uint 8` |
-| `fixstr`, `str 8`, `str 16`, `str 32`, `bin 8`, `bin 16`, `bin 32` | [String](../sql-reference/data-types/string.md) | `bin 8`, `bin 16`, `bin 32` |
-| `fixstr`, `str 8`, `str 16`, `str 32`, `bin 8`, `bin 16`, `bin 32` | [FixedString](../sql-reference/data-types/fixedstring.md) | `bin 8`, `bin 16`, `bin 32` |
-| `float 32` | [Float32](../sql-reference/data-types/float.md) | `float 32` |
-| `float 64` | [Float64](../sql-reference/data-types/float.md) | `float 64` |
-| `uint 16` | [Date](../sql-reference/data-types/date.md) | `uint 16` |
-| `uint 32` | [DateTime](../sql-reference/data-types/datetime.md) | `uint 32` |
-| `uint 64` | [DateTime64](../sql-reference/data-types/datetime.md) | `uint 64` |
-| `fixarray`, `array 16`, `array 32` | [Array](../sql-reference/data-types/array.md) | `fixarray`, `array 16`, `array 32` |
-| `fixmap`, `map 16`, `map 32` | [Map](../sql-reference/data-types/map.md) | `fixmap`, `map 16`, `map 32` |
+| `uint N`, `positive fixint` | [UIntN](/docs/en/sql-reference/data-types/int-uint.md) | `uint N` |
+| `int N`, `negative fixint` | [IntN](/docs/en/sql-reference/data-types/int-uint.md) | `int N` |
+| `bool` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `uint 8` |
+| `fixstr`, `str 8`, `str 16`, `str 32`, `bin 8`, `bin 16`, `bin 32` | [String](/docs/en/sql-reference/data-types/string.md) | `bin 8`, `bin 16`, `bin 32` |
+| `fixstr`, `str 8`, `str 16`, `str 32`, `bin 8`, `bin 16`, `bin 32` | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `bin 8`, `bin 16`, `bin 32` |
+| `float 32` | [Float32](/docs/en/sql-reference/data-types/float.md) | `float 32` |
+| `float 64` | [Float64](/docs/en/sql-reference/data-types/float.md) | `float 64` |
+| `uint 16` | [Date](/docs/en/sql-reference/data-types/date.md) | `uint 16` |
+| `uint 32` | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `uint 32` |
+| `uint 64` | [DateTime64](/docs/en/sql-reference/data-types/datetime.md) | `uint 64` |
+| `fixarray`, `array 16`, `array 32` | [Array](/docs/en/sql-reference/data-types/array.md) | `fixarray`, `array 16`, `array 32` |
+| `fixmap`, `map 16`, `map 32` | [Map](/docs/en/sql-reference/data-types/map.md) | `fixmap`, `map 16`, `map 32` |
Example:
@@ -2280,17 +2275,17 @@ $ clickhouse-client --query="SELECT * FROM msgpack FORMAT MsgPack" > tmp_msgpack
### MsgPack format settings {#msgpack-format-settings}
-- [input_format_msgpack_number_of_columns](../operations/settings/settings.md#input_format_msgpack_number_of_columns) - the number of columns in inserted MsgPack data. Used for automatic schema inference from data. Default value - `0`.
-- [output_format_msgpack_uuid_representation](../operations/settings/settings.md#output_format_msgpack_uuid_representation) - the way how to output UUID in MsgPack format. Default value - `EXT`.
+- [input_format_msgpack_number_of_columns](/docs/en/operations/settings/settings.md/#input_format_msgpack_number_of_columns) - the number of columns in inserted MsgPack data. Used for automatic schema inference from data. Default value - `0`.
+- [output_format_msgpack_uuid_representation](/docs/en/operations/settings/settings.md/#output_format_msgpack_uuid_representation) - the way how to output UUID in MsgPack format. Default value - `EXT`.
## MySQLDump {#mysqldump}
ClickHouse supports reading MySQL [dumps](https://dev.mysql.com/doc/refman/8.0/en/mysqldump.html).
It reads all data from INSERT queries belonging to one table in dump. If there are more than one table, by default it reads data from the first one.
-You can specify the name of the table from which to read data from using [input_format_mysql_dump_table_name](../operations/settings/settings.md#input_format_mysql_dump_table_name) settings.
-If setting [input_format_mysql_dump_map_columns](../operations/settings/settings.md#input_format_mysql_dump_map_columns) is set to 1 and
+You can specify the name of the table from which to read data from using [input_format_mysql_dump_table_name](/docs/en/operations/settings/settings.md/#input_format_mysql_dump_table_name) settings.
+If setting [input_format_mysql_dump_map_columns](/docs/en/operations/settings/settings.md/#input_format_mysql_dump_map_columns) is set to 1 and
dump contains CREATE query for specified table or column names in INSERT query the columns from input data will be mapped to the columns from the table by their names,
-columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1.
+columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings.md/#input_format_skip_unknown_fields) is set to 1.
This format supports schema inference: if the dump contains CREATE query for the specified table, the structure is extracted from it, otherwise schema is inferred from the data of INSERT queries.
Examples:
@@ -2349,3 +2344,5 @@ Query id: 17d59664-ebce-4053-bb79-d46a516fb590
│ 3 │
└───┘
```
+
+[Original article](https://clickhouse.com/docs/en/interfaces/formats)
diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md
new file mode 100644
index 00000000000..394e6906a23
--- /dev/null
+++ b/docs/en/interfaces/schema-inference.md
@@ -0,0 +1,1573 @@
+---
+slug: /en/interfaces/schema-inference
+sidebar_position: 21
+sidebar_label: Schema inference
+title: Automatic schema inference from input data
+---
+
+ClickHouse can automatically determine the structure of input data in almost all supported [Input formats](formats.md).
+This document will describe when schema inference is used, how it works with different input formats and which settings
+can control it.
+
+## Usage {#usage}
+
+Schema inference is used when ClickHouse needs to read the data in a specific data format and the structure is unknown.
+
+## Table functions [file](../sql-reference/table-functions/file.md), [s3](../sql-reference/table-functions/s3.md), [url](../sql-reference/table-functions/url.md), [hdfs](../sql-reference/table-functions/hdfs.md).
+
+These table functions have the optional argument `structure` with the structure of input data. If this argument is not specified or set to `auto`, the structure will be inferred from the data.
+
+**Example:**
+
+Let's say we have a file `hobbies.jsonl` in JSONEachRow format in the `user_files` directory with this content:
+```json
+{"id" : 1, "age" : 25, "name" : "Josh", "hobbies" : ["football", "cooking", "music"]}
+{"id" : 2, "age" : 19, "name" : "Alan", "hobbies" : ["tennis", "art"]}
+{"id" : 3, "age" : 32, "name" : "Lana", "hobbies" : ["fitness", "reading", "shopping"]}
+{"id" : 4, "age" : 47, "name" : "Brayan", "hobbies" : ["movies", "skydiving"]}
+```
+
+ClickHouse can read this data without you specifying its structure:
+```sql
+SELECT * FROM file('hobbies.jsonl')
+```
+```response
+┌─id─┬─age─┬─name───┬─hobbies──────────────────────────┐
+│ 1 │ 25 │ Josh │ ['football','cooking','music'] │
+│ 2 │ 19 │ Alan │ ['tennis','art'] │
+│ 3 │ 32 │ Lana │ ['fitness','reading','shopping'] │
+│ 4 │ 47 │ Brayan │ ['movies','skydiving'] │
+└────┴─────┴────────┴──────────────────────────────────┘
+```
+
+Note: the format `JSONEachRow` was automatically determined by the file extension `.jsonl`.
+
+You can see an automatically determined structure using the `DESCRIBE` query:
+```sql
+DESCRIBE file('hobbies.jsonl')
+```
+```response
+┌─name────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ id │ Nullable(Int64) │ │ │ │ │ │
+│ age │ Nullable(Int64) │ │ │ │ │ │
+│ name │ Nullable(String) │ │ │ │ │ │
+│ hobbies │ Array(Nullable(String)) │ │ │ │ │ │
+└─────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## Table engines [File](../engines/table-engines/special/file.md), [S3](../engines/table-engines/integrations/s3.md), [URL](../engines/table-engines/special/url.md), [HDFS](../engines/table-engines/integrations/hdfs.md)
+
+If the list of columns is not specified in `CREATE TABLE` query, the structure of the table will be inferred automatically from the data.
+
+**Example:**
+
+Let's use the file `hobbies.jsonl`. We can create a table with engine `File` with the data from this file:
+```sql
+CREATE TABLE hobbies ENGINE=File(JSONEachRow, 'hobbies.jsonl')
+```
+```response
+Ok.
+```
+```sql
+SELECT * FROM hobbies
+```
+```response
+┌─id─┬─age─┬─name───┬─hobbies──────────────────────────┐
+│ 1 │ 25 │ Josh │ ['football','cooking','music'] │
+│ 2 │ 19 │ Alan │ ['tennis','art'] │
+│ 3 │ 32 │ Lana │ ['fitness','reading','shopping'] │
+│ 4 │ 47 │ Brayan │ ['movies','skydiving'] │
+└────┴─────┴────────┴──────────────────────────────────┘
+```
+```sql
+DESCRIBE TABLE hobbies
+```
+```response
+┌─name────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ id │ Nullable(Int64) │ │ │ │ │ │
+│ age │ Nullable(Int64) │ │ │ │ │ │
+│ name │ Nullable(String) │ │ │ │ │ │
+│ hobbies │ Array(Nullable(String)) │ │ │ │ │ │
+└─────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## clickhouse-local
+
+`clickhouse-local` has an optional parameter `-S/--structure` with the structure of input data. If this parameter is not specified or set to `auto`, the structure will be inferred from the data.
+
+**Example:**
+
+Let's use the file `hobbies.jsonl`. We can query the data from this file using `clickhouse-local`:
+```shell
+clickhouse-local --file='hobbies.jsonl' --table='hobbies' --query='DESCRIBE TABLE hobbies'
+```
+```response
+id Nullable(Int64)
+age Nullable(Int64)
+name Nullable(String)
+hobbies Array(Nullable(String))
+```
+```shell
+clickhouse-local --file='hobbies.jsonl' --table='hobbies' --query='SELECT * FROM hobbies'
+```
+```response
+1 25 Josh ['football','cooking','music']
+2 19 Alan ['tennis','art']
+3 32 Lana ['fitness','reading','shopping']
+4 47 Brayan ['movies','skydiving']
+```
+
+# Using structure from insertion table {#using-structure-from-insertion-table}
+
+When table functions `file/s3/url/hdfs` are used to insert data into a table,
+there is an option to use the structure from the insertion table instead of extracting it from the data.
+It can improve insertion performance because schema inference can take some time. Also, it will be helpful when the table has an optimized schema, so
+no conversions between types will be performed.
+
+There is a special setting [use_structure_from_insertion_table_in_table_functions](/docs/en/operations/settings/settings.md/#use_structure_from_insertion_table_in_table_functions)
+that controls this behaviour. It has 3 possible values:
+- 0 - table function will extract the structure from the data.
+- 1 - table function will use the structure from the insertion table.
+- 2 - ClickHouse will automatically determine if it's possible to use the structure from the insertion table or use schema inference. Default value.
+
+**Example 1:**
+
+Let's create table `hobbies1` with the next structure:
+```sql
+CREATE TABLE hobbies1
+(
+ `id` UInt64,
+ `age` LowCardinality(UInt8),
+ `name` String,
+ `hobbies` Array(String)
+)
+ENGINE = MergeTree
+ORDER BY id;
+```
+
+And insert data from the file `hobbies.jsonl`:
+
+```sql
+INSERT INTO hobbies1 SELECT * FROM file(hobbies.jsonl)
+```
+
+In this case, all columns from the file are inserted into the table without changes, so ClickHouse will use the structure from the insertion table instead of schema inference.
+
+**Example 2:**
+
+Let's create table `hobbies2` with the next structure:
+```sql
+CREATE TABLE hobbies2
+(
+ `id` UInt64,
+ `age` LowCardinality(UInt8),
+ `hobbies` Array(String)
+)
+ ENGINE = MergeTree
+ORDER BY id;
+```
+
+And insert data from the file `hobbies.jsonl`:
+
+```sql
+INSERT INTO hobbies2 SELECT id, age, hobbies FROM file(hobbies.jsonl)
+```
+
+In this case, all columns in the `SELECT` query are present in the table, so ClickHouse will use the structure from the insertion table.
+Note that it will work only for input formats that support reading a subset of columns like JSONEachRow, TSKV, Parquet, etc. (so it won't work for example for TSV format).
+
+**Example 3:**
+
+Let's create table `hobbies3` with the next structure:
+
+```sql
+CREATE TABLE hobbies3
+(
+ `identifier` UInt64,
+ `age` LowCardinality(UInt8),
+ `hobbies` Array(String)
+)
+ ENGINE = MergeTree
+ORDER BY identifier;
+```
+
+And insert data from the file `hobbies.jsonl`:
+
+```sql
+INSERT INTO hobbies3 SELECT id, age, hobbies FROM file(hobbies.jsonl)
+```
+
+In this case, column `id` is used in the `SELECT` query, but the table doesn't have this column (it has a column with the name `identifier`),
+so ClickHouse cannot use the structure from the insertion table, and schema inference will be used.
+
+**Example 4:**
+
+Let's create table `hobbies4` with the next structure:
+
+```sql
+CREATE TABLE hobbies4
+(
+ `id` UInt64,
+ `any_hobby` Nullable(String)
+)
+ ENGINE = MergeTree
+ORDER BY id;
+```
+
+And insert data from the file `hobbies.jsonl`:
+
+```sql
+INSERT INTO hobbies4 SELECT id, empty(hobbies) ? NULL : hobbies[1] FROM file(hobbies.jsonl)
+```
+
+In this case, there are some operations performed on the column `hobbies` in the `SELECT` query to insert it into the table, so ClickHouse cannot use the structure from the insertion table, and schema inference will be used.
+
+# Schema inference cache {#schema-inference-cache}
+
+For most input formats schema inference reads some data to determine its structure and this process can take some time.
+To prevent inferring the same schema every time ClickHouse read the data from the same file, the inferred schema is cached and when accessing the same file again, ClickHouse will use the schema from the cache.
+
+There are special settings that control this cache:
+- `schema_inference_cache_max_elements_for_{file/s3/hdfs/url}` - the maximum number of cached schemas for the corresponding table function. The default value is `4096`. These settings should be set in the server config.
+- `use_cache_for_{file,s3,hdfs,url}_schema_inference` - allows turning on/off using cache for schema inference. These settings can be used in queries.
+
+The schema of the file can be changed by modifying the data or by changing format settings.
+For this reason, the schema inference cache identifies the schema by file source, format name, used format settings, and the last modification time of the file.
+
+Note: some files accessed by url in `url` table function may not contain information about the last modification time; for this case, there is a special setting
+`schema_inference_cache_require_modification_time_for_url`. Disabling this setting allows the use of the schema from cache without the last modification time for such files.
+
+There is also a system table [schema_inference_cache](../operations/system-tables/schema_inference_cache.md) with all current schemas in cache and system query `SYSTEM DROP SCHEMA CACHE [FOR File/S3/URL/HDFS]`
+that allows cleaning the schema cache for all sources, or for a specific source.
+
+**Examples:**
+
+Let's try to infer the structure of a sample dataset from s3 `github-2022.ndjson.gz` and see how the schema inference cache works:
+
+```sql
+DESCRIBE TABLE s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/github-2022.ndjson.gz')
+SETTINGS allow_experimental_object_type = 1
+```
+```response
+┌─name───────┬─type─────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ type │ Nullable(String) │ │ │ │ │ │
+│ actor │ Object(Nullable('json')) │ │ │ │ │ │
+│ repo │ Object(Nullable('json')) │ │ │ │ │ │
+│ created_at │ Nullable(String) │ │ │ │ │ │
+│ payload │ Object(Nullable('json')) │ │ │ │ │ │
+└────────────┴──────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+
+5 rows in set. Elapsed: 0.601 sec.
+```
+```sql
+DESCRIBE TABLE s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/github-2022.ndjson.gz')
+SETTINGS allow_experimental_object_type = 1
+```
+```response
+┌─name───────┬─type─────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ type │ Nullable(String) │ │ │ │ │ │
+│ actor │ Object(Nullable('json')) │ │ │ │ │ │
+│ repo │ Object(Nullable('json')) │ │ │ │ │ │
+│ created_at │ Nullable(String) │ │ │ │ │ │
+│ payload │ Object(Nullable('json')) │ │ │ │ │ │
+└────────────┴──────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+
+5 rows in set. Elapsed: 0.059 sec.
+```
+
+As you can see, the second query succeeded almost instantly.
+
+Let's try to change some settings that can affect inferred schema:
+
+```sql
+DESCRIBE TABLE s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/github-2022.ndjson.gz')
+SETTINGS input_format_json_read_objects_as_strings = 1
+
+┌─name───────┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ type │ Nullable(String) │ │ │ │ │ │
+│ actor │ Nullable(String) │ │ │ │ │ │
+│ repo │ Nullable(String) │ │ │ │ │ │
+│ created_at │ Nullable(String) │ │ │ │ │ │
+│ payload │ Nullable(String) │ │ │ │ │ │
+└────────────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+
+5 rows in set. Elapsed: 0.611 sec
+```
+
+As you can see, the schema from the cache was not used for the same file, because the setting that can affect inferred schema was changed.
+
+Let's check the content of `system.schema_inference_cache` table:
+
+```sql
+SELECT schema, format, source FROM system.schema_inference_cache WHERE storage='S3'
+```
+```response
+┌─schema──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─format─┬─source───────────────────────────────────────────────────────────────────────────────────────────────────┐
+│ type Nullable(String), actor Object(Nullable('json')), repo Object(Nullable('json')), created_at Nullable(String), payload Object(Nullable('json')) │ NDJSON │ datasets-documentation.s3.eu-west-3.amazonaws.com443/datasets-documentation/github/github-2022.ndjson.gz │
+│ type Nullable(String), actor Nullable(String), repo Nullable(String), created_at Nullable(String), payload Nullable(String) │ NDJSON │ datasets-documentation.s3.eu-west-3.amazonaws.com443/datasets-documentation/github/github-2022.ndjson.gz │
+└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴────────┴──────────────────────────────────────────────────────────────────────────────────────────────────────────┘
+```
+
+As you can see, there are two different schemas for the same file.
+
+We can clear the schema cache using a system query:
+```sql
+SYSTEM DROP SCHEMA CACHE FOR S3
+```
+```response
+Ok.
+```
+```sql
+SELECT count() FROM system.schema_inference_cache WHERE storage='S3'
+```
+```response
+┌─count()─┐
+│ 0 │
+└─────────┘
+```
+
+# Text formats {#text-formats}
+
+For text formats, ClickHouse reads the data row by row, extracts column values according to the format,
+and then uses some recursive parsers and heuristics to determine the type for each value. The maximum number of rows read from the data in schema inference
+is controlled by the setting `input_format_max_rows_to_read_for_schema_inference` with default value 25000.
+By default, all inferred types are [Nullable](../sql-reference/data-types/nullable.md), but you can change this by setting `schema_inference_make_columns_nullable` (see examples in the [settings](#settings-for-text-formats) section).
+
+## JSON formats {#json-formats}
+
+In JSON formats ClickHouse parses values according to the JSON specification and then tries to find the most appropriate data type for them.
+
+Let's see how it works, what types can be inferred and what specific settings can be used in JSON formats.
+
+**Examples**
+
+Here and further, the [format](../sql-reference/table-functions/format.md) table function will be used in examples.
+
+Integers, Floats, Bools, Strings:
+```sql
+DESC format(JSONEachRow, '{"int" : 42, "float" : 42.42, "string" : "Hello, World!"}');
+```
+```response
+┌─name───┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ int │ Nullable(Int64) │ │ │ │ │ │
+│ float │ Nullable(Float64) │ │ │ │ │ │
+│ bool │ Nullable(Bool) │ │ │ │ │ │
+│ string │ Nullable(String) │ │ │ │ │ │
+└────────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Dates, DateTimes:
+
+```sql
+DESC format(JSONEachRow, '{"date" : "2022-01-01", "datetime" : "2022-01-01 00:00:00"}')
+```
+```response
+┌─name─────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ date │ Nullable(Date) │ │ │ │ │ │
+│ datetime │ Nullable(DateTime64(9)) │ │ │ │ │ │
+└──────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Arrays:
+```sql
+DESC format(JSONEachRow, '{"arr" : [1, 2, 3], "nested_arrays" : [[1, 2, 3], [4, 5, 6], []]}')
+```
+```response
+┌─name──────────┬─type──────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ arr │ Array(Nullable(Int64)) │ │ │ │ │ │
+│ nested_arrays │ Array(Array(Nullable(Int64))) │ │ │ │ │ │
+└───────────────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+If an array contains `null`, ClickHouse will use types from the other array elements:
+```sql
+DESC format(JSONEachRow, '{"arr" : [null, 42, null]}')
+```
+```response
+┌─name─┬─type───────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ arr │ Array(Nullable(Int64)) │ │ │ │ │ │
+└──────┴────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Tuples:
+
+In JSON formats we treat Arrays with elements of different types as Tuples.
+```sql
+DESC format(JSONEachRow, '{"tuple" : [1, "Hello, World!", [1, 2, 3]]}')
+```
+```response
+┌─name──┬─type─────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ tuple │ Tuple(Nullable(Int64), Nullable(String), Array(Nullable(Int64))) │ │ │ │ │ │
+└───────┴──────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+If some values are `null` or empty, we use types of corresponding values from the other rows:
+```sql
+DESC format(JSONEachRow, $$
+ {"tuple" : [1, null, null]}
+ {"tuple" : [null, "Hello, World!", []]}
+ {"tuple" : [null, null, [1, 2, 3]]}
+ $$)
+```
+```response
+┌─name──┬─type─────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ tuple │ Tuple(Nullable(Int64), Nullable(String), Array(Nullable(Int64))) │ │ │ │ │ │
+└───────┴──────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Maps:
+
+In JSON we can read objects with values of the same type as Map type.
+```sql
+DESC format(JSONEachRow, '{"map" : {"key1" : 42, "key2" : 24, "key3" : 4}}')
+```
+```response
+┌─name─┬─type─────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ map │ Map(String, Nullable(Int64)) │ │ │ │ │ │
+└──────┴──────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+JSON Object type (if setting `allow_experimental_object_type` is enabled):
+
+```sql
+SET allow_experimental_object_type = 1
+DESC format(JSONEachRow, $$
+ {"obj" : {"key1" : 42}}
+ {"obj" : {"key2" : "Hello, World!"}}
+ {"obj" : {"key1" : 24, "key3" : {"a" : 42, "b" : null}}}
+ $$)
+```
+```response
+┌─name─┬─type─────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ obj │ Object(Nullable('json')) │ │ │ │ │ │
+└──────┴──────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Nested complex types:
+```sql
+DESC format(JSONEachRow, '{"value" : [[[42, 24], []], {"key1" : 42, "key2" : 24}]}')
+```
+```response
+┌─name──┬─type───────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ value │ Tuple(Array(Array(Nullable(Int64))), Map(String, Nullable(Int64))) │ │ │ │ │ │
+└───────┴────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+If ClickHouse cannot determine the type, because the data contains only nulls, an exception will be thrown:
+```sql
+DESC format(JSONEachRow, '{"arr" : [null, null]}')
+```
+```response
+Code: 652. DB::Exception: Received from localhost:9000. DB::Exception:
+Cannot determine type for column 'arr' by first 1 rows of data,
+most likely this column contains only Nulls or empty Arrays/Maps.
+...
+```
+
+### JSON settings {#json-settings}
+
+#### input_format_json_read_objects_as_strings
+
+Enabling this setting allows reading nested JSON objects as strings.
+This setting can be used to read nested JSON objects without using JSON object type.
+
+This setting is enabled by default.
+
+```sql
+SET input_format_json_read_objects_as_strings = 1;
+DESC format(JSONEachRow, $$
+ {"obj" : {"key1" : 42, "key2" : [1,2,3,4]}}
+ {"obj" : {"key3" : {"nested_key" : 1}}}
+ $$)
+```
+```response
+┌─name─┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ obj │ Nullable(String) │ │ │ │ │ │
+└──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+#### input_format_json_try_infer_numbers_from_strings
+
+Enabling this setting allows inferring numbers from string values.
+
+This setting is enabled by default.
+
+**Example:**
+
+```sql
+SET input_format_json_try_infer_numbers_from_strings = 1;
+DESC format(JSONEachRow, $$
+ {"value" : "42"}
+ {"value" : "424242424242"}
+ $$)
+```
+```reponse
+┌─name──┬─type────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ value │ Nullable(Int64) │ │ │ │ │ │
+└───────┴─────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+#### input_format_json_read_numbers_as_strings
+
+Enabling this setting allows reading numeric values as strings.
+
+This setting is disabled by default.
+
+**Example**
+
+```sql
+SET input_format_json_read_numbers_as_strings = 1;
+DESC format(JSONEachRow, $$
+ {"value" : 1055}
+ {"value" : "unknown"}
+ $$)
+```
+```response
+┌─name──┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ value │ Nullable(String) │ │ │ │ │ │
+└───────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+#### input_format_json_read_bools_as_numbers
+
+Enabling this setting allows reading Bool values as numbers.
+
+This setting is enabled by default.
+
+**Example:**
+
+```sql
+SET input_format_json_read_bools_as_numbers = 1;
+DESC format(JSONEachRow, $$
+ {"value" : true}
+ {"value" : 42}
+ $$)
+```
+```response
+┌─name──┬─type────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ value │ Nullable(Int64) │ │ │ │ │ │
+└───────┴─────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## CSV {#csv}
+
+In CSV format ClickHouse extracts column values from the row according to delimiters. ClickHouse expects all types except numbers and strings to be enclosed in double quotes. If the value is in double quotes, ClickHouse tries to parse
+the data inside quotes using the recursive parser and then tries to find the most appropriate data type for it. If the value is not in double quotes, ClickHouse tries to parse it as a number,
+and if the value is not a number, ClickHouse treats it as a string.
+
+If you don't want ClickHouse to try to determine complex types using some parsers and heuristics, you can disable setting `input_format_csv_use_best_effort_in_schema_inference`
+and ClickHouse will treat all columns as Strings.
+
+**Examples:**
+
+Integers, Floats, Bools, Strings:
+```sql
+DESC format(CSV, '42,42.42,true,"Hello,World!"')
+```
+```response
+┌─name─┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(Int64) │ │ │ │ │ │
+│ c2 │ Nullable(Float64) │ │ │ │ │ │
+│ c3 │ Nullable(Bool) │ │ │ │ │ │
+│ c4 │ Nullable(String) │ │ │ │ │ │
+└──────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Strings without quotes:
+```sql
+DESC format(CSV, 'Hello world!,World hello!')
+```
+```response
+┌─name─┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(String) │ │ │ │ │ │
+│ c2 │ Nullable(String) │ │ │ │ │ │
+└──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Dates, DateTimes:
+
+```sql
+DESC format(CSV, '"2020-01-01","2020-01-01 00:00:00"')
+```
+```response
+┌─name─┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(Date) │ │ │ │ │ │
+│ c2 │ Nullable(DateTime64(9)) │ │ │ │ │ │
+└──────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Arrays:
+```sql
+DESC format(CSV, '"[1,2,3]","[[1, 2], [], [3, 4]]"')
+```
+```response
+┌─name─┬─type──────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Array(Nullable(Int64)) │ │ │ │ │ │
+│ c2 │ Array(Array(Nullable(Int64))) │ │ │ │ │ │
+└──────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+```sql
+DESC format(CSV, $$"['Hello', 'world']","[['Abc', 'Def'], []]"$$)
+```
+```response
+┌─name─┬─type───────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Array(Nullable(String)) │ │ │ │ │ │
+│ c2 │ Array(Array(Nullable(String))) │ │ │ │ │ │
+└──────┴────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+If an array contains null, ClickHouse will use types from the other array elements:
+```sql
+DESC format(CSV, '"[NULL, 42, NULL]"')
+```
+```response
+┌─name─┬─type───────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Array(Nullable(Int64)) │ │ │ │ │ │
+└──────┴────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Maps:
+```sql
+DESC format(CSV, $$"{'key1' : 42, 'key2' : 24}"$$)
+```
+```response
+┌─name─┬─type─────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Map(String, Nullable(Int64)) │ │ │ │ │ │
+└──────┴──────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Nested Arrays and Maps:
+```sql
+DESC format(CSV, $$"[{'key1' : [[42, 42], []], 'key2' : [[null], [42]]}]"$$)
+```
+```response
+┌─name─┬─type──────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Array(Map(String, Array(Array(Nullable(Int64))))) │ │ │ │ │ │
+└──────┴───────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+If ClickHouse cannot determine the type inside quotes, because the data contains only nulls, ClickHouse will treat it as String:
+```sql
+DESC format(CSV, '"[NULL, NULL]"')
+```
+```response
+┌─name─┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(String) │ │ │ │ │ │
+└──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Example with disabled setting `input_format_csv_use_best_effort_in_schema_inference`:
+```sql
+SET input_format_csv_use_best_effort_in_schema_inference = 0
+DESC format(CSV, '"[1,2,3]",42.42,Hello World!')
+```
+```response
+┌─name─┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(String) │ │ │ │ │ │
+│ c2 │ Nullable(String) │ │ │ │ │ │
+│ c3 │ Nullable(String) │ │ │ │ │ │
+└──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## TSV/TSKV {#tsv-tskv}
+
+In TSV/TSKV formats ClickHouse extracts column value from the row according to tabular delimiters and then parses extracted value using
+the recursive parser to determine the most appropriate type. If the type cannot be determined, ClickHouse treats this value as String.
+
+If you don't want ClickHouse to try to determine complex types using some parsers and heuristics, you can disable setting `input_format_tsv_use_best_effort_in_schema_inference`
+and ClickHouse will treat all columns as Strings.
+
+
+**Examples:**
+
+Integers, Floats, Bools, Strings:
+```sql
+DESC format(TSV, '42 42.42 true Hello,World!')
+```
+```response
+┌─name─┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(Int64) │ │ │ │ │ │
+│ c2 │ Nullable(Float64) │ │ │ │ │ │
+│ c3 │ Nullable(Bool) │ │ │ │ │ │
+│ c4 │ Nullable(String) │ │ │ │ │ │
+└──────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+```sql
+DESC format(TSKV, 'int=42 float=42.42 bool=true string=Hello,World!\n')
+```
+```response
+┌─name───┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ int │ Nullable(Int64) │ │ │ │ │ │
+│ float │ Nullable(Float64) │ │ │ │ │ │
+│ bool │ Nullable(Bool) │ │ │ │ │ │
+│ string │ Nullable(String) │ │ │ │ │ │
+└────────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Dates, DateTimes:
+
+```sql
+DESC format(TSV, '2020-01-01 2020-01-01 00:00:00')
+```
+```response
+┌─name─┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(Date) │ │ │ │ │ │
+│ c2 │ Nullable(DateTime64(9)) │ │ │ │ │ │
+└──────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Arrays:
+```sql
+DESC format(TSV, '[1,2,3] [[1, 2], [], [3, 4]]')
+```
+```response
+┌─name─┬─type──────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Array(Nullable(Int64)) │ │ │ │ │ │
+│ c2 │ Array(Array(Nullable(Int64))) │ │ │ │ │ │
+└──────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+```sql
+DESC format(TSV, '[''Hello'', ''world''] [[''Abc'', ''Def''], []]')
+```
+```response
+┌─name─┬─type───────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Array(Nullable(String)) │ │ │ │ │ │
+│ c2 │ Array(Array(Nullable(String))) │ │ │ │ │ │
+└──────┴────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+If an array contains null, ClickHouse will use types from the other array elements:
+```sql
+DESC format(TSV, '[NULL, 42, NULL]')
+```
+```response
+┌─name─┬─type───────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Array(Nullable(Int64)) │ │ │ │ │ │
+└──────┴────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Tuples:
+```sql
+DESC format(TSV, $$(42, 'Hello, world!')$$)
+```
+```response
+┌─name─┬─type─────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Tuple(Nullable(Int64), Nullable(String)) │ │ │ │ │ │
+└──────┴──────────────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Maps:
+```sql
+DESC format(TSV, $${'key1' : 42, 'key2' : 24}$$)
+```
+```response
+┌─name─┬─type─────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Map(String, Nullable(Int64)) │ │ │ │ │ │
+└──────┴──────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Nested Arrays, Tuples and Maps:
+```sql
+DESC format(TSV, $$[{'key1' : [(42, 'Hello'), (24, NULL)], 'key2' : [(NULL, ','), (42, 'world!')]}]$$)
+```
+```response
+┌─name─┬─type────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Array(Map(String, Array(Tuple(Nullable(Int64), Nullable(String))))) │ │ │ │ │ │
+└──────┴─────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+If ClickHouse cannot determine the type, because the data contains only nulls, ClickHouse will treat it as String:
+```sql
+DESC format(TSV, '[NULL, NULL]')
+```
+```response
+┌─name─┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(String) │ │ │ │ │ │
+└──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Example with disabled setting `input_format_tsv_use_best_effort_in_schema_inference`:
+```sql
+SET input_format_tsv_use_best_effort_in_schema_inference = 0
+DESC format(TSV, '[1,2,3] 42.42 Hello World!')
+```
+```response
+┌─name─┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(String) │ │ │ │ │ │
+│ c2 │ Nullable(String) │ │ │ │ │ │
+│ c3 │ Nullable(String) │ │ │ │ │ │
+└──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## Values {#values}
+
+In Values format ClickHouse extracts column value from the row and then parses it using
+the recursive parser similar to how literals are parsed.
+
+**Examples:**
+
+Integers, Floats, Bools, Strings:
+```sql
+DESC format(Values, $$(42, 42.42, true, 'Hello,World!')$$)
+```
+```response
+┌─name─┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(Int64) │ │ │ │ │ │
+│ c2 │ Nullable(Float64) │ │ │ │ │ │
+│ c3 │ Nullable(Bool) │ │ │ │ │ │
+│ c4 │ Nullable(String) │ │ │ │ │ │
+└──────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Dates, DateTimes:
+
+```sql
+DESC format(Values, $$('2020-01-01', '2020-01-01 00:00:00')$$)
+```
+```response
+┌─name─┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(Date) │ │ │ │ │ │
+│ c2 │ Nullable(DateTime64(9)) │ │ │ │ │ │
+└──────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Arrays:
+```sql
+DESC format(Values, '([1,2,3], [[1, 2], [], [3, 4]])')
+```
+```response
+┌─name─┬─type──────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Array(Nullable(Int64)) │ │ │ │ │ │
+│ c2 │ Array(Array(Nullable(Int64))) │ │ │ │ │ │
+└──────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+If an array contains null, ClickHouse will use types from the other array elements:
+```sql
+DESC format(Values, '([NULL, 42, NULL])')
+```
+```response
+┌─name─┬─type───────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Array(Nullable(Int64)) │ │ │ │ │ │
+└──────┴────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Tuples:
+```sql
+DESC format(Values, $$((42, 'Hello, world!'))$$)
+```
+```response
+┌─name─┬─type─────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Tuple(Nullable(Int64), Nullable(String)) │ │ │ │ │ │
+└──────┴──────────────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Maps:
+```sql
+DESC format(Values, $$({'key1' : 42, 'key2' : 24})$$)
+```
+```response
+┌─name─┬─type─────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Map(String, Nullable(Int64)) │ │ │ │ │ │
+└──────┴──────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Nested Arrays, Tuples and Maps:
+```sql
+DESC format(Values, $$([{'key1' : [(42, 'Hello'), (24, NULL)], 'key2' : [(NULL, ','), (42, 'world!')]}])$$)
+```
+```response
+┌─name─┬─type────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Array(Map(String, Array(Tuple(Nullable(Int64), Nullable(String))))) │ │ │ │ │ │
+└──────┴─────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+If ClickHouse cannot determine the type, because the data contains only nulls, an exception will be thrown:
+```sql
+DESC format(Values, '([NULL, NULL])')
+```
+```response
+Code: 652. DB::Exception: Received from localhost:9000. DB::Exception:
+Cannot determine type for column 'c1' by first 1 rows of data,
+most likely this column contains only Nulls or empty Arrays/Maps.
+...
+```
+
+Example with disabled setting `input_format_tsv_use_best_effort_in_schema_inference`:
+```sql
+SET input_format_tsv_use_best_effort_in_schema_inference = 0
+DESC format(TSV, '[1,2,3] 42.42 Hello World!')
+```
+```response
+┌─name─┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(String) │ │ │ │ │ │
+│ c2 │ Nullable(String) │ │ │ │ │ │
+│ c3 │ Nullable(String) │ │ │ │ │ │
+└──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## CustomSeparated {#custom-separated}
+
+In CustomSeparated format ClickHouse first extracts all column values from the row according to specified delimiters and then tries to infer
+the data type for each value according to escaping rule.
+
+**Example**
+
+```sql
+SET format_custom_row_before_delimiter = '',
+ format_custom_row_after_delimiter = '\n',
+ format_custom_row_between_delimiter = '\n',
+ format_custom_result_before_delimiter = '\n',
+ format_custom_result_after_delimiter = '\n',
+ format_custom_field_delimiter = '',
+ format_custom_escaping_rule = 'Quoted'
+
+DESC format(CustomSeparated, $$
+42.42'Some string 1'[1, NULL, 3]
+
+NULL'Some string 3'[1, 2, NULL]
+
+$$)
+```
+```response
+┌─name─┬─type───────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(Float64) │ │ │ │ │ │
+│ c2 │ Nullable(String) │ │ │ │ │ │
+│ c3 │ Array(Nullable(Int64)) │ │ │ │ │ │
+└──────┴────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## Template {#template}
+
+In Template format ClickHouse first extracts all column values from the row according to the specified template and then tries to infer the
+data type for each value according to its escaping rule.
+
+**Example**
+
+Let's say we have a file `resultset` with the next content:
+```
+
+${data}
+```
+
+And a file `row_format` with the next content:
+```
+${column_1:CSV}${column_2:Quoted}${column_3:JSON}
+```
+
+Then we can make the next queries:
+
+```sql
+SET format_template_rows_between_delimiter = '\n',
+ format_template_row = 'row_format',
+ format_template_resultset = 'resultset_format'
+
+DESC format(Template, $$
+42.42'Some string 1'[1, null, 2]
+
+\N'Some string 3'[1, 2, null]
+
+$$)
+```
+```response
+┌─name─────┬─type───────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ column_1 │ Nullable(Float64) │ │ │ │ │ │
+│ column_2 │ Nullable(String) │ │ │ │ │ │
+│ column_3 │ Array(Nullable(Int64)) │ │ │ │ │ │
+└──────────┴────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## Regexp {#regexp}
+
+Similar to Template, in Regexp format ClickHouse first extracts all column values from the row according to specified regular expression and then tries to infer
+data type for each value according to the specified escaping rule.
+
+**Example**
+
+```sql
+SET format_regexp = '^Line: value_1=(.+?), value_2=(.+?), value_3=(.+?)',
+ format_regexp_escaping_rule = 'CSV'
+
+DESC format(Regexp, $$Line: value_1=42, value_2="Some string 1", value_3="[1, NULL, 3]"
+Line: value_1=2, value_2="Some string 2", value_3="[4, 5, NULL]"$$)
+```
+```response
+┌─name─┬─type───────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ c1 │ Nullable(Int64) │ │ │ │ │ │
+│ c2 │ Nullable(String) │ │ │ │ │ │
+│ c3 │ Array(Nullable(Int64)) │ │ │ │ │ │
+└──────┴────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## Settings for text formats {settings-for-text-formats}
+
+### input_format_max_rows_to_read_for_schema_inference
+
+This setting controls the maximum number of rows to be read while schema inference.
+The more rows are read, the more time is spent on schema inference, but the greater the chance to
+correctly determine the types (especially when the data contains a lot of nulls).
+
+Default value: `25000`.
+
+### column_names_for_schema_inference
+
+The list of column names to use in schema inference for formats without explicit column names. Specified names will be used instead of default `c1,c2,c3,...`. The format: `column1,column2,column3,...`.
+
+**Example**
+
+```sql
+DESC format(TSV, 'Hello, World! 42 [1, 2, 3]') settings column_names_for_schema_inference = 'str,int,arr'
+```
+```response
+┌─name─┬─type───────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ str │ Nullable(String) │ │ │ │ │ │
+│ int │ Nullable(Int64) │ │ │ │ │ │
+│ arr │ Array(Nullable(Int64)) │ │ │ │ │ │
+└──────┴────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+### schema_inference_hints
+
+The list of column names and types to use in schema inference instead of automatically determined types. The format: 'column_name1 column_type1, column_name2 column_type2, ...'.
+This setting can be used to specify the types of columns that could not be determined automatically or for optimizing the schema.
+
+**Example**
+
+```sql
+DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}'
+SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)'
+```
+```response
+┌─name────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ id │ Nullable(Int64) │ │ │ │ │ │
+│ age │ LowCardinality(UInt8) │ │ │ │ │ │
+│ name │ Nullable(String) │ │ │ │ │ │
+│ status │ Nullable(String) │ │ │ │ │ │
+│ hobbies │ Array(Nullable(String)) │ │ │ │ │ │
+└─────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+### schema_inference_make_columns_nullable
+
+Controls making inferred types `Nullable` in schema inference for formats without information about nullability.
+If the setting is enabled, all inferred type will be `Nullable`, if disabled, the inferred type will be `Nullable` only if the column contains `NULL` in a sample that is parsed during schema inference.
+
+Enabled by default.
+
+**Examples**
+
+```sql
+SET schema_inference_make_columns_nullable = 1
+DESC format(JSONEachRow, $$
+ {"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}
+ {"id" : 2, "age" : 19, "name" : "Alan", "status" : "married", "hobbies" : ["tennis", "art"]}
+ $$)
+```
+```response
+┌─name────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ id │ Nullable(Int64) │ │ │ │ │ │
+│ age │ Nullable(Int64) │ │ │ │ │ │
+│ name │ Nullable(String) │ │ │ │ │ │
+│ status │ Nullable(String) │ │ │ │ │ │
+│ hobbies │ Array(Nullable(String)) │ │ │ │ │ │
+└─────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+```sql
+SET schema_inference_make_columns_nullable = 0
+DESC format(JSONEachRow, $$
+ {"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}
+ {"id" : 2, "age" : 19, "name" : "Alan", "status" : "married", "hobbies" : ["tennis", "art"]}
+ $$)
+```
+```response
+
+┌─name────┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ id │ Int64 │ │ │ │ │ │
+│ age │ Int64 │ │ │ │ │ │
+│ name │ String │ │ │ │ │ │
+│ status │ Nullable(String) │ │ │ │ │ │
+│ hobbies │ Array(String) │ │ │ │ │ │
+└─────────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+### input_format_try_infer_integers
+
+If enabled, ClickHouse will try to infer integers instead of floats in schema inference for text formats.
+If all numbers in the column from sample data are integers, the result type will be `Int64`, if at least one number is float, the result type will be `Float64`.
+If the sample data contains only integers and at least one integer is positive and overflows `Int64`, ClickHouse will infer `UInt64`.
+
+Enabled by default.
+
+**Examples**
+
+```sql
+SET input_format_try_infer_integers = 0
+DESC format(JSONEachRow, $$
+ {"number" : 1}
+ {"number" : 2}
+ $$)
+```
+```response
+┌─name───┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ number │ Nullable(Float64) │ │ │ │ │ │
+└────────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+```sql
+SET input_format_try_infer_integers = 1
+DESC format(JSONEachRow, $$
+ {"number" : 1}
+ {"number" : 2}
+ $$)
+```
+```response
+┌─name───┬─type────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ number │ Nullable(Int64) │ │ │ │ │ │
+└────────┴─────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+```sql
+DESC format(JSONEachRow, $$
+ {"number" : 1}
+ {"number" : 18446744073709551615}
+ $$)
+```
+```response
+┌─name───┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ number │ Nullable(UInt64) │ │ │ │ │ │
+└────────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+```sql
+DESC format(JSONEachRow, $$
+ {"number" : 1}
+ {"number" : 2.2}
+ $$)
+```
+```response
+┌─name───┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ number │ Nullable(Float64) │ │ │ │ │ │
+└────────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+### input_format_try_infer_datetimes
+
+If enabled, ClickHouse will try to infer type `DateTime64` from string fields in schema inference for text formats.
+If all fields from a column in sample data were successfully parsed as datetimes, the result type will be `DateTime64(9)`,
+if at least one field was not parsed as datetime, the result type will be `String`.
+
+Enabled by default.
+
+**Examples**
+
+```sql
+SET input_format_try_infer_datetimes = 0
+DESC format(JSONEachRow, $$
+ {"datetime" : "2021-01-01 00:00:00.000"}
+ {"datetime" : "2022-01-01 00:00:00.000"}
+ $$)
+```
+```response
+┌─name─────┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ datetime │ Nullable(String) │ │ │ │ │ │
+└──────────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+```sql
+SET input_format_try_infer_datetimes = 1
+DESC format(JSONEachRow, $$
+ {"datetime" : "2021-01-01 00:00:00.000"}
+ {"datetime" : "2022-01-01 00:00:00.000"}
+ $$)
+```
+```response
+┌─name─────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ datetime │ Nullable(DateTime64(9)) │ │ │ │ │ │
+└──────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+```sql
+DESC format(JSONEachRow, $$
+ {"datetime" : "2021-01-01 00:00:00.000"}
+ {"datetime" : "unknown"}
+ $$)
+```
+```response
+┌─name─────┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ datetime │ Nullable(String) │ │ │ │ │ │
+└──────────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Note: Parsing datetimes during schema inference respect setting [date_time_input_format](/docs/en/operations/settings/settings.md#date_time_input_format)
+
+### input_format_try_infer_dates
+
+If enabled, ClickHouse will try to infer type `Date` from string fields in schema inference for text formats.
+If all fields from a column in sample data were successfully parsed as dates, the result type will be `Date`,
+if at least one field was not parsed as date, the result type will be `String`.
+
+Enabled by default.
+
+**Examples**
+
+```sql
+SET input_format_try_infer_datetimes = 0, input_format_try_infer_dates = 0
+DESC format(JSONEachRow, $$
+ {"date" : "2021-01-01"}
+ {"date" : "2022-01-01"}
+ $$)
+```
+```response
+┌─name─┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ date │ Nullable(String) │ │ │ │ │ │
+└──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+```sql
+SET input_format_try_infer_dates = 1
+DESC format(JSONEachRow, $$
+ {"date" : "2021-01-01"}
+ {"date" : "2022-01-01"}
+ $$)
+```
+```response
+┌─name─┬─type───────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ date │ Nullable(Date) │ │ │ │ │ │
+└──────┴────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+```sql
+DESC format(JSONEachRow, $$
+ {"date" : "2021-01-01"}
+ {"date" : "unknown"}
+ $$)
+```
+```response
+┌─name─┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ date │ Nullable(String) │ │ │ │ │ │
+└──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+# Self describing formats {#self-describing-formats}
+
+Self-describing formats contain information about the structure of the data in the data itself,
+it can be some header with a description, a binary type tree, or some kind of table.
+To automatically infer a schema from files in such formats, ClickHouse reads a part of the data containing
+information about the types and converts it into a schema of the ClickHouse table.
+
+## Formats with -WithNamesAndTypes suffix {#formats-with-names-and-types}
+
+ClickHouse supports some text formats with the suffix -WithNamesAndTypes. This suffix means that the data contains two additional rows with column names and types before the actual data.
+While schema inference for such formats, ClickHouse reads the first two rows and extracts column names and types.
+
+**Example**
+
+```sql
+DESC format(TSVWithNamesAndTypes,
+$$num str arr
+UInt8 String Array(UInt8)
+42 Hello, World! [1,2,3]
+$$)
+```
+```response
+┌─name─┬─type─────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ num │ UInt8 │ │ │ │ │ │
+│ str │ String │ │ │ │ │ │
+│ arr │ Array(UInt8) │ │ │ │ │ │
+└──────┴──────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## JSON formats with metadata {#json-with-metadata}
+
+Some JSON input formats ([JSON](formats.md#json), [JSONCompact](formats.md#json-compact), [JSONColumnsWithMetadata](formats.md#jsoncolumnswithmetadata)) contain metadata with column names and types.
+In schema inference for such formats, ClickHouse reads this metadata.
+
+**Example**
+```sql
+DESC format(JSON, $$
+{
+ "meta":
+ [
+ {
+ "name": "num",
+ "type": "UInt8"
+ },
+ {
+ "name": "str",
+ "type": "String"
+ },
+ {
+ "name": "arr",
+ "type": "Array(UInt8)"
+ }
+ ],
+
+ "data":
+ [
+ {
+ "num": 42,
+ "str": "Hello, World",
+ "arr": [1,2,3]
+ }
+ ],
+
+ "rows": 1,
+
+ "statistics":
+ {
+ "elapsed": 0.005723915,
+ "rows_read": 1,
+ "bytes_read": 1
+ }
+}
+$$)
+```
+```response
+┌─name─┬─type─────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ num │ UInt8 │ │ │ │ │ │
+│ str │ String │ │ │ │ │ │
+│ arr │ Array(UInt8) │ │ │ │ │ │
+└──────┴──────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## Avro {#avro}
+
+In Avro format ClickHouse reads its schema from the data and converts it to ClickHouse schema using the following type matches:
+
+| Avro data type | ClickHouse data type |
+|------------------------------------|--------------------------------------------------------------------------------|
+| `boolean` | [Bool](../sql-reference/data-types/boolean.md) |
+| `int` | [Int32](../sql-reference/data-types/int-uint.md) |
+| `long` | [Int64](../sql-reference/data-types/int-uint.md) |
+| `float` | [Float32](../sql-reference/data-types/float.md) |
+| `double` | [Float64](../sql-reference/data-types/float.md) |
+| `bytes`, `string` | [String](../sql-reference/data-types/string.md) |
+| `fixed` | [FixedString(N)](../sql-reference/data-types/fixedstring.md) |
+| `enum` | [Enum](../sql-reference/data-types/enum.md) |
+| `array(T)` | [Array(T)](../sql-reference/data-types/array.md) |
+| `union(null, T)`, `union(T, null)` | [Nullable(T)](../sql-reference/data-types/date.md) |
+| `null` | [Nullable(Nothing)](../sql-reference/data-types/special-data-types/nothing.md) |
+
+Other Avro types are not supported.
+
+## Parquet {#parquet}
+
+In Parquet format ClickHouse reads its schema from the data and converts it to ClickHouse schema using the following type matches:
+
+| Parquet data type | ClickHouse data type |
+|------------------------------|---------------------------------------------------------|
+| `BOOL` | [Bool](../sql-reference/data-types/boolean.md) |
+| `UINT8` | [UInt8](../sql-reference/data-types/int-uint.md) |
+| `INT8` | [Int8](../sql-reference/data-types/int-uint.md) |
+| `UINT16` | [UInt16](../sql-reference/data-types/int-uint.md) |
+| `INT16` | [Int16](../sql-reference/data-types/int-uint.md) |
+| `UINT32` | [UInt32](../sql-reference/data-types/int-uint.md) |
+| `INT32` | [Int32](../sql-reference/data-types/int-uint.md) |
+| `UINT64` | [UInt64](../sql-reference/data-types/int-uint.md) |
+| `INT64` | [Int64](../sql-reference/data-types/int-uint.md) |
+| `FLOAT` | [Float32](../sql-reference/data-types/float.md) |
+| `DOUBLE` | [Float64](../sql-reference/data-types/float.md) |
+| `DATE` | [Date32](../sql-reference/data-types/date32.md) |
+| `TIME (ms)` | [DateTime](../sql-reference/data-types/datetime.md) |
+| `TIMESTAMP`, `TIME (us, ns)` | [DateTime64](../sql-reference/data-types/datetime64.md) |
+| `STRING`, `BINARY` | [String](../sql-reference/data-types/string.md) |
+| `DECIMAL` | [Decimal](../sql-reference/data-types/decimal.md) |
+| `LIST` | [Array](../sql-reference/data-types/array.md) |
+| `STRUCT` | [Tuple](../sql-reference/data-types/tuple.md) |
+| `MAP` | [Map](../sql-reference/data-types/map.md) |
+
+Other Parquet types are not supported. By default, all inferred types are inside `Nullable`, but it can be changed using the setting `schema_inference_make_columns_nullable`.
+
+## Arrow {#arrow}
+
+In Arrow format ClickHouse reads its schema from the data and converts it to ClickHouse schema using the following type matches:
+
+| Arrow data type | ClickHouse data type |
+|---------------------------------|---------------------------------------------------------|
+| `BOOL` | [Bool](../sql-reference/data-types/boolean.md) |
+| `UINT8` | [UInt8](../sql-reference/data-types/int-uint.md) |
+| `INT8` | [Int8](../sql-reference/data-types/int-uint.md) |
+| `UINT16` | [UInt16](../sql-reference/data-types/int-uint.md) |
+| `INT16` | [Int16](../sql-reference/data-types/int-uint.md) |
+| `UINT32` | [UInt32](../sql-reference/data-types/int-uint.md) |
+| `INT32` | [Int32](../sql-reference/data-types/int-uint.md) |
+| `UINT64` | [UInt64](../sql-reference/data-types/int-uint.md) |
+| `INT64` | [Int64](../sql-reference/data-types/int-uint.md) |
+| `FLOAT`, `HALF_FLOAT` | [Float32](../sql-reference/data-types/float.md) |
+| `DOUBLE` | [Float64](../sql-reference/data-types/float.md) |
+| `DATE32` | [Date32](../sql-reference/data-types/date32.md) |
+| `DATE64` | [DateTime](../sql-reference/data-types/datetime.md) |
+| `TIMESTAMP`, `TIME32`, `TIME64` | [DateTime64](../sql-reference/data-types/datetime64.md) |
+| `STRING`, `BINARY` | [String](../sql-reference/data-types/string.md) |
+| `DECIMAL128`, `DECIMAL256` | [Decimal](../sql-reference/data-types/decimal.md) |
+| `LIST` | [Array](../sql-reference/data-types/array.md) |
+| `STRUCT` | [Tuple](../sql-reference/data-types/tuple.md) |
+| `MAP` | [Map](../sql-reference/data-types/map.md) |
+
+Other Arrow types are not supported. By default, all inferred types are inside `Nullable`, but it can be changed using the setting `schema_inference_make_columns_nullable`.
+
+## ORC {#orc}
+
+In ORC format ClickHouse reads its schema from the data and converts it to ClickHouse schema using the following type matches:
+
+| ORC data type | ClickHouse data type |
+|--------------------------------------|---------------------------------------------------------|
+| `Boolean` | [Bool](../sql-reference/data-types/boolean.md) |
+| `Tinyint` | [Int8](../sql-reference/data-types/int-uint.md) |
+| `Smallint` | [Int16](../sql-reference/data-types/int-uint.md) |
+| `Int` | [Int32](../sql-reference/data-types/int-uint.md) |
+| `Bigint` | [Int64](../sql-reference/data-types/int-uint.md) |
+| `Float` | [Float32](../sql-reference/data-types/float.md) |
+| `Double` | [Float64](../sql-reference/data-types/float.md) |
+| `Date` | [Date32](../sql-reference/data-types/date32.md) |
+| `Timestamp` | [DateTime64](../sql-reference/data-types/datetime64.md) |
+| `String`, `Char`, `Varchar`,`BINARY` | [String](../sql-reference/data-types/string.md) |
+| `Decimal` | [Decimal](../sql-reference/data-types/decimal.md) |
+| `List` | [Array](../sql-reference/data-types/array.md) |
+| `Struct` | [Tuple](../sql-reference/data-types/tuple.md) |
+| `Map` | [Map](../sql-reference/data-types/map.md) |
+
+Other ORC types are not supported. By default, all inferred types are inside `Nullable`, but it can be changed using the setting `schema_inference_make_columns_nullable`.
+
+## Native {#native}
+
+Native format is used inside ClickHouse and contains the schema in the data.
+In schema inference, ClickHouse reads the schema from the data without any transformations.
+
+# Formats with external schema {#formats-with-external-schema}
+
+Such formats require a schema describing the data in a separate file in a specific schema language.
+To automatically infer a schema from files in such formats, ClickHouse reads external schema from a separate file and transforms it to a ClickHouse table schema.
+
+# Protobuf {#protobuf}
+
+In schema inference for Protobuf format ClickHouse uses the following type matches:
+
+| Protobuf data type | ClickHouse data type |
+|-------------------------------|---------------------------------------------------|
+| `bool` | [UInt8](../sql-reference/data-types/int-uint.md) |
+| `float` | [Float32](../sql-reference/data-types/float.md) |
+| `double` | [Float64](../sql-reference/data-types/float.md) |
+| `int32`, `sint32`, `sfixed32` | [Int32](../sql-reference/data-types/int-uint.md) |
+| `int64`, `sint64`, `sfixed64` | [Int64](../sql-reference/data-types/int-uint.md) |
+| `uint32`, `fixed32` | [UInt32](../sql-reference/data-types/int-uint.md) |
+| `uint64`, `fixed64` | [UInt64](../sql-reference/data-types/int-uint.md) |
+| `string`, `bytes` | [String](../sql-reference/data-types/string.md) |
+| `enum` | [Enum](../sql-reference/data-types/enum.md) |
+| `repeated T` | [Array(T)](../sql-reference/data-types/array.md) |
+| `message`, `group` | [Tuple](../sql-reference/data-types/tuple.md) |
+
+# CapnProto {#capnproto}
+
+In schema inference for CapnProto format ClickHouse uses the following type matches:
+
+| CapnProto data type | ClickHouse data type |
+|------------------------------------|--------------------------------------------------------|
+| `Bool` | [UInt8](../sql-reference/data-types/int-uint.md) |
+| `Int8` | [Int8](../sql-reference/data-types/int-uint.md) |
+| `UInt8` | [UInt8](../sql-reference/data-types/int-uint.md) |
+| `Int16` | [Int16](../sql-reference/data-types/int-uint.md) |
+| `UInt16` | [UInt16](../sql-reference/data-types/int-uint.md) |
+| `Int32` | [Int32](../sql-reference/data-types/int-uint.md) |
+| `UInt32` | [UInt32](../sql-reference/data-types/int-uint.md) |
+| `Int64` | [Int64](../sql-reference/data-types/int-uint.md) |
+| `UInt64` | [UInt64](../sql-reference/data-types/int-uint.md) |
+| `Float32` | [Float32](../sql-reference/data-types/float.md) |
+| `Float64` | [Float64](../sql-reference/data-types/float.md) |
+| `Text`, `Data` | [String](../sql-reference/data-types/string.md) |
+| `enum` | [Enum](../sql-reference/data-types/enum.md) |
+| `List` | [Array](../sql-reference/data-types/array.md) |
+| `struct` | [Tuple](../sql-reference/data-types/tuple.md) |
+| `union(T, Void)`, `union(Void, T)` | [Nullable(T)](../sql-reference/data-types/nullable.md) |
+
+# Strong-typed binary formats {#strong-typed-binary-formats}
+
+In such formats, each serialized value contains information about its type (and possibly about its name), but there is no information about the whole table.
+In schema inference for such formats, ClickHouse reads data row by row (up to `input_format_max_rows_to_read_for_schema_inference` rows) and extracts
+the type (and possibly name) for each value from the data and then converts these types to ClickHouse types.
+
+## MsgPack {msgpack}
+
+In MsgPack format there is no delimiter between rows, to use schema inference for this format you should specify the number of columns in the table
+using the setting `input_format_msgpack_number_of_columns`. ClickHouse uses the following type matches:
+
+| MessagePack data type (`INSERT`) | ClickHouse data type |
+|--------------------------------------------------------------------|-----------------------------------------------------------|
+| `int N`, `uint N`, `negative fixint`, `positive fixint` | [Int64](../sql-reference/data-types/int-uint.md) |
+| `bool` | [UInt8](../sql-reference/data-types/int-uint.md) |
+| `fixstr`, `str 8`, `str 16`, `str 32`, `bin 8`, `bin 16`, `bin 32` | [String](../sql-reference/data-types/string.md) |
+| `float 32` | [Float32](../sql-reference/data-types/float.md) |
+| `float 64` | [Float64](../sql-reference/data-types/float.md) |
+| `uint 16` | [Date](../sql-reference/data-types/date.md) |
+| `uint 32` | [DateTime](../sql-reference/data-types/datetime.md) |
+| `uint 64` | [DateTime64](../sql-reference/data-types/datetime.md) |
+| `fixarray`, `array 16`, `array 32` | [Array](../sql-reference/data-types/array.md) |
+| `fixmap`, `map 16`, `map 32` | [Map](../sql-reference/data-types/map.md) |
+
+By default, all inferred types are inside `Nullable`, but it can be changed using the setting `schema_inference_make_columns_nullable`.
+
+## BSONEachRow {#bsoneachrow}
+
+In BSONEachRow each row of data is presented as a BSON document. In schema inference ClickHouse reads BSON documents one by one and extracts
+values, names, and types from the data and then transforms these types to ClickHouse types using the following type matches:
+
+| BSON Type | ClickHouse type |
+|-----------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------------------------|
+| `\x08` boolean | [Bool](../sql-reference/data-types/boolean.md) |
+| `\x10` int32 | [Int32](../sql-reference/data-types/int-uint.md) |
+| `\x12` int64 | [Int64](../sql-reference/data-types/int-uint.md) |
+| `\x01` double | [Float64](../sql-reference/data-types/float.md) |
+| `\x09` datetime | [DateTime64](../sql-reference/data-types/datetime64.md) |
+| `\x05` binary with`\x00` binary subtype, `\x02` string, `\x0E` symbol, `\x0D` JavaScript code | [String](../sql-reference/data-types/string.md) |
+| `\x07` ObjectId, | [FixedString(12)](../sql-reference/data-types/fixedstring.md) |
+| `\x05` binary with `\x04` uuid subtype, size = 16 | [UUID](../sql-reference/data-types/uuid.md) |
+| `\x04` array | [Array](../sql-reference/data-types/array.md)/[Tuple](../sql-reference/data-types/tuple.md) (if nested types are different) |
+| `\x03` document | [Named Tuple](../sql-reference/data-types/tuple.md)/[Map](../sql-reference/data-types/map.md) (with String keys) |
+
+By default, all inferred types are inside `Nullable`, but it can be changed using the setting `schema_inference_make_columns_nullable`.
+
+# Formats with constant schema {#formats-with-constant-schema}
+
+Data in such formats always have the same schema.
+
+## LineAsString {#line-as-string}
+
+In this format, ClickHouse reads the whole line from the data into a single column with `String` data type. The inferred type for this format is always `String` and the column name is `line`.
+
+**Example**
+
+```sql
+DESC format(LineAsString, 'Hello\nworld!')
+```
+```response
+┌─name─┬─type───┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ line │ String │ │ │ │ │ │
+└──────┴────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## JSONAsString {#json-as-string}
+
+In this format, ClickHouse reads the whole JSON object from the data into a single column with `String` data type. The inferred type for this format is always `String` and the column name is `json`.
+
+**Example**
+
+```sql
+DESC format(JSONAsString, '{"x" : 42, "y" : "Hello, World!"}')
+```
+```response
+┌─name─┬─type───┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ json │ String │ │ │ │ │ │
+└──────┴────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+## JSONAsObject {#json-as-object}
+
+In this format, ClickHouse reads the whole JSON object from the data into a single column with `Object('json')` data type. Inferred type for this format is always `String` and the column name is `json`.
+
+Note: This format works only if `allow_experimental_object_type` is enabled.
+
+**Example**
+
+```sql
+DESC format(JSONAsString, '{"x" : 42, "y" : "Hello, World!"}') SETTINGS allow_experimental_object_type=1
+```
+```response
+┌─name─┬─type───────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ json │ Object('json') │ │ │ │ │ │
+└──────┴────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+[Original article](https://clickhouse.com/docs/en/interfaces/schema-inference)
diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md
index 061d95c1152..4feb434d762 100644
--- a/docs/en/operations/backup.md
+++ b/docs/en/operations/backup.md
@@ -9,6 +9,29 @@ slug: /en/operations/backup
- [Backup/restore using an S3 disk](#backuprestore-using-an-s3-disk)
- [Alternatives](#alternatives)
+## Command summary
+
+```bash
+ BACKUP|RESTORE
+ TABLE [db.]table_name [AS [db.]table_name_in_backup]
+ [PARTITION[S] partition_expr [,...]] |
+ DICTIONARY [db.]dictionary_name [AS [db.]name_in_backup] |
+ DATABASE database_name [AS database_name_in_backup]
+ [EXCEPT TABLES ...] |
+ TEMPORARY TABLE table_name [AS table_name_in_backup] |
+ VIEW view_name [AS view_name_in_backup]
+ ALL TEMPORARY TABLES [EXCEPT ...] |
+ ALL DATABASES [EXCEPT ...] } [,...]
+ [ON CLUSTER 'cluster_name']
+ TO|FROM File('/') | Disk('', '/') | S3('/', '', '')
+ [SETTINGS base_backup = File('/') | Disk(...) | S3('/', '', '')]
+
+```
+
+:::note ALL
+`ALL` is only applicable to the `RESTORE` command.
+:::
+
## Background
While [replication](../engines/table-engines/mergetree-family/replication.md) provides protection from hardware failures, it does not protect against human errors: accidental deletion of data, deletion of the wrong table or a table on the wrong cluster, and software bugs that result in incorrect data processing or data corruption. In many cases mistakes like these will affect all replicas. ClickHouse has built-in safeguards to prevent some types of mistakes — for example, by default [you can’t just drop tables with a MergeTree-like engine containing more than 50 Gb of data](server-configuration-parameters/settings.md#max-table-size-to-drop). However, these safeguards do not cover all possible cases and can be circumvented.
diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md
index 338ecf9ffd3..9f99570fb22 100644
--- a/docs/en/operations/settings/merge-tree-settings.md
+++ b/docs/en/operations/settings/merge-tree-settings.md
@@ -127,6 +127,13 @@ Default value: 100000.
A large number of parts in a table reduces performance of ClickHouse queries and increases ClickHouse boot time. Most often this is a consequence of an incorrect design (mistakes when choosing a partitioning strategy - too small partitions).
+## simultaneous_parts_removal_limit {#simultaneous-parts-removal-limit}
+
+If there are a lot of outdated parts cleanup thread will try to delete up to `simultaneous_parts_removal_limit` parts during one iteration.
+`simultaneous_parts_removal_limit` set to `0` means unlimited.
+
+Default value: 0.
+
## replicated_deduplication_window {#replicated-deduplication-window}
The number of most recently inserted blocks for which ClickHouse Keeper stores hash sums to check for duplicates.
diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md
index 35d6f47852a..895e071a560 100644
--- a/docs/en/operations/settings/settings.md
+++ b/docs/en/operations/settings/settings.md
@@ -1011,6 +1011,12 @@ The default value is 7500.
The smaller the value, the more often data is flushed into the table. Setting the value too low leads to poor performance.
+## stream_poll_timeout_ms {#stream_poll_timeout_ms}
+
+Timeout for polling data from/to streaming storages.
+
+Default value: 500.
+
## load_balancing {#settings-load_balancing}
Specifies the algorithm of replicas selection that is used for distributed query processing.
@@ -3625,7 +3631,7 @@ z IPv4
Controls making inferred types `Nullable` in schema inference for formats without information about nullability.
If the setting is enabled, the inferred type will be `Nullable` only if column contains `NULL` in a sample that is parsed during schema inference.
-Default value: `false`.
+Default value: `true`.
## input_format_try_infer_integers {#input_format_try_infer_integers}
diff --git a/docs/en/operations/system-tables/schema_inference_cache.md b/docs/en/operations/system-tables/schema_inference_cache.md
new file mode 100644
index 00000000000..31b0671dc34
--- /dev/null
+++ b/docs/en/operations/system-tables/schema_inference_cache.md
@@ -0,0 +1,70 @@
+---
+slug: /en/operations/system-tables/schema_inference_cache
+---
+# Schema inference cache
+
+Contains information about all cached file schemas.
+
+Columns:
+- `storage` ([String](/docs/en/sql-reference/data-types/string.md)) — Storage name: File, URL, S3 or HDFS.
+- `source` ([String](/docs/en/sql-reference/data-types/string.md)) — File source.
+- `format` ([String](/docs/en/sql-reference/data-types/string.md)) — Format name.
+- `additional_format_info` ([String](/docs/en/sql-reference/data-types/string.md)) - Additional information required to identify the schema. For example, format specific settings.
+- `registration_time` ([DateTime](/docs/en/sql-reference/data-types/datetime.md)) — Timestamp when schema was added in cache.
+- `schema` ([String](/docs/en/sql-reference/data-types/string.md)) - Cached schema.
+
+**Example**
+
+Let's say we have a file `data.jsonl` with this content:
+```json
+{"id" : 1, "age" : 25, "name" : "Josh", "hobbies" : ["football", "cooking", "music"]}
+{"id" : 2, "age" : 19, "name" : "Alan", "hobbies" : ["tennis", "art"]}
+{"id" : 3, "age" : 32, "name" : "Lana", "hobbies" : ["fitness", "reading", "shopping"]}
+{"id" : 4, "age" : 47, "name" : "Brayan", "hobbies" : ["movies", "skydiving"]}
+```
+
+:::tip
+Place `data.jsonl` in the `user_files_path` directory. You can find this by looking
+in your ClickHouse configuration files. The default is:
+```
+/var/lib/clickhouse/user_files/
+```
+:::
+
+Open `clickhouse-client` and run the `DESCRIBE` query:
+
+```sql
+DESCRIBE file('data.jsonl') SETTINGS input_format_try_infer_integers=0;
+```
+
+```response
+┌─name────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ id │ Nullable(Float64) │ │ │ │ │ │
+│ age │ Nullable(Float64) │ │ │ │ │ │
+│ name │ Nullable(String) │ │ │ │ │ │
+│ hobbies │ Array(Nullable(String)) │ │ │ │ │ │
+└─────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
+
+Let's see the content of the `system.schema_inference_cache` table:
+
+```sql
+SELECT *
+FROM system.schema_inference_cache
+FORMAT Vertical
+```
+```response
+Row 1:
+──────
+storage: File
+source: /home/droscigno/user_files/data.jsonl
+format: JSONEachRow
+additional_format_info: schema_inference_hints=, max_rows_to_read_for_schema_inference=25000, schema_inference_make_columns_nullable=true, try_infer_integers=false, try_infer_dates=true, try_infer_datetimes=true, try_infer_numbers_from_strings=true, read_bools_as_numbers=true, try_infer_objects=false
+registration_time: 2022-12-29 17:49:52
+schema: id Nullable(Float64), age Nullable(Float64), name Nullable(String), hobbies Array(Nullable(String))
+```
+
+
+**See also**
+- [Automatic schema inference from input data](/docs/en/interfaces/schema-inference.md)
+
diff --git a/docs/en/sql-reference/aggregate-functions/reference/grouparraylast.md b/docs/en/sql-reference/aggregate-functions/reference/grouparraylast.md
new file mode 100644
index 00000000000..1b9681dc852
--- /dev/null
+++ b/docs/en/sql-reference/aggregate-functions/reference/grouparraylast.md
@@ -0,0 +1,41 @@
+---
+slug: /en/sql-reference/aggregate-functions/reference/grouparraylast
+sidebar_position: 110
+---
+
+# groupArrayLast
+
+Syntax: `groupArrayLast(max_size)(x)`
+
+Creates an array of last argument values.
+For example, `groupArrayLast(1)(x)` is equivalent to `[anyLast (x)]`.
+
+In some cases, you can still rely on the order of execution. This applies to cases when `SELECT` comes from a subquery that uses `ORDER BY`.
+
+**Example**
+
+Query:
+
+```sql
+select groupArrayLast(2)(number+1) numbers from numbers(10)
+```
+
+Result:
+
+```text
+┌─numbers─┐
+│ [9,10] │
+└─────────┘
+```
+
+In compare to `groupArray`:
+
+```sql
+select groupArray(2)(number+1) numbers from numbers(10)
+```
+
+```text
+┌─numbers─┐
+│ [1,2] │
+└─────────┘
+```
diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md
index ee17c37100c..40161145419 100644
--- a/docs/en/sql-reference/aggregate-functions/reference/index.md
+++ b/docs/en/sql-reference/aggregate-functions/reference/index.md
@@ -32,6 +32,7 @@ ClickHouse-specific aggregate functions:
- [topK](../../../sql-reference/aggregate-functions/reference/topk.md)
- [topKWeighted](../../../sql-reference/aggregate-functions/reference/topkweighted.md)
- [groupArray](../../../sql-reference/aggregate-functions/reference/grouparray.md)
+- [groupArrayLast](../../../sql-reference/aggregate-functions/reference/grouparraylast.md)
- [groupUniqArray](../../../sql-reference/aggregate-functions/reference/groupuniqarray.md)
- [groupArrayInsertAt](../../../sql-reference/aggregate-functions/reference/grouparrayinsertat.md)
- [groupArrayMovingAvg](../../../sql-reference/aggregate-functions/reference/grouparraymovingavg.md)
diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md
index c5d48945649..897945a6d9d 100644
--- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md
+++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md
@@ -825,6 +825,23 @@ Setting fields:
The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared.
:::
+## Null
+
+A special source that can be used to create dummy (empty) dictionaries. Such dictionaries can useful for tests or with setups with separated data and query nodes at nodes with Distributed tables.
+
+``` sql
+CREATE DICTIONARY null_dict (
+ id UInt64,
+ val UInt8,
+ default_val UInt8 DEFAULT 123,
+ nullable_val Nullable(UInt8)
+)
+PRIMARY KEY id
+SOURCE(NULL())
+LAYOUT(FLAT())
+LIFETIME(0);
+```
+
## Related Content
-- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse)
\ No newline at end of file
+- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse)
diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md
index be8e26daa87..89fa72de8bf 100644
--- a/docs/en/sql-reference/functions/date-time-functions.md
+++ b/docs/en/sql-reference/functions/date-time-functions.md
@@ -410,35 +410,35 @@ Converts a date with time to a certain fixed date, while preserving the time.
## toRelativeYearNum
-Converts a date with time or date to the number of the year, starting from a certain fixed point in the past.
+Converts a date or date with time to the number of the year, starting from a certain fixed point in the past.
## toRelativeQuarterNum
-Converts a date with time or date to the number of the quarter, starting from a certain fixed point in the past.
+Converts a date or date with time to the number of the quarter, starting from a certain fixed point in the past.
## toRelativeMonthNum
-Converts a date with time or date to the number of the month, starting from a certain fixed point in the past.
+Converts a date or date with time to the number of the month, starting from a certain fixed point in the past.
## toRelativeWeekNum
-Converts a date with time or date to the number of the week, starting from a certain fixed point in the past.
+Converts a date or date with time to the number of the week, starting from a certain fixed point in the past.
## toRelativeDayNum
-Converts a date with time or date to the number of the day, starting from a certain fixed point in the past.
+Converts a date or date with time to the number of the day, starting from a certain fixed point in the past.
## toRelativeHourNum
-Converts a date with time or date to the number of the hour, starting from a certain fixed point in the past.
+Converts a date or date with time to the number of the hour, starting from a certain fixed point in the past.
## toRelativeMinuteNum
-Converts a date with time or date to the number of the minute, starting from a certain fixed point in the past.
+Converts a date or date with time to the number of the minute, starting from a certain fixed point in the past.
## toRelativeSecondNum
-Converts a date with time or date to the number of the second, starting from a certain fixed point in the past.
+Converts a date or date with time to the number of the second, starting from a certain fixed point in the past.
## toISOYear
@@ -517,6 +517,154 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d
└────────────┴───────────┴───────────┴───────────┘
```
+## age
+
+Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 second.
+E.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for `day` unit, 0 months for `month` unit, 0 years for `year` unit.
+
+
+**Syntax**
+
+``` sql
+age('unit', startdate, enddate, [timezone])
+```
+
+**Arguments**
+
+- `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md).
+ Possible values:
+
+ - `second` (possible abbreviations: `ss`, `s`)
+ - `minute` (possible abbreviations: `mi`, `n`)
+ - `hour` (possible abbreviations: `hh`, `h`)
+ - `day` (possible abbreviations: `dd`, `d`)
+ - `week` (possible abbreviations: `wk`, `ww`)
+ - `month` (possible abbreviations: `mm`, `m`)
+ - `quarter` (possible abbreviations: `qq`, `q`)
+ - `year` (possible abbreviations: `yyyy`, `yy`)
+
+- `startdate` — The first time value to subtract (the subtrahend). [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
+
+- `enddate` — The second time value to subtract from (the minuend). [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
+
+- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). If specified, it is applied to both `startdate` and `enddate`. If not specified, timezones of `startdate` and `enddate` are used. If they are not the same, the result is unspecified. [String](../../sql-reference/data-types/string.md).
+
+**Returned value**
+
+Difference between `enddate` and `startdate` expressed in `unit`.
+
+Type: [Int](../../sql-reference/data-types/int-uint.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT age('hour', toDateTime('2018-01-01 22:30:00'), toDateTime('2018-01-02 23:00:00'));
+```
+
+Result:
+
+``` text
+┌─age('hour', toDateTime('2018-01-01 22:30:00'), toDateTime('2018-01-02 23:00:00'))─┐
+│ 24 │
+└───────────────────────────────────────────────────────────────────────────────────┘
+```
+
+Query:
+
+``` sql
+SELECT
+ toDate('2022-01-01') AS e,
+ toDate('2021-12-29') AS s,
+ age('day', s, e) AS day_age,
+ age('month', s, e) AS month__age,
+ age('year', s, e) AS year_age;
+```
+
+Result:
+
+``` text
+┌──────────e─┬──────────s─┬─day_age─┬─month__age─┬─year_age─┐
+│ 2022-01-01 │ 2021-12-29 │ 3 │ 0 │ 0 │
+└────────────┴────────────┴─────────┴────────────┴──────────┘
+```
+
+
+## date\_diff
+
+Returns the count of the specified `unit` boundaries crossed between the `startdate` and `enddate`.
+The difference is calculated using relative units, e.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for day unit (see [toRelativeDayNum](#torelativedaynum)), 1 month for month unit (see [toRelativeMonthNum](#torelativemonthnum)), 1 year for year unit (see [toRelativeYearNum](#torelativeyearnum)).
+
+**Syntax**
+
+``` sql
+date_diff('unit', startdate, enddate, [timezone])
+```
+
+Aliases: `dateDiff`, `DATE_DIFF`.
+
+**Arguments**
+
+- `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md).
+ Possible values:
+
+ - `second` (possible abbreviations: `ss`, `s`)
+ - `minute` (possible abbreviations: `mi`, `n`)
+ - `hour` (possible abbreviations: `hh`, `h`)
+ - `day` (possible abbreviations: `dd`, `d`)
+ - `week` (possible abbreviations: `wk`, `ww`)
+ - `month` (possible abbreviations: `mm`, `m`)
+ - `quarter` (possible abbreviations: `qq`, `q`)
+ - `year` (possible abbreviations: `yyyy`, `yy`)
+
+- `startdate` — The first time value to subtract (the subtrahend). [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
+
+- `enddate` — The second time value to subtract from (the minuend). [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
+
+- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). If specified, it is applied to both `startdate` and `enddate`. If not specified, timezones of `startdate` and `enddate` are used. If they are not the same, the result is unspecified. [String](../../sql-reference/data-types/string.md).
+
+**Returned value**
+
+Difference between `enddate` and `startdate` expressed in `unit`.
+
+Type: [Int](../../sql-reference/data-types/int-uint.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'));
+```
+
+Result:
+
+``` text
+┌─dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'))─┐
+│ 25 │
+└────────────────────────────────────────────────────────────────────────────────────────┘
+```
+
+Query:
+
+``` sql
+SELECT
+ toDate('2022-01-01') AS e,
+ toDate('2021-12-29') AS s,
+ dateDiff('day', s, e) AS day_diff,
+ dateDiff('month', s, e) AS month__diff,
+ dateDiff('year', s, e) AS year_diff;
+```
+
+Result:
+
+``` text
+┌──────────e─┬──────────s─┬─day_diff─┬─month__diff─┬─year_diff─┐
+│ 2022-01-01 │ 2021-12-29 │ 3 │ 1 │ 1 │
+└────────────┴────────────┴──────────┴─────────────┴───────────┘
+```
+
## date\_trunc
Truncates date and time data to the specified part of date.
@@ -637,80 +785,6 @@ Result:
└───────────────────────────────────────────────┘
```
-## date\_diff
-
-Returns the difference between two dates or dates with time values.
-The difference is calculated using relative units, e.g. the difference between `2022-01-01` and `2021-12-29` is 3 days for day unit (see [toRelativeDayNum](#torelativedaynum)), 1 month for month unit (see [toRelativeMonthNum](#torelativemonthnum)), 1 year for year unit (see [toRelativeYearNum](#torelativeyearnum)).
-
-**Syntax**
-
-``` sql
-date_diff('unit', startdate, enddate, [timezone])
-```
-
-Aliases: `dateDiff`, `DATE_DIFF`.
-
-**Arguments**
-
-- `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md).
- Possible values:
-
- - `second`
- - `minute`
- - `hour`
- - `day`
- - `week`
- - `month`
- - `quarter`
- - `year`
-
-- `startdate` — The first time value to subtract (the subtrahend). [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
-
-- `enddate` — The second time value to subtract from (the minuend). [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
-
-- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). If specified, it is applied to both `startdate` and `enddate`. If not specified, timezones of `startdate` and `enddate` are used. If they are not the same, the result is unspecified. [String](../../sql-reference/data-types/string.md).
-
-**Returned value**
-
-Difference between `enddate` and `startdate` expressed in `unit`.
-
-Type: [Int](../../sql-reference/data-types/int-uint.md).
-
-**Example**
-
-Query:
-
-``` sql
-SELECT dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'));
-```
-
-Result:
-
-``` text
-┌─dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'))─┐
-│ 25 │
-└────────────────────────────────────────────────────────────────────────────────────────┘
-```
-
-Query:
-
-``` sql
-SELECT
- toDate('2022-01-01') AS e,
- toDate('2021-12-29') AS s,
- dateDiff('day', s, e) AS day_diff,
- dateDiff('month', s, e) AS month__diff,
- dateDiff('year', s, e) AS year_diff;
-```
-
-Result:
-
-``` text
-┌──────────e─┬──────────s─┬─day_diff─┬─month__diff─┬─year_diff─┐
-│ 2022-01-01 │ 2021-12-29 │ 3 │ 1 │ 1 │
-└────────────┴────────────┴──────────┴─────────────┴───────────┘
-```
-
## date\_sub
Subtracts the time interval or date interval from the provided date or date with time.
@@ -1085,6 +1159,8 @@ SELECT timeSlots(toDateTime64('1980-12-12 21:01:02.1234', 4, 'UTC'), toDecimal64
Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column.
+formatDateTime uses MySQL datetime format style, refer to https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-format.
+
**Syntax**
``` sql
@@ -1158,6 +1234,64 @@ Result:
└─────────────────────────────────────────────────────────────────────┘
```
+**See Also**
+
+- [formatDateTimeInJodaSyntax](##formatDateTimeInJodaSyntax)
+
+
+## formatDateTimeInJodaSyntax
+
+Similar to formatDateTime, except that it formats datetime in Joda style instead of MySQL style. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html.
+
+
+**Replacement fields**
+
+Using replacement fields, you can define a pattern for the resulting string.
+
+
+| Placeholder | Description | Presentation | Examples |
+| ----------- | ----------- | ------------- | -------- |
+| G | era | text | AD |
+| C | century of era (>=0) | number | 20 |
+| Y | year of era (>=0) | year | 1996 |
+| x | weekyear(not supported yet) | year | 1996 |
+| w | week of weekyear(not supported yet) | number | 27 |
+| e | day of week | number | 2 |
+| E | day of week | text | Tuesday; Tue |
+| y | year | year | 1996 |
+| D | day of year | number | 189 |
+| M | month of year | month | July; Jul; 07 |
+| d | day of month | number | 10 |
+| a | halfday of day | text | PM |
+| K | hour of halfday (0~11) | number | 0 |
+| h | clockhour of halfday (1~12) | number | 12 |
+| H | hour of day (0~23) | number | 0 |
+| k | clockhour of day (1~24) | number | 24 |
+| m | minute of hour | number | 30 |
+| s | second of minute | number | 55 |
+| S | fraction of second(not supported yet) | number | 978 |
+| z | time zone(short name not supported yet) | text | Pacific Standard Time; PST |
+| Z | time zone offset/id(not supported yet) | zone | -0800; -08:00; America/Los_Angeles |
+| ' | escape for text | delimiter| |
+| '' | single quote | literal | ' |
+
+**Example**
+
+Query:
+
+``` sql
+SELECT formatDateTimeInJodaSyntax(toDateTime('2010-01-04 12:34:56'), 'yyyy-MM-dd HH:mm:ss')
+```
+
+Result:
+
+```
+┌─formatDateTimeInJodaSyntax(toDateTime('2010-01-04 12:34:56'), 'yyyy-MM-dd HH:mm:ss')─┐
+│ 2010-01-04 12:34:56 │
+└─────────────────────────────────────────────────────────────────────────────────────────┘
+```
+
+
## dateName
Returns specified part of date.
@@ -1241,6 +1375,8 @@ Result:
Function converts Unix timestamp to a calendar date and a time of a day. When there is only a single argument of [Integer](../../sql-reference/data-types/int-uint.md) type, it acts in the same way as [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime) and return [DateTime](../../sql-reference/data-types/datetime.md) type.
+FROM_UNIXTIME uses MySQL datetime format style, refer to https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-format.
+
Alias: `fromUnixTimestamp`.
**Example:**
@@ -1273,6 +1409,28 @@ SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime;
└─────────────────────┘
```
+**See Also**
+
+- [fromUnixTimestampInJodaSyntax](##fromUnixTimestampInJodaSyntax)
+
+
+## fromUnixTimestampInJodaSyntax
+Similar to FROM_UNIXTIME, except that it formats time in Joda style instead of MySQL style. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html.
+
+**Example:**
+Query:
+``` sql
+SELECT fromUnixTimestampInJodaSyntax(1669804872, 'yyyy-MM-dd HH:mm:ss', 'UTC');
+```
+
+Result:
+```
+┌─fromUnixTimestampInJodaSyntax(1669804872, 'yyyy-MM-dd HH:mm:ss', 'UTC')─┐
+│ 2022-11-30 10:41:12 │
+└────────────────────────────────────────────────────────────────────────────┘
+```
+
+
## toModifiedJulianDay
Converts a [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar) date in text form `YYYY-MM-DD` to a [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Variants) number in Int32. This function supports date from `0000-01-01` to `9999-12-31`. It raises an exception if the argument cannot be parsed as a date, or the date is invalid.
diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md
index 4efa2131eb6..f0c0d3e4802 100644
--- a/docs/en/sql-reference/functions/random-functions.md
+++ b/docs/en/sql-reference/functions/random-functions.md
@@ -68,6 +68,483 @@ Result:
└────────────┴────────────┴──────────────┴────────────────┴─────────────────┴──────────────────────┘
```
+# Functions for Generating Random Numbers based on Distributions
+
+:::note
+These functions are available starting from 22.10.
+:::
+
+
+
+## randUniform
+
+Return random number based on [continuous uniform distribution](https://en.wikipedia.org/wiki/Continuous_uniform_distribution) in a specified range from `min` to `max`.
+
+**Syntax**
+
+``` sql
+randUniform(min, max)
+```
+
+**Arguments**
+
+- `min` - `Float64` - min value of the range,
+- `max` - `Float64` - max value of the range.
+
+**Returned value**
+
+- Pseudo-random number.
+
+Type: [Float64](/docs/en/sql-reference/data-types/float.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT randUniform(5.5, 10) FROM numbers(5)
+```
+
+Result:
+
+``` text
+┌─randUniform(5.5, 10)─┐
+│ 8.094978491443102 │
+│ 7.3181248914450885 │
+│ 7.177741903868262 │
+│ 6.483347380953762 │
+│ 6.122286382885112 │
+└──────────────────────┘
+```
+
+
+
+## randNormal
+
+Return random number based on [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution).
+
+**Syntax**
+
+``` sql
+randNormal(meam, variance)
+```
+
+**Arguments**
+
+- `meam` - `Float64` mean value of distribution,
+- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance).
+
+**Returned value**
+
+- Pseudo-random number.
+
+Type: [Float64](/docs/en/sql-reference/data-types/float.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT randNormal(10, 2) FROM numbers(5)
+```
+
+Result:
+
+``` text
+┌──randNormal(10, 2)─┐
+│ 13.389228911709653 │
+│ 8.622949707401295 │
+│ 10.801887062682981 │
+│ 4.5220192605895315 │
+│ 10.901239123982567 │
+└────────────────────┘
+```
+
+
+
+## randLogNormal
+
+Return random number based on [log-normal distribution](https://en.wikipedia.org/wiki/Log-normal_distribution).
+
+**Syntax**
+
+``` sql
+randLogNormal(meam, variance)
+```
+
+**Arguments**
+
+- `meam` - `Float64` mean value of distribution,
+- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance).
+
+**Returned value**
+
+- Pseudo-random number.
+
+Type: [Float64](/docs/en/sql-reference/data-types/float.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT randLogNormal(100, 5) FROM numbers(5)
+```
+
+Result:
+
+``` text
+┌─randLogNormal(100, 5)─┐
+│ 1.295699673937363e48 │
+│ 9.719869109186684e39 │
+│ 6.110868203189557e42 │
+│ 9.912675872925529e39 │
+│ 2.3564708490552458e42 │
+└───────────────────────┘
+```
+
+
+
+## randBinomial
+
+Return random number based on [binomial distribution](https://en.wikipedia.org/wiki/Binomial_distribution).
+
+**Syntax**
+
+``` sql
+randBinomial(experiments, probability)
+```
+
+**Arguments**
+
+- `experiments` - `UInt64` number of experiments,
+- `probability` - `Float64` - probability of success in each experiment (values in `0...1` range only).
+
+**Returned value**
+
+- Pseudo-random number.
+
+Type: [UInt64](/docs/en/sql-reference/data-types/int-uint.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT randBinomial(100, .75) FROM numbers(5)
+```
+
+Result:
+
+``` text
+┌─randBinomial(100, 0.75)─┐
+│ 74 │
+│ 78 │
+│ 76 │
+│ 77 │
+│ 80 │
+└─────────────────────────┘
+```
+
+
+
+## randNegativeBinomial
+
+Return random number based on [negative binomial distribution](https://en.wikipedia.org/wiki/Negative_binomial_distribution).
+
+**Syntax**
+
+``` sql
+randNegativeBinomial(experiments, probability)
+```
+
+**Arguments**
+
+- `experiments` - `UInt64` number of experiments,
+- `probability` - `Float64` - probability of failure in each experiment (values in `0...1` range only).
+
+**Returned value**
+
+- Pseudo-random number.
+
+Type: [UInt64](/docs/en/sql-reference/data-types/int-uint.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT randNegativeBinomial(100, .75) FROM numbers(5)
+```
+
+Result:
+
+``` text
+┌─randNegativeBinomial(100, 0.75)─┐
+│ 33 │
+│ 32 │
+│ 39 │
+│ 40 │
+│ 50 │
+└─────────────────────────────────┘
+```
+
+
+
+## randPoisson
+
+Return random number based on [Poisson distribution](https://en.wikipedia.org/wiki/Poisson_distribution).
+
+**Syntax**
+
+``` sql
+randPoisson(n)
+```
+
+**Arguments**
+
+- `n` - `UInt64` mean number of occurrences.
+
+**Returned value**
+
+- Pseudo-random number.
+
+Type: [UInt64](/docs/en/sql-reference/data-types/int-uint.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT randPoisson(10) FROM numbers(5)
+```
+
+Result:
+
+``` text
+┌─randPoisson(10)─┐
+│ 8 │
+│ 8 │
+│ 7 │
+│ 10 │
+│ 6 │
+└─────────────────┘
+```
+
+
+
+## randBernoulli
+
+Return random number based on [Bernoulli distribution](https://en.wikipedia.org/wiki/Bernoulli_distribution).
+
+**Syntax**
+
+``` sql
+randBernoulli(probability)
+```
+
+**Arguments**
+
+- `probability` - `Float64` - probability of success (values in `0...1` range only).
+
+**Returned value**
+
+- Pseudo-random number.
+
+Type: [UInt64](/docs/en/sql-reference/data-types/int-uint.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT randBernoulli(.75) FROM numbers(5)
+```
+
+Result:
+
+``` text
+┌─randBernoulli(0.75)─┐
+│ 1 │
+│ 1 │
+│ 0 │
+│ 1 │
+│ 1 │
+└─────────────────────┘
+```
+
+
+
+## randExponential
+
+Return random number based on [exponential distribution](https://en.wikipedia.org/wiki/Exponential_distribution).
+
+**Syntax**
+
+``` sql
+randExponential(lambda)
+```
+
+**Arguments**
+
+- `lambda` - `Float64` lambda value.
+
+**Returned value**
+
+- Pseudo-random number.
+
+Type: [Float64](/docs/en/sql-reference/data-types/float.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT randExponential(1/10) FROM numbers(5)
+```
+
+Result:
+
+``` text
+┌─randExponential(divide(1, 10))─┐
+│ 44.71628934340778 │
+│ 4.211013337903262 │
+│ 10.809402553207766 │
+│ 15.63959406553284 │
+│ 1.8148392319860158 │
+└────────────────────────────────┘
+```
+
+
+
+## randChiSquared
+
+Return random number based on [Chi-square distribution](https://en.wikipedia.org/wiki/Chi-squared_distribution) - a distribution of a sum of the squares of k independent standard normal random variables.
+
+**Syntax**
+
+``` sql
+randChiSquared(degree_of_freedom)
+```
+
+**Arguments**
+
+- `degree_of_freedom` - `Float64` degree of freedom.
+
+**Returned value**
+
+- Pseudo-random number.
+
+Type: [Float64](/docs/en/sql-reference/data-types/float.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT randChiSquared(10) FROM numbers(5)
+```
+
+Result:
+
+``` text
+┌─randChiSquared(10)─┐
+│ 10.015463656521543 │
+│ 9.621799919882768 │
+│ 2.71785015634699 │
+│ 11.128188665931908 │
+│ 4.902063104425469 │
+└────────────────────┘
+```
+
+
+
+## randStudentT
+
+Return random number based on [Student's t-distribution](https://en.wikipedia.org/wiki/Student%27s_t-distribution).
+
+**Syntax**
+
+``` sql
+randStudentT(degree_of_freedom)
+```
+
+**Arguments**
+
+- `degree_of_freedom` - `Float64` degree of freedom.
+
+**Returned value**
+
+- Pseudo-random number.
+
+Type: [Float64](/docs/en/sql-reference/data-types/float.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT randStudentT(10) FROM numbers(5)
+```
+
+Result:
+
+``` text
+┌─────randStudentT(10)─┐
+│ 1.2217309938538725 │
+│ 1.7941971681200541 │
+│ -0.28192176076784664 │
+│ 0.2508897721303792 │
+│ -2.7858432909761186 │
+└──────────────────────┘
+```
+
+
+
+## randFisherF
+
+Return random number based on [F-distribution](https://en.wikipedia.org/wiki/F-distribution).
+
+**Syntax**
+
+``` sql
+randFisherF(d1, d2)
+```
+
+**Arguments**
+
+- `d1` - `Float64` d1 degree of freedom in `X = (S1 / d1) / (S2 / d2)`,
+- `d2` - `Float64` d2 degree of freedom in `X = (S1 / d1) / (S2 / d2)`,
+
+**Returned value**
+
+- Pseudo-random number.
+
+Type: [Float64](/docs/en/sql-reference/data-types/float.md).
+
+**Example**
+
+Query:
+
+``` sql
+SELECT randFisherF(10, 3) FROM numbers(5)
+```
+
+Result:
+
+``` text
+┌──randFisherF(10, 3)─┐
+│ 7.286287504216609 │
+│ 0.26590779413050386 │
+│ 0.22207610901168987 │
+│ 0.7953362728449572 │
+│ 0.19278885985221572 │
+└─────────────────────┘
+```
+
+
+
+
# Random Functions for Working with Strings
## randomString
diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md
index d82728b9721..bae45de07e9 100644
--- a/docs/en/sql-reference/functions/type-conversion-functions.md
+++ b/docs/en/sql-reference/functions/type-conversion-functions.md
@@ -14,7 +14,7 @@ ClickHouse has the [same behavior as C++ programs](https://en.cppreference.com/w
## toInt(8\|16\|32\|64\|128\|256)
-Converts an input value to the [Int](../../sql-reference/data-types/int-uint.md) data type. This function family includes:
+Converts an input value to the [Int](/docs/en/sql-reference/data-types/int-uint.md) data type. This function family includes:
- `toInt8(expr)` — Results in the `Int8` data type.
- `toInt16(expr)` — Results in the `Int16` data type.
@@ -25,7 +25,7 @@ Converts an input value to the [Int](../../sql-reference/data-types/int-uint.md)
**Arguments**
-- `expr` — [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a number or a string with the decimal representation of a number. Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped.
+- `expr` — [Expression](/docs/en/sql-reference/syntax.md/#syntax-expressions) returning a number or a string with the decimal representation of a number. Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped.
**Returned value**
@@ -33,7 +33,7 @@ Integer value in the `Int8`, `Int16`, `Int32`, `Int64`, `Int128` or `Int256` dat
Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers.
-The behavior of functions for the [NaN and Inf](../../sql-reference/data-types/float.md#data_type-float-nan-inf) arguments is undefined. Remember about [numeric conversions issues](#numeric-conversion-issues), when using the functions.
+The behavior of functions for the [NaN and Inf](/docs/en/sql-reference/data-types/float.md/#data_type-float-nan-inf) arguments is undefined. Remember about [numeric conversions issues](#numeric-conversion-issues), when using the functions.
**Example**
@@ -114,7 +114,7 @@ Result:
## toUInt(8\|16\|32\|64\|256)
-Converts an input value to the [UInt](../../sql-reference/data-types/int-uint.md) data type. This function family includes:
+Converts an input value to the [UInt](/docs/en/sql-reference/data-types/int-uint.md) data type. This function family includes:
- `toUInt8(expr)` — Results in the `UInt8` data type.
- `toUInt16(expr)` — Results in the `UInt16` data type.
@@ -124,7 +124,7 @@ Converts an input value to the [UInt](../../sql-reference/data-types/int-uint.md
**Arguments**
-- `expr` — [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a number or a string with the decimal representation of a number. Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped.
+- `expr` — [Expression](/docs/en/sql-reference/syntax.md/#syntax-expressions) returning a number or a string with the decimal representation of a number. Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped.
**Returned value**
@@ -132,7 +132,7 @@ Integer value in the `UInt8`, `UInt16`, `UInt32`, `UInt64` or `UInt256` data typ
Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers.
-The behavior of functions for negative arguments and for the [NaN and Inf](../../sql-reference/data-types/float.md#data_type-float-nan-inf) arguments is undefined. If you pass a string with a negative number, for example `'-32'`, ClickHouse raises an exception. Remember about [numeric conversions issues](#numeric-conversion-issues), when using the functions.
+The behavior of functions for negative arguments and for the [NaN and Inf](/docs/en/sql-reference/data-types/float.md/#data_type-float-nan-inf) arguments is undefined. If you pass a string with a negative number, for example `'-32'`, ClickHouse raises an exception. Remember about [numeric conversions issues](#numeric-conversion-issues), when using the functions.
**Example**
@@ -166,7 +166,111 @@ Result:
## toDate
-Alias: `DATE`.
+Converts the argument to `Date` data type.
+
+If the argument is `DateTime` or `DateTime64`, it truncates it, leaving the date component of the DateTime:
+```sql
+SELECT
+ now() AS x,
+ toDate(x)
+```
+```response
+┌───────────────────x─┬─toDate(now())─┐
+│ 2022-12-30 13:44:17 │ 2022-12-30 │
+└─────────────────────┴───────────────┘
+```
+
+If the argument is a string, it is parsed as Date or DateTime. If it was parsed as DateTime, the date component is being used:
+```sql
+SELECT
+ toDate('2022-12-30') AS x,
+ toTypeName(x)
+```
+```response
+┌──────────x─┬─toTypeName(toDate('2022-12-30'))─┐
+│ 2022-12-30 │ Date │
+└────────────┴──────────────────────────────────┘
+
+1 row in set. Elapsed: 0.001 sec.
+```
+```sql
+SELECT
+ toDate('2022-12-30 01:02:03') AS x,
+ toTypeName(x)
+```
+```response
+┌──────────x─┬─toTypeName(toDate('2022-12-30 01:02:03'))─┐
+│ 2022-12-30 │ Date │
+└────────────┴───────────────────────────────────────────┘
+```
+
+If the argument is a number and it looks like a UNIX timestamp (is greater than 65535), it is interpreted as a DateTime, then truncated to Date in the current timezone. The timezone argument can be specified as a second argument of the function. The truncation to Date depends on the timezone:
+
+```sql
+SELECT
+ now() AS current_time,
+ toUnixTimestamp(current_time) AS ts,
+ toDateTime(ts) AS time_Amsterdam,
+ toDateTime(ts, 'Pacific/Apia') AS time_Samoa,
+ toDate(time_Amsterdam) AS date_Amsterdam,
+ toDate(time_Samoa) AS date_Samoa,
+ toDate(ts) AS date_Amsterdam_2,
+ toDate(ts, 'Pacific/Apia') AS date_Samoa_2
+```
+```response
+Row 1:
+──────
+current_time: 2022-12-30 13:51:54
+ts: 1672404714
+time_Amsterdam: 2022-12-30 13:51:54
+time_Samoa: 2022-12-31 01:51:54
+date_Amsterdam: 2022-12-30
+date_Samoa: 2022-12-31
+date_Amsterdam_2: 2022-12-30
+date_Samoa_2: 2022-12-31
+```
+
+The example above demonstrates how the same UNIX timestamp can be interpreted as different dates in different time zones.
+
+If the argument is a number and it is smaller than 65536, it is interpreted as the number of days since 1970-01-01 (a UNIX day) and converted to Date. It corresponds to the internal numeric representation of the `Date` data type. Example:
+
+```sql
+SELECT toDate(12345)
+```
+```response
+┌─toDate(12345)─┐
+│ 2003-10-20 │
+└───────────────┘
+```
+
+This conversion does not depend on timezones.
+
+If the argument does not fit in the range of the Date type, it results in an implementation-defined behavior, that can saturate to the maximum supported date or overflow:
+```sql
+SELECT toDate(10000000000.)
+```
+```response
+┌─toDate(10000000000.)─┐
+│ 2106-02-07 │
+└──────────────────────┘
+```
+
+The function `toDate` can be also written in alternative forms:
+
+```sql
+SELECT
+ now() AS time,
+ toDate(time),
+ DATE(time),
+ CAST(time, 'Date')
+```
+```response
+┌────────────────time─┬─toDate(now())─┬─DATE(now())─┬─CAST(now(), 'Date')─┐
+│ 2022-12-30 13:54:58 │ 2022-12-30 │ 2022-12-30 │ 2022-12-30 │
+└─────────────────────┴───────────────┴─────────────┴─────────────────────┘
+```
+
+Have a nice day working with dates and times.
## toDateOrZero
@@ -184,7 +288,7 @@ Alias: `DATE`.
## toDate32
-Converts the argument to the [Date32](../../sql-reference/data-types/date32.md) data type. If the value is outside the range returns the border values supported by `Date32`. If the argument has [Date](../../sql-reference/data-types/date.md) type, borders of `Date` are taken into account.
+Converts the argument to the [Date32](/docs/en/sql-reference/data-types/date32.md) data type. If the value is outside the range, `toDate32` returns the border values supported by `Date32`. If the argument has [Date](/docs/en/sql-reference/data-types/date.md) type, borders of `Date` are taken into account.
**Syntax**
@@ -194,13 +298,13 @@ toDate32(expr)
**Arguments**
-- `expr` — The value. [String](../../sql-reference/data-types/string.md), [UInt32](../../sql-reference/data-types/int-uint.md) or [Date](../../sql-reference/data-types/date.md).
+- `expr` — The value. [String](/docs/en/sql-reference/data-types/string.md), [UInt32](/docs/en/sql-reference/data-types/int-uint.md) or [Date](/docs/en/sql-reference/data-types/date.md).
**Returned value**
- A calendar date.
-Type: [Date32](../../sql-reference/data-types/date32.md).
+Type: [Date32](/docs/en/sql-reference/data-types/date32.md).
**Example**
@@ -242,7 +346,7 @@ SELECT toDate32(toDate('1899-01-01')) AS value, toTypeName(value);
## toDate32OrZero
-The same as [toDate32](#todate32) but returns the min value of [Date32](../../sql-reference/data-types/date32.md) if invalid argument is received.
+The same as [toDate32](#todate32) but returns the min value of [Date32](/docs/en/sql-reference/data-types/date32.md) if an invalid argument is received.
**Example**
@@ -262,7 +366,7 @@ Result:
## toDate32OrNull
-The same as [toDate32](#todate32) but returns `NULL` if invalid argument is received.
+The same as [toDate32](#todate32) but returns `NULL` if an invalid argument is received.
**Example**
@@ -282,7 +386,7 @@ Result:
## toDate32OrDefault
-Converts the argument to the [Date32](../../sql-reference/data-types/date32.md) data type. If the value is outside the range returns the lower border value supported by `Date32`. If the argument has [Date](../../sql-reference/data-types/date.md) type, borders of `Date` are taken into account. Returns default value if invalid argument is received.
+Converts the argument to the [Date32](/docs/en/sql-reference/data-types/date32.md) data type. If the value is outside the range, `toDate32OrDefault` returns the lower border value supported by `Date32`. If the argument has [Date](/docs/en/sql-reference/data-types/date.md) type, borders of `Date` are taken into account. Returns default value if an invalid argument is received.
**Example**
@@ -304,7 +408,7 @@ Result:
## toDateTime64
-Converts the argument to the [DateTime64](../../sql-reference/data-types/datetime64.md) data type.
+Converts the argument to the [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) data type.
**Syntax**
@@ -314,7 +418,7 @@ toDateTime64(expr, scale, [timezone])
**Arguments**
-- `expr` — The value. [String](../../sql-reference/data-types/string.md), [UInt32](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [DateTime](../../sql-reference/data-types/datetime.md).
+- `expr` — The value. [String](/docs/en/sql-reference/data-types/string.md), [UInt32](/docs/en/sql-reference/data-types/int-uint.md), [Float](/docs/en/sql-reference/data-types/float.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md).
- `scale` - Tick size (precision): 10-precision seconds. Valid range: [ 0 : 9 ].
- `timezone` - Time zone of the specified datetime64 object.
@@ -322,7 +426,7 @@ toDateTime64(expr, scale, [timezone])
- A calendar date and time of day, with sub-second precision.
-Type: [DateTime64](../../sql-reference/data-types/datetime64.md).
+Type: [DateTime64](/docs/en/sql-reference/data-types/datetime64.md).
**Example**
@@ -378,7 +482,7 @@ SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Asia/Istanbul') AS value, toTypeN
## toDecimal(32\|64\|128\|256)
-Converts `value` to the [Decimal](../../sql-reference/data-types/decimal.md) data type with precision of `S`. The `value` can be a number or a string. The `S` (scale) parameter specifies the number of decimal places.
+Converts `value` to the [Decimal](/docs/en/sql-reference/data-types/decimal.md) data type with precision of `S`. The `value` can be a number or a string. The `S` (scale) parameter specifies the number of decimal places.
- `toDecimal32(value, S)`
- `toDecimal64(value, S)`
@@ -387,7 +491,7 @@ Converts `value` to the [Decimal](../../sql-reference/data-types/decimal.md) dat
## toDecimal(32\|64\|128\|256)OrNull
-Converts an input string to a [Nullable(Decimal(P,S))](../../sql-reference/data-types/decimal.md) data type value. This family of functions include:
+Converts an input string to a [Nullable(Decimal(P,S))](/docs/en/sql-reference/data-types/decimal.md) data type value. This family of functions includes:
- `toDecimal32OrNull(expr, S)` — Results in `Nullable(Decimal32(S))` data type.
- `toDecimal64OrNull(expr, S)` — Results in `Nullable(Decimal64(S))` data type.
@@ -398,7 +502,7 @@ These functions should be used instead of `toDecimal*()` functions, if you prefe
**Arguments**
-- `expr` — [Expression](../../sql-reference/syntax.md#syntax-expressions), returns a value in the [String](../../sql-reference/data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`.
+- `expr` — [Expression](/docs/en/sql-reference/syntax.md/#syntax-expressions), returns a value in the [String](/docs/en/sql-reference/data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`.
- `S` — Scale, the number of decimal places in the resulting value.
**Returned value**
@@ -441,7 +545,7 @@ Result:
## toDecimal(32\|64\|128\|256)OrDefault
-Converts an input string to a [Decimal(P,S)](../../sql-reference/data-types/decimal.md) data type value. This family of functions include:
+Converts an input string to a [Decimal(P,S)](/docs/en/sql-reference/data-types/decimal.md) data type value. This family of functions includes:
- `toDecimal32OrDefault(expr, S)` — Results in `Decimal32(S)` data type.
- `toDecimal64OrDefault(expr, S)` — Results in `Decimal64(S)` data type.
@@ -452,7 +556,7 @@ These functions should be used instead of `toDecimal*()` functions, if you prefe
**Arguments**
-- `expr` — [Expression](../../sql-reference/syntax.md#syntax-expressions), returns a value in the [String](../../sql-reference/data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`.
+- `expr` — [Expression](/docs/en/sql-reference/syntax.md/#syntax-expressions), returns a value in the [String](/docs/en/sql-reference/data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`.
- `S` — Scale, the number of decimal places in the resulting value.
**Returned value**
@@ -494,7 +598,7 @@ Result:
## toDecimal(32\|64\|128\|256)OrZero
-Converts an input value to the [Decimal(P,S)](../../sql-reference/data-types/decimal.md) data type. This family of functions include:
+Converts an input value to the [Decimal(P,S)](/docs/en/sql-reference/data-types/decimal.md) data type. This family of functions includes:
- `toDecimal32OrZero( expr, S)` — Results in `Decimal32(S)` data type.
- `toDecimal64OrZero( expr, S)` — Results in `Decimal64(S)` data type.
@@ -505,7 +609,7 @@ These functions should be used instead of `toDecimal*()` functions, if you prefe
**Arguments**
-- `expr` — [Expression](../../sql-reference/syntax.md#syntax-expressions), returns a value in the [String](../../sql-reference/data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`.
+- `expr` — [Expression](/docs/en/sql-reference/syntax.md/#syntax-expressions), returns a value in the [String](/docs/en/sql-reference/data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`.
- `S` — Scale, the number of decimal places in the resulting value.
**Returned value**
@@ -564,7 +668,7 @@ YYYY-MM-DD hh:mm:ss
As an exception, if converting from UInt32, Int32, UInt64, or Int64 numeric types to Date, and if the number is greater than or equal to 65536, the number is interpreted as a Unix timestamp (and not as the number of days) and is rounded to the date. This allows support for the common occurrence of writing ‘toDate(unix_timestamp)’, which otherwise would be an error and would require writing the more cumbersome ‘toDate(toDateTime(unix_timestamp))’.
-Conversion between a date and date with time is performed the natural way: by adding a null time or dropping the time.
+Conversion between a date and a date with time is performed the natural way: by adding a null time or dropping the time.
Conversion between numeric types uses the same rules as assignments between different numeric types in C++.
@@ -643,15 +747,15 @@ These functions accept a string and interpret the bytes placed at the beginning
## reinterpretAsString
-This function accepts a number or date or date with time, and returns a string containing bytes representing the corresponding value in host order (little endian). Null bytes are dropped from the end. For example, a UInt32 type value of 255 is a string that is one byte long.
+This function accepts a number or date or date with time and returns a string containing bytes representing the corresponding value in host order (little endian). Null bytes are dropped from the end. For example, a UInt32 type value of 255 is a string that is one byte long.
## reinterpretAsFixedString
-This function accepts a number or date or date with time, and returns a FixedString containing bytes representing the corresponding value in host order (little endian). Null bytes are dropped from the end. For example, a UInt32 type value of 255 is a FixedString that is one byte long.
+This function accepts a number or date or date with time and returns a FixedString containing bytes representing the corresponding value in host order (little endian). Null bytes are dropped from the end. For example, a UInt32 type value of 255 is a FixedString that is one byte long.
## reinterpretAsUUID
-Accepts 16 bytes string and returns UUID containing bytes representing the corresponding value in network byte order (big-endian). If the string isn't long enough, the function works as if the string is padded with the necessary number of null bytes to the end. If the string longer than 16 bytes, the extra bytes at the end are ignored.
+Accepts 16 bytes string and returns UUID containing bytes representing the corresponding value in network byte order (big-endian). If the string isn't long enough, the function works as if the string is padded with the necessary number of null bytes to the end. If the string is longer than 16 bytes, the extra bytes at the end are ignored.
**Syntax**
@@ -661,11 +765,11 @@ reinterpretAsUUID(fixed_string)
**Arguments**
-- `fixed_string` — Big-endian byte string. [FixedString](../../sql-reference/data-types/fixedstring.md#fixedstring).
+- `fixed_string` — Big-endian byte string. [FixedString](/docs/en/sql-reference/data-types/fixedstring.md/#fixedstring).
**Returned value**
-- The UUID type value. [UUID](../../sql-reference/data-types/uuid.md#uuid-data-type).
+- The UUID type value. [UUID](/docs/en/sql-reference/data-types/uuid.md/#uuid-data-type).
**Examples**
@@ -718,7 +822,7 @@ reinterpret(x, type)
**Arguments**
- `x` — Any type.
-- `type` — Destination type. [String](../../sql-reference/data-types/string.md).
+- `type` — Destination type. [String](/docs/en/sql-reference/data-types/string.md).
**Returned value**
@@ -757,7 +861,7 @@ x::t
**Arguments**
- `x` — A value to convert. May be of any type.
-- `T` — The name of the target data type. [String](../../sql-reference/data-types/string.md).
+- `T` — The name of the target data type. [String](/docs/en/sql-reference/data-types/string.md).
- `t` — The target data type.
**Returned value**
@@ -806,9 +910,9 @@ Result:
└─────────────────────┴─────────────────────┴────────────┴─────────────────────┴───────────────────────────┘
```
-Conversion to FixedString(N) only works for arguments of type [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md).
+Conversion to FixedString(N) only works for arguments of type [String](/docs/en/sql-reference/data-types/string.md) or [FixedString](/docs/en/sql-reference/data-types/fixedstring.md).
-Type conversion to [Nullable](../../sql-reference/data-types/nullable.md) and back is supported.
+Type conversion to [Nullable](/docs/en/sql-reference/data-types/nullable.md) and back is supported.
**Example**
@@ -844,7 +948,7 @@ Result:
**See also**
-- [cast_keep_nullable](../../operations/settings/settings.md#cast_keep_nullable) setting
+- [cast_keep_nullable](/docs/en/operations/settings/settings.md/#cast_keep_nullable) setting
## accurateCast(x, T)
@@ -882,7 +986,7 @@ Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Value in c
## accurateCastOrNull(x, T)
-Converts input value `x` to the specified data type `T`. Always returns [Nullable](../../sql-reference/data-types/nullable.md) type and returns [NULL](../../sql-reference/syntax.md#null-literal) if the casted value is not representable in the target type.
+Converts input value `x` to the specified data type `T`. Always returns [Nullable](/docs/en/sql-reference/data-types/nullable.md) type and returns [NULL](/docs/en/sql-reference/syntax.md/#null-literal) if the casted value is not representable in the target type.
**Syntax**
@@ -991,7 +1095,7 @@ Result:
## toInterval(Year\|Quarter\|Month\|Week\|Day\|Hour\|Minute\|Second)
-Converts a Number type argument to an [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type.
+Converts a Number type argument to an [Interval](/docs/en/sql-reference/data-types/special-data-types/interval.md) data type.
**Syntax**
@@ -1039,7 +1143,7 @@ Result:
## parseDateTimeBestEffort
## parseDateTime32BestEffort
-Converts a date and time in the [String](../../sql-reference/data-types/string.md) representation to [DateTime](../../sql-reference/data-types/datetime.md#data_type-datetime) data type.
+Converts a date and time in the [String](/docs/en/sql-reference/data-types/string.md) representation to [DateTime](/docs/en/sql-reference/data-types/datetime.md/#data_type-datetime) data type.
The function parses [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601), [RFC 1123 - 5.2.14 RFC-822 Date and Time Specification](https://tools.ietf.org/html/rfc1123#page-55), ClickHouse’s and some other date and time formats.
@@ -1051,8 +1155,8 @@ parseDateTimeBestEffort(time_string [, time_zone])
**Arguments**
-- `time_string` — String containing a date and time to convert. [String](../../sql-reference/data-types/string.md).
-- `time_zone` — Time zone. The function parses `time_string` according to the time zone. [String](../../sql-reference/data-types/string.md).
+- `time_string` — String containing a date and time to convert. [String](/docs/en/sql-reference/data-types/string.md).
+- `time_zone` — Time zone. The function parses `time_string` according to the time zone. [String](/docs/en/sql-reference/data-types/string.md).
**Supported non-standard formats**
@@ -1175,7 +1279,7 @@ Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) function except
## parseDateTime64BestEffort
-Same as [parseDateTimeBestEffort](#parsedatetimebesteffort) function but also parse milliseconds and microseconds and returns [DateTime](../../sql-reference/functions/type-conversion-functions.md#data_type-datetime) data type.
+Same as [parseDateTimeBestEffort](#parsedatetimebesteffort) function but also parse milliseconds and microseconds and returns [DateTime](/docs/en/sql-reference/functions/type-conversion-functions.md/#data_type-datetime) data type.
**Syntax**
@@ -1185,13 +1289,13 @@ parseDateTime64BestEffort(time_string [, precision [, time_zone]])
**Parameters**
-- `time_string` — String containing a date or date with time to convert. [String](../../sql-reference/data-types/string.md).
-- `precision` — Required precision. `3` — for milliseconds, `6` — for microseconds. Default — `3`. Optional. [UInt8](../../sql-reference/data-types/int-uint.md).
-- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md).
+- `time_string` — String containing a date or date with time to convert. [String](/docs/en/sql-reference/data-types/string.md).
+- `precision` — Required precision. `3` — for milliseconds, `6` — for microseconds. Default — `3`. Optional. [UInt8](/docs/en/sql-reference/data-types/int-uint.md).
+- `time_zone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](/docs/en/sql-reference/data-types/string.md).
**Returned value**
-- `time_string` converted to the [DateTime](../../sql-reference/data-types/datetime.md) data type.
+- `time_string` converted to the [DateTime](/docs/en/sql-reference/data-types/datetime.md) data type.
**Examples**
@@ -1242,7 +1346,7 @@ Same as for [parseDateTime64BestEffort](#parsedatetime64besteffort), except that
## toLowCardinality
-Converts input parameter to the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) version of same data type.
+Converts input parameter to the [LowCardinality](/docs/en/sql-reference/data-types/lowcardinality.md) version of same data type.
To convert data from the `LowCardinality` data type use the [CAST](#type_conversion_function-cast) function. For example, `CAST(x as String)`.
@@ -1254,7 +1358,7 @@ toLowCardinality(expr)
**Arguments**
-- `expr` — [Expression](../../sql-reference/syntax.md#syntax-expressions) resulting in one of the [supported data types](../../sql-reference/data-types/index.md#data_types).
+- `expr` — [Expression](/docs/en/sql-reference/syntax.md/#syntax-expressions) resulting in one of the [supported data types](/docs/en/sql-reference/data-types/index.md/#data_types).
**Returned values**
@@ -1388,12 +1492,12 @@ formatRow(format, x, y, ...)
**Arguments**
-- `format` — Text format. For example, [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated).
+- `format` — Text format. For example, [CSV](/docs/en/interfaces/formats.md/#csv), [TSV](/docs/en/interfaces/formats.md/#tabseparated).
- `x`,`y`, ... — Expressions.
**Returned value**
-- A formatted string (for text formats it's usually terminated with the new line character).
+- A formatted string. (for text formats it's usually terminated with the new line character).
**Example**
@@ -1417,9 +1521,39 @@ Result:
└──────────────────────────────────┘
```
+**Note**: If format contains suffix/prefix, it will be written in each row.
+
+**Example**
+
+Query:
+
+``` sql
+SELECT formatRow('CustomSeparated', number, 'good')
+FROM numbers(3)
+SETTINGS format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter=''
+```
+
+Result:
+
+``` text
+┌─formatRow('CustomSeparated', number, 'good')─┐
+│
+0 good
+ │
+│
+1 good
+ │
+│
+2 good
+ │
+└──────────────────────────────────────────────┘
+```
+
+Note: Only row-based formats are supported in this function.
+
## formatRowNoNewline
-Converts arbitrary expressions into a string via given format. The function trims the last `\n` if any.
+Converts arbitrary expressions into a string via given format. Differs from formatRow in that this function trims the last `\n` if any.
**Syntax**
@@ -1429,7 +1563,7 @@ formatRowNoNewline(format, x, y, ...)
**Arguments**
-- `format` — Text format. For example, [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated).
+- `format` — Text format. For example, [CSV](/docs/en/interfaces/formats.md/#csv), [TSV](/docs/en/interfaces/formats.md/#tabseparated).
- `x`,`y`, ... — Expressions.
**Returned value**
@@ -1457,7 +1591,7 @@ Result:
## snowflakeToDateTime
-Extracts time from [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as [DateTime](../data-types/datetime.md) format.
+Extracts time from [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as [DateTime](/docs/en/sql-reference/data-types/datetime.md) format.
**Syntax**
@@ -1467,12 +1601,12 @@ snowflakeToDateTime(value [, time_zone])
**Parameters**
-- `value` — Snowflake ID. [Int64](../data-types/int-uint.md).
-- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md).
+- `value` — Snowflake ID. [Int64](/docs/en/sql-reference/data-types/int-uint.md).
+- `time_zone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](/docs/en/sql-reference/data-types/string.md).
**Returned value**
-- Input value converted to the [DateTime](../data-types/datetime.md) data type.
+- Input value converted to the [DateTime](/docs/en/sql-reference/data-types/datetime.md) data type.
**Example**
@@ -1493,7 +1627,7 @@ Result:
## snowflakeToDateTime64
-Extracts time from [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as [DateTime64](../data-types/datetime64.md) format.
+Extracts time from [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) format.
**Syntax**
@@ -1503,12 +1637,12 @@ snowflakeToDateTime64(value [, time_zone])
**Parameters**
-- `value` — Snowflake ID. [Int64](../data-types/int-uint.md).
-- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md).
+- `value` — Snowflake ID. [Int64](/docs/en/sql-reference/data-types/int-uint.md).
+- `time_zone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](/docs/en/sql-reference/data-types/string.md).
**Returned value**
-- Input value converted to the [DateTime64](../data-types/datetime64.md) data type.
+- Input value converted to the [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) data type.
**Example**
@@ -1529,7 +1663,7 @@ Result:
## dateTimeToSnowflake
-Converts [DateTime](../data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time.
+Converts [DateTime](/docs/en/sql-reference/data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time.
**Syntax**
@@ -1539,11 +1673,11 @@ dateTimeToSnowflake(value)
**Parameters**
-- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md).
+- `value` — Date and time. [DateTime](/docs/en/sql-reference/data-types/datetime.md).
**Returned value**
-- Input value converted to the [Int64](../data-types/int-uint.md) data type as the first Snowflake ID at that time.
+- Input value converted to the [Int64](/docs/en/sql-reference/data-types/int-uint.md) data type as the first Snowflake ID at that time.
**Example**
@@ -1563,7 +1697,7 @@ Result:
## dateTime64ToSnowflake
-Convert [DateTime64](../data-types/datetime64.md) to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time.
+Convert [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time.
**Syntax**
@@ -1573,11 +1707,11 @@ dateTime64ToSnowflake(value)
**Parameters**
-- `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md).
+- `value` — Date and time. [DateTime64](/docs/en/sql-reference/data-types/datetime64.md).
**Returned value**
-- Input value converted to the [Int64](../data-types/int-uint.md) data type as the first Snowflake ID at that time.
+- Input value converted to the [Int64](/docs/en/sql-reference/data-types/int-uint.md) data type as the first Snowflake ID at that time.
**Example**
diff --git a/docs/en/sql-reference/statements/select/from.md b/docs/en/sql-reference/statements/select/from.md
index 3013a173c16..b751384cb72 100644
--- a/docs/en/sql-reference/statements/select/from.md
+++ b/docs/en/sql-reference/statements/select/from.md
@@ -21,12 +21,11 @@ Subquery is another `SELECT` query that may be specified in parenthesis inside `
When `FINAL` is specified, ClickHouse fully merges the data before returning the result and thus performs all data transformations that happen during merges for the given table engine.
-It is applicable when selecting data from tables that use the [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)-engine family. Also supported for:
+It is applicable when selecting data from ReplacingMergeTree, SummingMergeTree, AggregatingMergeTree, CollapsingMergeTree and VersionedCollapsingMergeTree tables.
-- [Replicated](../../../engines/table-engines/mergetree-family/replication.md) versions of `MergeTree` engines.
-- [View](../../../engines/table-engines/special/view.md), [Buffer](../../../engines/table-engines/special/buffer.md), [Distributed](../../../engines/table-engines/special/distributed.md), and [MaterializedView](../../../engines/table-engines/special/materializedview.md) engines that operate over other engines, provided they were created over `MergeTree`-engine tables.
+`SELECT` queries with `FINAL` are executed in parallel. The [max_final_threads](../../../operations/settings/settings.md#max-final-threads) setting limits the number of threads used.
-Now `SELECT` queries with `FINAL` are executed in parallel and slightly faster. But there are drawbacks (see below). The [max_final_threads](../../../operations/settings/settings.md#max-final-threads) setting limits the number of threads used.
+There are drawbacks to using `FINAL` (see below).
### Drawbacks
diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md
index 0713fe42f38..80472178ae2 100644
--- a/docs/ru/development/developer-instruction.md
+++ b/docs/ru/development/developer-instruction.md
@@ -169,12 +169,6 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)"
cmake -D CMAKE_BUILD_TYPE=Debug ..
-В случае использования на разработческой машине старого HDD или SSD, а также при желании использовать меньше места для артефактов сборки можно использовать следующую команду:
-```bash
-cmake -DUSE_DEBUG_HELPERS=1 -DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 ..
-```
-При этом надо учесть, что получаемые в результате сборки исполнимые файлы будут динамически слинкованы с библиотеками, и поэтому фактически станут непереносимыми на другие компьютеры (либо для этого нужно будет предпринять значительно больше усилий по сравнению со статической сборкой). Плюсом же в данном случае является значительно меньшее время сборки (это проявляется не на первой сборке, а на последующих, после внесения изменений в исходный код - тратится меньшее время на линковку по сравнению со статической сборкой) и значительно меньшее использование места на жёстком диске (экономия более, чем в 3 раза по сравнению со статической сборкой). Для целей разработки, когда планируются только отладочные запуски на том же компьютере, где осуществлялась сборка, это может быть наиболее удобным вариантом.
-
Вы можете изменить вариант сборки, выполнив новую команду в директории build.
Запустите ninja для сборки:
diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md
index f430f5cae51..8fbcaf9568b 100644
--- a/docs/ru/sql-reference/functions/date-time-functions.md
+++ b/docs/ru/sql-reference/functions/date-time-functions.md
@@ -424,23 +424,23 @@ WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 SELECT toStartOfSecond(d
## toRelativeYearNum {#torelativeyearnum}
-Переводит дату-с-временем или дату в номер года, начиная с некоторого фиксированного момента в прошлом.
+Переводит дату или дату-с-временем в номер года, начиная с некоторого фиксированного момента в прошлом.
## toRelativeQuarterNum {#torelativequarternum}
-Переводит дату-с-временем или дату в номер квартала, начиная с некоторого фиксированного момента в прошлом.
+Переводит дату или дату-с-временем в номер квартала, начиная с некоторого фиксированного момента в прошлом.
## toRelativeMonthNum {#torelativemonthnum}
-Переводит дату-с-временем или дату в номер месяца, начиная с некоторого фиксированного момента в прошлом.
+Переводит дату или дату-с-временем в номер месяца, начиная с некоторого фиксированного момента в прошлом.
## toRelativeWeekNum {#torelativeweeknum}
-Переводит дату-с-временем или дату в номер недели, начиная с некоторого фиксированного момента в прошлом.
+Переводит дату или дату-с-временем в номер недели, начиная с некоторого фиксированного момента в прошлом.
## toRelativeDayNum {#torelativedaynum}
-Переводит дату-с-временем или дату в номер дня, начиная с некоторого фиксированного момента в прошлом.
+Переводит дату или дату-с-временем в номер дня, начиная с некоторого фиксированного момента в прошлом.
## toRelativeHourNum {#torelativehournum}
@@ -456,7 +456,7 @@ WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 SELECT toStartOfSecond(d
## toISOYear {#toisoyear}
-Переводит дату-с-временем или дату в число типа UInt16, содержащее номер ISO года. ISO год отличается от обычного года, потому что в соответствии с [ISO 8601:1988](https://en.wikipedia.org/wiki/ISO_8601) ISO год начинается необязательно первого января.
+Переводит дату или дату-с-временем в число типа UInt16, содержащее номер ISO года. ISO год отличается от обычного года, потому что в соответствии с [ISO 8601:1988](https://en.wikipedia.org/wiki/ISO_8601) ISO год начинается необязательно первого января.
**Пример**
@@ -479,7 +479,7 @@ SELECT
## toISOWeek {#toisoweek}
-Переводит дату-с-временем или дату в число типа UInt8, содержащее номер ISO недели.
+Переводит дату или дату-с-временем в число типа UInt8, содержащее номер ISO недели.
Начало ISO года отличается от начала обычного года, потому что в соответствии с [ISO 8601:1988](https://en.wikipedia.org/wiki/ISO_8601) первая неделя года - это неделя с четырьмя или более днями в этом году.
1 Января 2017 г. - воскресение, т.е. первая ISO неделя 2017 года началась в понедельник 2 января, поэтому 1 января 2017 это последняя неделя 2016 года.
@@ -503,7 +503,7 @@ SELECT
```
## toWeek(date\[, mode\]\[, timezone\]) {#toweek}
-Переводит дату-с-временем или дату в число UInt8, содержащее номер недели. Второй аргументам mode задает режим, начинается ли неделя с воскресенья или с понедельника и должно ли возвращаемое значение находиться в диапазоне от 0 до 53 или от 1 до 53. Если аргумент mode опущен, то используется режим 0.
+Переводит дату или дату-с-временем в число UInt8, содержащее номер недели. Второй аргументам mode задает режим, начинается ли неделя с воскресенья или с понедельника и должно ли возвращаемое значение находиться в диапазоне от 0 до 53 или от 1 до 53. Если аргумент mode опущен, то используется режим 0.
`toISOWeek() ` эквивалентно `toWeek(date,3)`.
@@ -569,6 +569,132 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d
└────────────┴───────────┴───────────┴───────────┘
```
+## age
+
+Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 секунду.
+Например, разница между `2021-12-29` и `2022-01-01` 3 дня для единицы `day`, 0 месяцев для единицы `month`, 0 лет для единицы `year`.
+
+**Синтаксис**
+
+``` sql
+age('unit', startdate, enddate, [timezone])
+```
+
+**Аргументы**
+
+- `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md).
+ Возможные значения:
+
+ - `second` (возможные сокращения: `ss`, `s`)
+ - `minute` (возможные сокращения: `mi`, `n`)
+ - `hour` (возможные сокращения: `hh`, `h`)
+ - `day` (возможные сокращения: `dd`, `d`)
+ - `week` (возможные сокращения: `wk`, `ww`)
+ - `month` (возможные сокращения: `mm`, `m`)
+ - `quarter` (возможные сокращения: `qq`, `q`)
+ - `year` (возможные сокращения: `yyyy`, `yy`)
+
+- `startdate` — первая дата или дата со временем, которая вычитается из `enddate`. [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) или [DateTime64](../../sql-reference/data-types/datetime64.md).
+
+- `enddate` — вторая дата или дата со временем, из которой вычитается `startdate`. [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) или [DateTime64](../../sql-reference/data-types/datetime64.md).
+
+- `timezone` — [часовой пояс](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (необязательно). Если этот аргумент указан, то он применяется как для `startdate`, так и для `enddate`. Если этот аргумент не указан, то используются часовые пояса аргументов `startdate` и `enddate`. Если часовые пояса аргументов `startdate` и `enddate` не совпадают, то результат не определен. [String](../../sql-reference/data-types/string.md).
+
+**Возвращаемое значение**
+
+Разница между `enddate` и `startdate`, выраженная в `unit`.
+
+Тип: [Int](../../sql-reference/data-types/int-uint.md).
+
+**Пример**
+
+Запрос:
+
+``` sql
+SELECT age('hour', toDateTime('2018-01-01 22:30:00'), toDateTime('2018-01-02 23:00:00'));
+```
+
+Результат:
+
+``` text
+┌─age('hour', toDateTime('2018-01-01 22:30:00'), toDateTime('2018-01-02 23:00:00'))─┐
+│ 24 │
+└───────────────────────────────────────────────────────────────────────────────────┘
+```
+
+Запрос:
+
+``` sql
+SELECT
+ toDate('2022-01-01') AS e,
+ toDate('2021-12-29') AS s,
+ age('day', s, e) AS day_age,
+ age('month', s, e) AS month__age,
+ age('year', s, e) AS year_age;
+```
+
+Результат:
+
+``` text
+┌──────────e─┬──────────s─┬─day_age─┬─month__age─┬─year_age─┐
+│ 2022-01-01 │ 2021-12-29 │ 3 │ 0 │ 0 │
+└────────────┴────────────┴─────────┴────────────┴──────────┘
+```
+
+## date\_diff {#date_diff}
+
+Вычисляет разницу указанных границ `unit` пересекаемых между `startdate` и `enddate`.
+
+**Синтаксис**
+
+``` sql
+date_diff('unit', startdate, enddate, [timezone])
+```
+
+Синонимы: `dateDiff`, `DATE_DIFF`.
+
+**Аргументы**
+
+- `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md).
+ Возможные значения:
+
+ - `second` (возможные сокращения: `ss`, `s`)
+ - `minute` (возможные сокращения: `mi`, `n`)
+ - `hour` (возможные сокращения: `hh`, `h`)
+ - `day` (возможные сокращения: `dd`, `d`)
+ - `week` (возможные сокращения: `wk`, `ww`)
+ - `month` (возможные сокращения: `mm`, `m`)
+ - `quarter` (возможные сокращения: `qq`, `q`)
+ - `year` (возможные сокращения: `yyyy`, `yy`)
+
+- `startdate` — первая дата или дата со временем, которая вычитается из `enddate`. [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) или [DateTime64](../../sql-reference/data-types/datetime64.md).
+
+- `enddate` — вторая дата или дата со временем, из которой вычитается `startdate`. [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) или [DateTime64](../../sql-reference/data-types/datetime64.md).
+
+- `timezone` — [часовой пояс](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (необязательно). Если этот аргумент указан, то он применяется как для `startdate`, так и для `enddate`. Если этот аргумент не указан, то используются часовые пояса аргументов `startdate` и `enddate`. Если часовые пояса аргументов `startdate` и `enddate` не совпадают, то результат не определен. [String](../../sql-reference/data-types/string.md).
+
+**Возвращаемое значение**
+
+Разница между `enddate` и `startdate`, выраженная в `unit`.
+
+Тип: [Int](../../sql-reference/data-types/int-uint.md).
+
+**Пример**
+
+Запрос:
+
+``` sql
+SELECT dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'));
+```
+
+Результат:
+
+``` text
+┌─dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'))─┐
+│ 25 │
+└────────────────────────────────────────────────────────────────────────────────────────┘
+```
+
## date_trunc {#date_trunc}
Отсекает от даты и времени части, меньшие чем указанная часть.
@@ -689,60 +815,6 @@ SELECT date_add(YEAR, 3, toDate('2018-01-01'));
└───────────────────────────────────────────────┘
```
-## date\_diff {#date_diff}
-
-Вычисляет разницу между двумя значениями дат или дат со временем.
-
-**Синтаксис**
-
-``` sql
-date_diff('unit', startdate, enddate, [timezone])
-```
-
-Синонимы: `dateDiff`, `DATE_DIFF`.
-
-**Аргументы**
-
-- `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md).
- Возможные значения:
-
- - `second`
- - `minute`
- - `hour`
- - `day`
- - `week`
- - `month`
- - `quarter`
- - `year`
-
-- `startdate` — первая дата или дата со временем, которая вычитается из `enddate`. [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) или [DateTime64](../../sql-reference/data-types/datetime64.md).
-
-- `enddate` — вторая дата или дата со временем, из которой вычитается `startdate`. [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) или [DateTime64](../../sql-reference/data-types/datetime64.md).
-
-- `timezone` — [часовой пояс](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (необязательно). Если этот аргумент указан, то он применяется как для `startdate`, так и для `enddate`. Если этот аргумент не указан, то используются часовые пояса аргументов `startdate` и `enddate`. Если часовые пояса аргументов `startdate` и `enddate` не совпадают, то результат не определен. [String](../../sql-reference/data-types/string.md).
-
-**Возвращаемое значение**
-
-Разница между `enddate` и `startdate`, выраженная в `unit`.
-
-Тип: [Int](../../sql-reference/data-types/int-uint.md).
-
-**Пример**
-
-Запрос:
-
-``` sql
-SELECT dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'));
-```
-
-Результат:
-
-``` text
-┌─dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'))─┐
-│ 25 │
-└────────────────────────────────────────────────────────────────────────────────────────┘
-```
-
## date\_sub {#date_sub}
Вычитает интервал времени или даты из указанной даты или даты со временем.
diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md
index 0f6f61173b6..c2beb55fee1 100644
--- a/docs/ru/sql-reference/functions/type-conversion-functions.md
+++ b/docs/ru/sql-reference/functions/type-conversion-functions.md
@@ -1316,7 +1316,7 @@ formatRow(format, x, y, ...)
**Возвращаемое значение**
-- Отформатированная строка (в текстовых форматах обычно с завершающим переводом строки).
+- Отформатированная строка. (в текстовых форматах обычно с завершающим переводом строки).
**Пример**
@@ -1340,9 +1340,39 @@ FROM numbers(3);
└──────────────────────────────────┘
```
+**Примечание**: если формат содержит префикс/суффикс, то он будет записан в каждой строке.
+
+**Пример**
+
+Запрос:
+
+``` sql
+SELECT formatRow('CustomSeparated', number, 'good')
+FROM numbers(3)
+SETTINGS format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter=''
+```
+
+Результат:
+
+``` text
+┌─formatRow('CustomSeparated', number, 'good')─┐
+│
+0 good
+ │
+│
+1 good
+ │
+│
+2 good
+ │
+└──────────────────────────────────────────────┘
+```
+
+**Примечание**: данная функция поддерживает только строковые форматы вывода.
+
## formatRowNoNewline {#formatrownonewline}
-Преобразует произвольные выражения в строку заданного формата. При этом удаляет лишние переводы строк `\n`, если они появились.
+Преобразует произвольные выражения в строку заданного формата. Отличается от функции formatRow тем, что удаляет лишний перевод строки `\n` а конце, если он есть.
**Синтаксис**
diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md
index 67b1eb228f1..e0c8b41147a 100644
--- a/docs/zh/development/continuous-integration.md
+++ b/docs/zh/development/continuous-integration.md
@@ -85,7 +85,6 @@ git push
- **Build type**: `Debug` or `RelWithDebInfo` (cmake).
- **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan).
- **Bundled**: `bundled` 构建使用来自 `contrib` 库, 而 `unbundled` 构建使用系统库.
-- **Splitted**: `splitted` is a [split build](https://clickhouse.com/docs/en/development/build/#split-build)
- **Status**: `成功` 或 `失败`
- **Build log**: 链接到构建和文件复制日志, 当构建失败时很有用.
- **Build time**.
@@ -97,7 +96,6 @@ git push
- `clickhouse`: Main built binary.
- `clickhouse-odbc-bridge`
- `unit_tests_dbms`: 带有 ClickHouse 单元测试的 GoogleTest 二进制文件.
- - `shared_build.tgz`: 使用共享库构建.
- `performance.tgz`: 用于性能测试的特殊包.
## 特殊构建检查 {#special-buildcheck}
@@ -123,14 +121,6 @@ git push
of error.
```
-## 冒烟测试 {#split-build-smoke-test}
-检查[拆分构建](./build.md#split-build)配置中的服务器构建是否可以启动并运行简单查询.如果失败:
-```
-* Fix other test errors first;
-* Build the server in [split build](./build.md#split-build) configuration
- locally and check whether it can start and run `select 1`.
-```
-
## 兼容性检查 {#compatibility-check}
检查`clickhouse`二进制文件是否可以在带有旧libc版本的发行版上运行.如果失败, 请向维护人员寻求帮助.
diff --git a/docs/zh/engines/table-engines/log-family/index.md b/docs/zh/engines/table-engines/log-family/index.md
index 56776522445..1b24984f75f 100644
--- a/docs/zh/engines/table-engines/log-family/index.md
+++ b/docs/zh/engines/table-engines/log-family/index.md
@@ -11,7 +11,7 @@ sidebar_position: 29
这系列的引擎有:
- [StripeLog](stripelog.md)
-- [日志](log.md)
+- [Log](log.md)
- [TinyLog](tinylog.md)
## 共同属性 {#table_engines-log-engine-family-common-properties}
diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt
index 5b6c5b26633..9e4f66562ca 100644
--- a/programs/CMakeLists.txt
+++ b/programs/CMakeLists.txt
@@ -13,12 +13,6 @@ option (ENABLE_CLICKHOUSE_SERVER "Server mode (main mode)" ${ENABLE_CLICKHOUSE_A
option (ENABLE_CLICKHOUSE_CLIENT "Client mode (interactive tui/shell that connects to the server)"
${ENABLE_CLICKHOUSE_ALL})
-# Don't create self-extracting clickhouse for split build
-if (ENABLE_CLICKHOUSE_SELF_EXTRACTING AND SPLIT_SHARED_LIBRARIES)
- message (STATUS "Self-extracting on split build is not supported")
- unset (ENABLE_CLICKHOUSE_SELF_EXTRACTING CACHE)
-endif ()
-
# https://clickhouse.com/docs/en/operations/utilities/clickhouse-local/
option (ENABLE_CLICKHOUSE_LOCAL "Local files fast processing mode" ${ENABLE_CLICKHOUSE_ALL})
@@ -173,10 +167,6 @@ else()
message(STATUS "ClickHouse keeper-converter mode: OFF")
endif()
-if(NOT (USE_STATIC_LIBRARIES OR SPLIT_SHARED_LIBRARIES))
- set(CLICKHOUSE_ONE_SHARED ON)
-endif()
-
if (ENABLE_CLICKHOUSE_DISKS)
message(STATUS "Clickhouse disks mode: ON")
else()
@@ -192,11 +182,7 @@ endif()
configure_file (config_tools.h.in ${CONFIG_INCLUDE_PATH}/config_tools.h)
macro(clickhouse_target_link_split_lib target name)
- if(NOT CLICKHOUSE_ONE_SHARED)
- target_link_libraries(${target} PRIVATE clickhouse-${name}-lib)
- else()
- target_link_libraries(${target} PRIVATE clickhouse-lib)
- endif()
+ target_link_libraries(${target} PRIVATE clickhouse-${name}-lib)
endmacro()
macro(clickhouse_program_add_library name)
@@ -208,18 +194,16 @@ macro(clickhouse_program_add_library name)
set(CLICKHOUSE_${name_uc}_LINK ${CLICKHOUSE_${name_uc}_LINK} PARENT_SCOPE)
set(CLICKHOUSE_${name_uc}_INCLUDE ${CLICKHOUSE_${name_uc}_INCLUDE} PARENT_SCOPE)
- if(NOT CLICKHOUSE_ONE_SHARED)
- add_library(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_SOURCES})
+ add_library(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_SOURCES})
- set(_link ${CLICKHOUSE_${name_uc}_LINK}) # can't use ${} in if()
- if(_link)
- target_link_libraries(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_LINK})
- endif()
+ set(_link ${CLICKHOUSE_${name_uc}_LINK}) # can't use ${} in if()
+ if(_link)
+ target_link_libraries(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_LINK})
+ endif()
- set(_include ${CLICKHOUSE_${name_uc}_INCLUDE}) # can't use ${} in if()
- if (_include)
- target_include_directories(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_INCLUDE})
- endif()
+ set(_include ${CLICKHOUSE_${name_uc}_INCLUDE}) # can't use ${} in if()
+ if (_include)
+ target_include_directories(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_INCLUDE})
endif()
endmacro()
@@ -263,68 +247,8 @@ if (ENABLE_CLICKHOUSE_SELF_EXTRACTING)
add_subdirectory (self-extracting)
endif ()
-if (CLICKHOUSE_ONE_SHARED)
- add_library(clickhouse-lib SHARED
- ${CLICKHOUSE_SERVER_SOURCES}
- ${CLICKHOUSE_CLIENT_SOURCES}
- ${CLICKHOUSE_LOCAL_SOURCES}
- ${CLICKHOUSE_BENCHMARK_SOURCES}
- ${CLICKHOUSE_COPIER_SOURCES}
- ${CLICKHOUSE_EXTRACT_FROM_CONFIG_SOURCES}
- ${CLICKHOUSE_COMPRESSOR_SOURCES}
- ${CLICKHOUSE_FORMAT_SOURCES}
- ${CLICKHOUSE_OBFUSCATOR_SOURCES}
- ${CLICKHOUSE_GIT_IMPORT_SOURCES}
- ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}
- ${CLICKHOUSE_KEEPER_SOURCES}
- ${CLICKHOUSE_KEEPER_CONVERTER_SOURCES}
- ${CLICKHOUSE_STATIC_FILES_DISK_UPLOADER_SOURCES}
- ${CLICKHOUSE_SU_SOURCES})
-
- target_link_libraries(clickhouse-lib
- ${CLICKHOUSE_SERVER_LINK}
- ${CLICKHOUSE_CLIENT_LINK}
- ${CLICKHOUSE_LOCAL_LINK}
- ${CLICKHOUSE_BENCHMARK_LINK}
- ${CLICKHOUSE_COPIER_LINK}
- ${CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK}
- ${CLICKHOUSE_COMPRESSOR_LINK}
- ${CLICKHOUSE_FORMAT_LINK}
- ${CLICKHOUSE_OBFUSCATOR_LINK}
- ${CLICKHOUSE_GIT_IMPORT_LINK}
- ${CLICKHOUSE_ODBC_BRIDGE_LINK}
- ${CLICKHOUSE_KEEPER_LINK}
- ${CLICKHOUSE_KEEPER_CONVERTER_LINK}
- ${CLICKHOUSE_STATIC_FILES_DISK_UPLOADER_LINK}
- ${CLICKHOUSE_SU_LINK})
-
- target_include_directories(clickhouse-lib
- ${CLICKHOUSE_SERVER_INCLUDE}
- ${CLICKHOUSE_CLIENT_INCLUDE}
- ${CLICKHOUSE_LOCAL_INCLUDE}
- ${CLICKHOUSE_BENCHMARK_INCLUDE}
- ${CLICKHOUSE_COPIER_INCLUDE}
- ${CLICKHOUSE_EXTRACT_FROM_CONFIG_INCLUDE}
- ${CLICKHOUSE_COMPRESSOR_INCLUDE}
- ${CLICKHOUSE_FORMAT_INCLUDE}
- ${CLICKHOUSE_OBFUSCATOR_INCLUDE}
- ${CLICKHOUSE_GIT_IMPORT_INCLUDE}
- ${CLICKHOUSE_ODBC_BRIDGE_INCLUDE}
- ${CLICKHOUSE_KEEPER_INCLUDE}
- ${CLICKHOUSE_KEEPER_CONVERTER_INCLUDE})
-
- set_target_properties(clickhouse-lib PROPERTIES SOVERSION ${VERSION_MAJOR}.${VERSION_MINOR} VERSION ${VERSION_SO} OUTPUT_NAME clickhouse DEBUG_POSTFIX "")
- install (TARGETS clickhouse-lib LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} COMPONENT clickhouse)
-endif()
-
clickhouse_add_executable (clickhouse main.cpp)
-if (NOT USE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES)
- # Shared split (dev) build: In CI, the server is run with custom LD_LIBRARY_PATH. This makes the harmful env check re-execute the
- # process in a clean environment but as in CI the containing directory is not included in DT_RUNPATH/DT_RPATH, the server won't come up.
- target_compile_definitions(clickhouse PRIVATE DISABLE_HARMFUL_ENV_VAR_CHECK)
-endif ()
-
# A library that prevent usage of several functions from libc.
if (ARCH_AMD64 AND OS_LINUX AND NOT OS_ANDROID)
set (HARMFUL_LIB harmful)
diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp
index 017b28fe082..26099b352a3 100644
--- a/programs/benchmark/Benchmark.cpp
+++ b/programs/benchmark/Benchmark.cpp
@@ -683,7 +683,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
("confidence", value()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)")
("query_id", value()->default_value(""), "")
("max-consecutive-errors", value()->default_value(0), "set number of allowed consecutive errors")
- ("continue_on_errors", "continue testing even if a query fails")
+ ("ignore-error,continue_on_errors", "continue testing even if a query fails")
("reconnect", "establish new connection for every query")
("client-side-time", "display the time including network communication instead of server-side time; note that for server versions before 22.8 we always display client-side time")
;
@@ -738,7 +738,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
options["query_id"].as(),
options["query"].as(),
options["max-consecutive-errors"].as(),
- options.count("continue_on_errors"),
+ options.count("ignore-error"),
options.count("reconnect"),
options.count("client-side-time"),
print_stacktrace,
diff --git a/programs/benchmark/CMakeLists.txt b/programs/benchmark/CMakeLists.txt
index 3fa8deb6bd9..ad211399bb5 100644
--- a/programs/benchmark/CMakeLists.txt
+++ b/programs/benchmark/CMakeLists.txt
@@ -10,6 +10,4 @@ set (CLICKHOUSE_BENCHMARK_LINK
clickhouse_program_add(benchmark)
-if(NOT CLICKHOUSE_ONE_SHARED)
- target_link_libraries (clickhouse-benchmark-lib PRIVATE clickhouse-client-lib)
-endif()
+target_link_libraries (clickhouse-benchmark-lib PRIVATE clickhouse-client-lib)
diff --git a/programs/clickhouse-split-helper b/programs/clickhouse-split-helper
deleted file mode 100755
index 14a86f76097..00000000000
--- a/programs/clickhouse-split-helper
+++ /dev/null
@@ -1,12 +0,0 @@
-#!/bin/sh
-
-# Helper for split build mode.
-# Allows to run commands like
-# clickhouse client
-# clickhouse server
-# ...
-
-set -e
-CMD=$1
-shift
-clickhouse-$CMD $*
diff --git a/programs/client/CMakeLists.txt b/programs/client/CMakeLists.txt
index d212da59908..e160355ef7b 100644
--- a/programs/client/CMakeLists.txt
+++ b/programs/client/CMakeLists.txt
@@ -13,6 +13,10 @@ set (CLICKHOUSE_CLIENT_LINK
string_utils
)
+if (TARGET ch_rust::skim)
+ list(APPEND CLICKHOUSE_CLIENT_LINK PRIVATE ch_rust::skim)
+endif()
+
# Always use internal readpassphrase
list(APPEND CLICKHOUSE_CLIENT_LINK PRIVATE readpassphrase)
diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp
index 9923b8b365a..af1a019e1f8 100644
--- a/programs/client/Client.cpp
+++ b/programs/client/Client.cpp
@@ -30,9 +30,10 @@
#include
#include
-#include
-#include
#include
+#include
+#include
+#include
#include
#include
@@ -41,6 +42,8 @@
#include
#include
+#include
+
#include
#include
@@ -827,6 +830,20 @@ bool Client::processWithFuzzing(const String & full_query)
WriteBufferFromOStream ast_buf(std::cout, 4096);
formatAST(*query, ast_buf, false /*highlight*/);
ast_buf.next();
+ if (const auto * insert = query->as())
+ {
+ /// For inserts with data it's really useful to have the data itself available in the logs, as formatAST doesn't print it
+ if (insert->hasInlinedData())
+ {
+ String bytes;
+ {
+ auto read_buf = getReadBufferFromASTInsertQuery(query);
+ WriteBufferFromString write_buf(bytes);
+ copyData(*read_buf, write_buf);
+ }
+ std::cout << std::endl << bytes;
+ }
+ }
std::cout << std::endl << std::endl;
try
diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp
index de85572d5c6..d568012bb26 100644
--- a/programs/install/Install.cpp
+++ b/programs/install/Install.cpp
@@ -1051,18 +1051,12 @@ namespace
return pid;
}
- int stop(const fs::path & pid_file, bool force, bool do_not_kill, unsigned max_tries)
+ bool sendSignalAndWaitForStop(const fs::path & pid_file, int signal, unsigned max_tries, unsigned wait_ms, const char * signal_name)
{
- if (force && do_not_kill)
- throw Exception(ErrorCodes::BAD_ARGUMENTS, "Specified flags are incompatible");
-
int pid = isRunning(pid_file);
if (!pid)
- return 0;
-
- int signal = force ? SIGKILL : SIGTERM;
- const char * signal_name = force ? "kill" : "terminate";
+ return true;
if (0 == kill(pid, signal))
fmt::print("Sent {} signal to process with pid {}.\n", signal_name, pid);
@@ -1078,46 +1072,51 @@ namespace
fmt::print("Server stopped\n");
break;
}
- sleepForSeconds(1);
+ sleepForMilliseconds(wait_ms);
}
- if (try_num == max_tries)
+ return try_num < max_tries;
+ }
+
+ int stop(const fs::path & pid_file, bool force, bool do_not_kill, unsigned max_tries)
+ {
+ if (force && do_not_kill)
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "Specified flags are incompatible");
+
+ int signal = force ? SIGKILL : SIGTERM;
+ const char * signal_name = force ? "kill" : "terminate";
+
+ if (sendSignalAndWaitForStop(pid_file, signal, max_tries, 1000, signal_name))
+ return 0;
+
+ int pid = isRunning(pid_file);
+ if (!pid)
+ return 0;
+
+ if (do_not_kill)
{
- if (do_not_kill)
- {
- fmt::print("Process (pid = {}) is still running. Will not try to kill it.\n", pid);
- return 1;
- }
-
- fmt::print("Will terminate forcefully (pid = {}).\n", pid);
- if (0 == kill(pid, 9))
- fmt::print("Sent kill signal (pid = {}).\n", pid);
- else
- throwFromErrno("Cannot send kill signal", ErrorCodes::SYSTEM_ERROR);
-
- /// Wait for the process (100 seconds).
- constexpr size_t num_kill_check_tries = 1000;
- constexpr size_t kill_check_delay_ms = 100;
- for (size_t i = 0; i < num_kill_check_tries; ++i)
- {
- fmt::print("Waiting for server to be killed\n");
- if (!isRunning(pid_file))
- {
- fmt::print("Server exited\n");
- break;
- }
- sleepForMilliseconds(kill_check_delay_ms);
- }
-
- if (isRunning(pid_file))
- {
- throw Exception(ErrorCodes::CANNOT_KILL,
- "The server process still exists after {} tries (delay: {} ms)",
- num_kill_check_tries, kill_check_delay_ms);
- }
+ fmt::print("Process (pid = {}) is still running. Will not try to kill it.\n", pid);
+ return 1;
}
- return 0;
+ /// Send termination signal again, the server will receive it and immediately terminate.
+ fmt::print("Will send the termination signal again to force the termination (pid = {}).\n", pid);
+ if (sendSignalAndWaitForStop(pid_file, signal, std::min(10U, max_tries), 1000, signal_name))
+ return 0;
+
+ /// Send kill signal. Total wait is 100 seconds.
+ constexpr size_t num_kill_check_tries = 1000;
+ constexpr size_t kill_check_delay_ms = 100;
+ fmt::print("Will terminate forcefully (pid = {}).\n", pid);
+ if (sendSignalAndWaitForStop(pid_file, SIGKILL, num_kill_check_tries, kill_check_delay_ms, signal_name))
+ return 0;
+
+ if (!isRunning(pid_file))
+ return 0;
+
+ throw Exception(ErrorCodes::CANNOT_KILL,
+ "The server process still exists after {} tries (delay: {} ms)",
+ num_kill_check_tries, kill_check_delay_ms);
}
}
diff --git a/programs/local/CMakeLists.txt b/programs/local/CMakeLists.txt
index ad4406156c4..6943af48ab9 100644
--- a/programs/local/CMakeLists.txt
+++ b/programs/local/CMakeLists.txt
@@ -14,8 +14,10 @@ set (CLICKHOUSE_LOCAL_LINK
clickhouse_program_add(local)
-if(NOT CLICKHOUSE_ONE_SHARED)
- target_link_libraries(clickhouse-local-lib PRIVATE clickhouse-server-lib)
+target_link_libraries(clickhouse-local-lib PRIVATE clickhouse-server-lib)
+
+if (TARGET ch_rust::skim)
+ target_link_libraries(clickhouse-local-lib PRIVATE ch_rust::skim)
endif()
# Always use internal readpassphrase
diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp
index 1614fb1a8b4..8ce87558630 100644
--- a/programs/local/LocalServer.cpp
+++ b/programs/local/LocalServer.cpp
@@ -207,7 +207,7 @@ void LocalServer::tryInitPath()
global_context->setPath(path);
- global_context->setTemporaryStorage(path + "tmp", "", 0);
+ global_context->setTemporaryStoragePath(path + "tmp/", 0);
global_context->setFlagsPath(path + "flags");
global_context->setUserFilesPath(""); // user's files are everywhere
diff --git a/programs/main.cpp b/programs/main.cpp
index f40bafc7027..389eae92091 100644
--- a/programs/main.cpp
+++ b/programs/main.cpp
@@ -345,7 +345,7 @@ struct Checker
;
-#if !defined(DISABLE_HARMFUL_ENV_VAR_CHECK) && !defined(USE_MUSL)
+#if !defined(USE_MUSL)
/// NOTE: We will migrate to full static linking or our own dynamic loader to make this code obsolete.
void checkHarmfulEnvironmentVariables(char ** argv)
{
@@ -457,7 +457,7 @@ int main(int argc_, char ** argv_)
/// Note: we forbid dlopen in our code.
updatePHDRCache();
-#if !defined(DISABLE_HARMFUL_ENV_VAR_CHECK) && !defined(USE_MUSL)
+#if !defined(USE_MUSL)
checkHarmfulEnvironmentVariables(argv_);
#endif
diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp
index f98b33c7f87..a4ddd8553c6 100644
--- a/programs/server/Server.cpp
+++ b/programs/server/Server.cpp
@@ -205,46 +205,6 @@ int mainEntryClickHouseServer(int argc, char ** argv)
namespace
{
-void setupTmpPath(Poco::Logger * log, const std::string & path)
-try
-{
- LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
-
- fs::create_directories(path);
-
- /// Clearing old temporary files.
- fs::directory_iterator dir_end;
- size_t unknown_files = 0;
- for (fs::directory_iterator it(path); it != dir_end; ++it)
- {
- if (it->is_regular_file() && startsWith(it->path().filename(), "tmp"))
- {
- LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
- fs::remove(it->path());
- }
- else
- {
- unknown_files++;
- if (unknown_files < 100)
- LOG_DEBUG(log, "Found unknown {} {} in temporary path",
- it->is_regular_file() ? "file" : (it->is_directory() ? "directory" : "element"),
- it->path().string());
- }
- }
-
- if (unknown_files)
- LOG_DEBUG(log, "Found {} unknown files in temporary path", unknown_files);
-}
-catch (...)
-{
- DB::tryLogCurrentException(
- log,
- fmt::format(
- "Caught exception while setup temporary path: {}. It is ok to skip this exception as cleaning old temporary files is not "
- "necessary",
- path));
-}
-
size_t waitServersToFinish(std::vector & servers, size_t seconds_to_wait)
{
const size_t sleep_max_ms = 1000 * seconds_to_wait;
@@ -460,6 +420,33 @@ void Server::createServer(
}
}
+
+#if defined(OS_LINUX)
+namespace
+{
+
+void setOOMScore(int value, Poco::Logger * log)
+{
+ try
+ {
+ std::string value_string = std::to_string(value);
+ DB::WriteBufferFromFile buf("/proc/self/oom_score_adj");
+ buf.write(value_string.c_str(), value_string.size());
+ buf.next();
+ buf.close();
+ }
+ catch (const Poco::Exception & e)
+ {
+ LOG_WARNING(log, "Failed to adjust OOM score: '{}'.", e.displayText());
+ return;
+ }
+ LOG_INFO(log, "Set OOM score adjustment to {}", value);
+}
+
+}
+#endif
+
+
void Server::uninitialize()
{
logger().information("shutting down");
@@ -743,6 +730,13 @@ try
global_context->addWarningMessage("Server was built with sanitizer. It will work slowly.");
#endif
+ const auto memory_amount = getMemoryAmount();
+
+ LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.",
+ formatReadableSizeWithBinarySuffix(memory_amount),
+ getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores
+ std::thread::hardware_concurrency());
+
sanityChecks(*this);
// Initialize global thread pool. Do it before we fetch configs from zookeeper
@@ -816,8 +810,6 @@ try
Settings::checkNoSettingNamesAtTopLevel(config(), config_path);
- const auto memory_amount = getMemoryAmount();
-
#if defined(OS_LINUX)
std::string executable_path = getExecutablePath();
@@ -916,6 +908,21 @@ try
}
}
}
+
+ int default_oom_score = 0;
+
+#if !defined(NDEBUG)
+ /// In debug version on Linux, increase oom score so that clickhouse is killed
+ /// first, instead of some service. Use a carefully chosen random score of 555:
+ /// the maximum is 1000, and chromium uses 300 for its tab processes. Ignore
+ /// whatever errors that occur, because it's just a debugging aid and we don't
+ /// care if it breaks.
+ default_oom_score = 555;
+#endif
+
+ int oom_score = config().getInt("oom_score", default_oom_score);
+ if (oom_score)
+ setOOMScore(oom_score, log);
#endif
global_context->setRemoteHostFilter(config());
@@ -1013,13 +1020,21 @@ try
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone());
/// Storage with temporary data for processing of heavy queries.
+ if (auto temporary_policy = config().getString("tmp_policy", ""); !temporary_policy.empty())
+ {
+ size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0);
+ global_context->setTemporaryStoragePolicy(temporary_policy, max_size);
+ }
+ else if (auto temporary_cache = config().getString("temporary_data_in_cache", ""); !temporary_cache.empty())
+ {
+ size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0);
+ global_context->setTemporaryStorageInCache(temporary_cache, max_size);
+ }
+ else
{
std::string temporary_path = config().getString("tmp_path", path / "tmp/");
- std::string temporary_policy = config().getString("tmp_policy", "");
size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0);
- const VolumePtr & volume = global_context->setTemporaryStorage(temporary_path, temporary_policy, max_size);
- for (const DiskPtr & disk : volume->getDisks())
- setupTmpPath(log, disk->getPath());
+ global_context->setTemporaryStoragePath(temporary_path, max_size);
}
/** Directory with 'flags': files indicating temporary settings for the server set by system administrator.
@@ -1076,8 +1091,8 @@ try
bool continue_if_corrupted = config().getBool("merge_tree_metadata_cache.continue_if_corrupted", false);
try
{
- LOG_DEBUG(
- log, "Initializing merge tree metadata cache lru_cache_size:{} continue_if_corrupted:{}", size, continue_if_corrupted);
+ LOG_DEBUG(log, "Initializing MergeTree metadata cache, lru_cache_size: {} continue_if_corrupted: {}",
+ ReadableSize(size), continue_if_corrupted);
global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size);
}
catch (...)
@@ -1426,7 +1441,7 @@ try
}
catch (...)
{
- tryLogCurrentException(log);
+ tryLogCurrentException(log, "Caught exception while setting up access control.");
throw;
}
@@ -1750,13 +1765,6 @@ try
main_config_reloader->start();
access_control.startPeriodicReloading();
- {
- LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.",
- formatReadableSizeWithBinarySuffix(memory_amount),
- getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores
- std::thread::hardware_concurrency());
- }
-
/// try to load dictionaries immediately, throw on error and die
try
{
diff --git a/programs/server/config.d/graphite.xml b/programs/server/config.d/graphite.xml
new file mode 120000
index 00000000000..69a0411e243
--- /dev/null
+++ b/programs/server/config.d/graphite.xml
@@ -0,0 +1 @@
+../../../tests/config/config.d/graphite.xml
\ No newline at end of file
diff --git a/programs/server/config.xml b/programs/server/config.xml
index 0cbc3d9339e..5f4a9fb983f 100644
--- a/programs/server/config.xml
+++ b/programs/server/config.xml
@@ -1464,4 +1464,8 @@
I don't recommend to change this setting.
false
-->
+
+
diff --git a/rust/CMakeLists.txt b/rust/CMakeLists.txt
index bf62fcbb151..1f11423a557 100644
--- a/rust/CMakeLists.txt
+++ b/rust/CMakeLists.txt
@@ -39,5 +39,21 @@ function(clickhouse_import_crate)
corrosion_import_crate(NO_STD ${ARGN})
endfunction()
-add_subdirectory (BLAKE3)
-add_subdirectory (skim)
+# Add crate from the build directory.
+#
+# Our crates has configuration files:
+# - config for cargo (see config.toml.in)
+# - and possibly config for build (build.rs.in)
+#
+# And to avoid overlaps different builds for one source directory, crate will
+# be copied from source directory to the binary directory.
+file(COPY ".cargo" DESTINATION "${CMAKE_CURRENT_BINARY_DIR}")
+function(add_rust_subdirectory src)
+ set(dst "${CMAKE_CURRENT_BINARY_DIR}/${src}")
+ message(STATUS "Copy ${src} to ${dst}")
+ file(COPY "${src}" DESTINATION "${CMAKE_CURRENT_BINARY_DIR}")
+ add_subdirectory("${dst}" "${dst}")
+endfunction()
+
+add_rust_subdirectory (BLAKE3)
+add_rust_subdirectory (skim)
diff --git a/rust/skim/CMakeLists.txt b/rust/skim/CMakeLists.txt
index e626dd5742e..1e7a43aba7c 100644
--- a/rust/skim/CMakeLists.txt
+++ b/rust/skim/CMakeLists.txt
@@ -35,15 +35,7 @@ add_custom_command(OUTPUT ${ffi_binding_final_path}
DEPENDS cargo-build__ch_rust_skim_rust)
add_library(_ch_rust_skim_ffi ${ffi_binding_final_path})
-if (USE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES)
- # static
-else()
- if (OS_DARWIN)
- target_link_libraries(_ch_rust_skim_ffi PRIVATE -Wl,-undefined,dynamic_lookup)
- else()
- target_link_libraries(_ch_rust_skim_ffi PRIVATE -Wl,--unresolved-symbols=ignore-all)
- endif()
-endif()
+
# cxx bridge compiles such bindings
set_target_properties(_ch_rust_skim_ffi PROPERTIES COMPILE_FLAGS "${CXXBRIDGE_CXXFLAGS}")
diff --git a/rust/skim/build.rs.in b/rust/skim/build.rs.in
index 0135c93222f..f0dd49e4130 100644
--- a/rust/skim/build.rs.in
+++ b/rust/skim/build.rs.in
@@ -5,4 +5,5 @@ fn main() {
}
build.compile("skim");
println!("cargo:rerun-if-changed=src/lib.rs");
+ println!("cargo:rerun-if-changed=.cargo/config.toml");
}
diff --git a/rust/skim/include/skim.h b/rust/skim/include/skim.h
index 12cd257567b..8148474eba3 100644
--- a/rust/skim/include/skim.h
+++ b/rust/skim/include/skim.h
@@ -87,4 +87,4 @@ private:
} // namespace cxxbridge1
} // namespace rust
-::rust::String skim(::std::vector<::std::string> const &words);
+::rust::String skim(::std::string const &prefix, ::std::vector<::std::string> const &words);
diff --git a/rust/skim/src/lib.rs b/rust/skim/src/lib.rs
index 29160329287..90f39cc8382 100644
--- a/rust/skim/src/lib.rs
+++ b/rust/skim/src/lib.rs
@@ -5,7 +5,7 @@ use cxx::{CxxString, CxxVector};
#[cxx::bridge]
mod ffi {
extern "Rust" {
- fn skim(words: &CxxVector) -> Result;
+ fn skim(prefix: &CxxString, words: &CxxVector) -> Result;
}
}
@@ -18,7 +18,7 @@ impl SkimItem for Item {
}
}
-fn skim(words: &CxxVector) -> Result {
+fn skim(prefix: &CxxString, words: &CxxVector) -> Result {
// Let's check is terminal available. To avoid panic.
if let Err(err) = TermInfo::from_env() {
return Err(format!("{}", err));
@@ -26,6 +26,7 @@ fn skim(words: &CxxVector) -> Result {
let options = SkimOptionsBuilder::default()
.height(Some("30%"))
+ .query(Some(prefix.to_str().unwrap()))
.tac(true)
.tiebreak(Some("-score".to_string()))
.build()
diff --git a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp
index 432b1f39f84..359c6051abb 100644
--- a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp
+++ b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp
@@ -49,14 +49,16 @@ private:
public:
AggregateFunctionThrow(const DataTypes & argument_types_, const Array & parameters_, Float64 throw_probability_)
- : IAggregateFunctionDataHelper(argument_types_, parameters_), throw_probability(throw_probability_) {}
+ : IAggregateFunctionDataHelper(argument_types_, parameters_, createResultType())
+ , throw_probability(throw_probability_)
+ {}
String getName() const override
{
return "aggThrow";
}
- DataTypePtr getReturnType() const override
+ static DataTypePtr createResultType()
{
return std::make_shared();
}
diff --git a/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.h b/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.h
index e891fb191f6..da060ceb18e 100644
--- a/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.h
+++ b/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.h
@@ -37,10 +37,10 @@ class AggregateFunctionAnalysisOfVariance final : public IAggregateFunctionDataH
{
public:
explicit AggregateFunctionAnalysisOfVariance(const DataTypes & arguments, const Array & params)
- : IAggregateFunctionDataHelper(arguments, params)
+ : IAggregateFunctionDataHelper(arguments, params, createResultType())
{}
- DataTypePtr getReturnType() const override
+ DataTypePtr createResultType() const
{
DataTypes types {std::make_shared>(), std::make_shared>() };
Strings names {"f_statistic", "p_value"};
diff --git a/src/AggregateFunctions/AggregateFunctionArgMinMax.h b/src/AggregateFunctions/AggregateFunctionArgMinMax.h
index decb572b019..568b70fe77e 100644
--- a/src/AggregateFunctions/AggregateFunctionArgMinMax.h
+++ b/src/AggregateFunctions/AggregateFunctionArgMinMax.h
@@ -38,7 +38,6 @@ template
class AggregateFunctionArgMinMax final : public IAggregateFunctionDataHelper>
{
private:
- const DataTypePtr & type_res;
const DataTypePtr & type_val;
const SerializationPtr serialization_res;
const SerializationPtr serialization_val;
@@ -47,10 +46,9 @@ private:
public:
AggregateFunctionArgMinMax(const DataTypePtr & type_res_, const DataTypePtr & type_val_)
- : Base({type_res_, type_val_}, {})
- , type_res(this->argument_types[0])
+ : Base({type_res_, type_val_}, {}, type_res_)
, type_val(this->argument_types[1])
- , serialization_res(type_res->getDefaultSerialization())
+ , serialization_res(type_res_->getDefaultSerialization())
, serialization_val(type_val->getDefaultSerialization())
{
if (!type_val->isComparable())
@@ -63,11 +61,6 @@ public:
return StringRef(Data::ValueData_t::name()) == StringRef("min") ? "argMin" : "argMax";
}
- DataTypePtr getReturnType() const override
- {
- return type_res;
- }
-
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
if (this->data(place).value.changeIfBetter(*columns[1], row_num, arena))
diff --git a/src/AggregateFunctions/AggregateFunctionArray.h b/src/AggregateFunctions/AggregateFunctionArray.h
index c6e29e77318..c0e676c33e7 100644
--- a/src/AggregateFunctions/AggregateFunctionArray.h
+++ b/src/AggregateFunctions/AggregateFunctionArray.h
@@ -30,7 +30,7 @@ private:
public:
AggregateFunctionArray(AggregateFunctionPtr nested_, const DataTypes & arguments, const Array & params_)
- : IAggregateFunctionHelper(arguments, params_)
+ : IAggregateFunctionHelper(arguments, params_, createResultType(nested_))
, nested_func(nested_), num_arguments(arguments.size())
{
assert(parameters == nested_func->getParameters());
@@ -44,9 +44,9 @@ public:
return nested_func->getName() + "Array";
}
- DataTypePtr getReturnType() const override
+ static DataTypePtr createResultType(const AggregateFunctionPtr & nested_)
{
- return nested_func->getReturnType();
+ return nested_->getResultType();
}
const IAggregateFunction & getBaseAggregateFunctionWithSameStateRepresentation() const override
diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h
index ee46a40023d..a86c7d042fc 100644
--- a/src/AggregateFunctions/AggregateFunctionAvg.h
+++ b/src/AggregateFunctions/AggregateFunctionAvg.h
@@ -10,6 +10,7 @@
#include
#include
#include
+#include
#include "config.h"
@@ -83,10 +84,20 @@ public:
using Fraction = AvgFraction;
explicit AggregateFunctionAvgBase(const DataTypes & argument_types_,
- UInt32 num_scale_ = 0, UInt32 denom_scale_ = 0)
- : Base(argument_types_, {}), num_scale(num_scale_), denom_scale(denom_scale_) {}
+ UInt32 num_scale_ = 0, UInt32 denom_scale_ = 0)
+ : Base(argument_types_, {}, createResultType())
+ , num_scale(num_scale_)
+ , denom_scale(denom_scale_)
+ {}
- DataTypePtr getReturnType() const override { return std::make_shared>(); }
+ AggregateFunctionAvgBase(const DataTypes & argument_types_, const DataTypePtr & result_type_,
+ UInt32 num_scale_ = 0, UInt32 denom_scale_ = 0)
+ : Base(argument_types_, {}, result_type_)
+ , num_scale(num_scale_)
+ , denom_scale(denom_scale_)
+ {}
+
+ DataTypePtr createResultType() const { return std::make_shared>(); }
bool allocatesMemoryInArena() const override { return false; }
@@ -135,7 +146,7 @@ public:
for (const auto & argument : this->argument_types)
can_be_compiled &= canBeNativeType(*argument);
- auto return_type = getReturnType();
+ auto return_type = this->getResultType();
can_be_compiled &= canBeNativeType(*return_type);
return can_be_compiled;
diff --git a/src/AggregateFunctions/AggregateFunctionBitwise.h b/src/AggregateFunctions/AggregateFunctionBitwise.h
index b8d3bc79007..6c94a72bf32 100644
--- a/src/AggregateFunctions/AggregateFunctionBitwise.h
+++ b/src/AggregateFunctions/AggregateFunctionBitwise.h
@@ -97,11 +97,12 @@ class AggregateFunctionBitwise final : public IAggregateFunctionDataHelper>({type}, {}) {}
+ : IAggregateFunctionDataHelper>({type}, {}, createResultType())
+ {}
String getName() const override { return Data::name(); }
- DataTypePtr getReturnType() const override
+ static DataTypePtr createResultType()
{
return std::make_shared>();
}
@@ -137,7 +138,7 @@ public:
bool isCompilable() const override
{
- auto return_type = getReturnType();
+ auto return_type = this->getResultType();
return canBeNativeType(*return_type);
}
@@ -151,7 +152,7 @@ public:
{
llvm::IRBuilder<> & b = static_cast &>(builder);
- auto * return_type = toNativeType(b, getReturnType());
+ auto * return_type = toNativeType(b, this->getResultType());
auto * value_ptr = aggregate_data_ptr;
auto * value = b.CreateLoad(return_type, value_ptr);
@@ -166,7 +167,7 @@ public:
{
llvm::IRBuilder<> & b = static_cast &>(builder);
- auto * return_type = toNativeType(b, getReturnType());
+ auto * return_type = toNativeType(b, this->getResultType());
auto * value_dst_ptr = aggregate_data_dst_ptr;
auto * value_dst = b.CreateLoad(return_type, value_dst_ptr);
@@ -183,7 +184,7 @@ public:
{
llvm::IRBuilder<> & b = static_cast &>(builder);
- auto * return_type = toNativeType(b, getReturnType());
+ auto * return_type = toNativeType(b, this->getResultType());
auto * value_ptr = aggregate_data_ptr;
return b.CreateLoad(return_type, value_ptr);
diff --git a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h
index 34e3fa2f747..8fca88889b8 100644
--- a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h
+++ b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h
@@ -112,7 +112,7 @@ public:
}
explicit AggregateFunctionBoundingRatio(const DataTypes & arguments)
- : IAggregateFunctionDataHelper(arguments, {})
+ : IAggregateFunctionDataHelper(arguments, {}, std::make_shared())
{
const auto * x_arg = arguments.at(0).get();
const auto * y_arg = arguments.at(1).get();
@@ -122,11 +122,6 @@ public:
ErrorCodes::BAD_ARGUMENTS);
}
- DataTypePtr getReturnType() const override
- {
- return std::make_shared();
- }
-
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
diff --git a/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.cpp b/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.cpp
index 93b5de0c5ab..65dce832789 100644
--- a/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.cpp
+++ b/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.cpp
@@ -46,9 +46,9 @@ private:
}
public:
- AggregateFunctionCategoricalIV(const DataTypes & arguments_, const Array & params_) :
- IAggregateFunctionHelper{arguments_, params_},
- category_count{arguments_.size() - 1}
+ AggregateFunctionCategoricalIV(const DataTypes & arguments_, const Array & params_)
+ : IAggregateFunctionHelper{arguments_, params_, createResultType()}
+ , category_count{arguments_.size() - 1}
{
// notice: argument types has been checked before
}
@@ -121,7 +121,7 @@ public:
buf.readStrict(place, sizeOfData());
}
- DataTypePtr getReturnType() const override
+ static DataTypePtr createResultType()
{
return std::make_shared(
std::make_shared>());
diff --git a/src/AggregateFunctions/AggregateFunctionCount.h b/src/AggregateFunctions/AggregateFunctionCount.h
index 6e2c86f065b..91409463409 100644
--- a/src/AggregateFunctions/AggregateFunctionCount.h
+++ b/src/AggregateFunctions/AggregateFunctionCount.h
@@ -39,11 +39,13 @@ namespace ErrorCodes
class AggregateFunctionCount final : public IAggregateFunctionDataHelper
{
public:
- explicit AggregateFunctionCount(const DataTypes & argument_types_) : IAggregateFunctionDataHelper(argument_types_, {}) {}
+ explicit AggregateFunctionCount(const DataTypes & argument_types_)
+ : IAggregateFunctionDataHelper(argument_types_, {}, createResultType())
+ {}
String getName() const override { return "count"; }
- DataTypePtr getReturnType() const override
+ static DataTypePtr createResultType()
{
return std::make_shared();
}
@@ -167,7 +169,7 @@ public:
{
llvm::IRBuilder<> & b = static_cast &>(builder);
- auto * return_type = toNativeType(b, getReturnType());
+ auto * return_type = toNativeType(b, this->getResultType());
auto * count_value_ptr = aggregate_data_ptr;
auto * count_value = b.CreateLoad(return_type, count_value_ptr);
@@ -180,7 +182,7 @@ public:
{
llvm::IRBuilder<> & b = static_cast &>(builder);
- auto * return_type = toNativeType(b, getReturnType());
+ auto * return_type = toNativeType(b, this->getResultType());
auto * count_value_dst_ptr = aggregate_data_dst_ptr;
auto * count_value_dst = b.CreateLoad(return_type, count_value_dst_ptr);
@@ -197,7 +199,7 @@ public:
{
llvm::IRBuilder<> & b = static_cast &>(builder);
- auto * return_type = toNativeType(b, getReturnType());
+ auto * return_type = toNativeType(b, this->getResultType());
auto * count_value_ptr = aggregate_data_ptr;
return b.CreateLoad(return_type, count_value_ptr);
@@ -214,7 +216,7 @@ class AggregateFunctionCountNotNullUnary final
{
public:
AggregateFunctionCountNotNullUnary(const DataTypePtr & argument, const Array & params)
- : IAggregateFunctionDataHelper({argument}, params)
+ : IAggregateFunctionDataHelper({argument}, params, createResultType())
{
if (!argument->isNullable())
throw Exception("Logical error: not Nullable data type passed to AggregateFunctionCountNotNullUnary", ErrorCodes::LOGICAL_ERROR);
@@ -222,7 +224,7 @@ public:
String getName() const override { return "count"; }
- DataTypePtr getReturnType() const override
+ static DataTypePtr createResultType()
{
return std::make_shared();
}
@@ -311,7 +313,7 @@ public:
{
llvm::IRBuilder<> & b = static_cast &>(builder);
- auto * return_type = toNativeType(b, getReturnType());
+ auto * return_type = toNativeType(b, this->getResultType());
auto * is_null_value = b.CreateExtractValue(values[0], {1});
auto * increment_value = b.CreateSelect(is_null_value, llvm::ConstantInt::get(return_type, 0), llvm::ConstantInt::get(return_type, 1));
@@ -327,7 +329,7 @@ public:
{
llvm::IRBuilder<> & b = static_cast &>(builder);
- auto * return_type = toNativeType(b, getReturnType());
+ auto * return_type = toNativeType(b, this->getResultType());
auto * count_value_dst_ptr = aggregate_data_dst_ptr;
auto * count_value_dst = b.CreateLoad(return_type, count_value_dst_ptr);
@@ -344,7 +346,7 @@ public:
{
llvm::IRBuilder<> & b = static_cast &>(builder);
- auto * return_type = toNativeType(b, getReturnType());
+ auto * return_type = toNativeType(b, this->getResultType());
auto * count_value_ptr = aggregate_data_ptr;
return b.CreateLoad(return_type, count_value_ptr);
diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSum.h b/src/AggregateFunctions/AggregateFunctionDeltaSum.h
index 36d0ef55346..199d2706d3a 100644
--- a/src/AggregateFunctions/AggregateFunctionDeltaSum.h
+++ b/src/AggregateFunctions/AggregateFunctionDeltaSum.h
@@ -31,7 +31,7 @@ class AggregationFunctionDeltaSum final
{
public:
AggregationFunctionDeltaSum(const DataTypes & arguments, const Array & params)
- : IAggregateFunctionDataHelper, AggregationFunctionDeltaSum>{arguments, params}
+ : IAggregateFunctionDataHelper, AggregationFunctionDeltaSum>{arguments, params, createResultType()}
{}
AggregationFunctionDeltaSum()
@@ -40,7 +40,7 @@ public:
String getName() const override { return "deltaSum"; }
- DataTypePtr getReturnType() const override { return std::make_shared>(); }
+ static DataTypePtr createResultType() { return std::make_shared>(); }
bool allocatesMemoryInArena() const override { return false; }
diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h
index a311910de7f..5ca07bb0bdf 100644
--- a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h
+++ b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h
@@ -38,7 +38,7 @@ public:
: IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData,
AggregationFunctionDeltaSumTimestamp
- >{arguments, params}
+ >{arguments, params, createResultType()}
{}
AggregationFunctionDeltaSumTimestamp()
@@ -52,7 +52,7 @@ public:
String getName() const override { return "deltaSumTimestamp"; }
- DataTypePtr getReturnType() const override { return std::make_shared>(); }
+ static DataTypePtr createResultType() { return std::make_shared>(); }
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
diff --git a/src/AggregateFunctions/AggregateFunctionDistinct.h b/src/AggregateFunctions/AggregateFunctionDistinct.h
index 2d7362ba4cc..e09e0ef621d 100644
--- a/src/AggregateFunctions/AggregateFunctionDistinct.h
+++ b/src/AggregateFunctions/AggregateFunctionDistinct.h
@@ -168,7 +168,7 @@ private:
public:
AggregateFunctionDistinct(AggregateFunctionPtr nested_func_, const DataTypes & arguments, const Array & params_)
- : IAggregateFunctionDataHelper(arguments, params_)
+ : IAggregateFunctionDataHelper(arguments, params_, nested_func_->getResultType())
, nested_func(nested_func_)
, arguments_num(arguments.size())
{
@@ -255,11 +255,6 @@ public:
return nested_func->getName() + "Distinct";
}
- DataTypePtr getReturnType() const override
- {
- return nested_func->getReturnType();
- }
-
bool allocatesMemoryInArena() const override
{
return true;
diff --git a/src/AggregateFunctions/AggregateFunctionEntropy.h b/src/AggregateFunctions/AggregateFunctionEntropy.h
index a51dd0537bf..9321b5c5825 100644
--- a/src/AggregateFunctions/AggregateFunctionEntropy.h
+++ b/src/AggregateFunctions/AggregateFunctionEntropy.h
@@ -92,14 +92,14 @@ private:
public:
explicit AggregateFunctionEntropy(const DataTypes & argument_types_)
- : IAggregateFunctionDataHelper, AggregateFunctionEntropy>(argument_types_, {})
+ : IAggregateFunctionDataHelper, AggregateFunctionEntropy>(argument_types_, {}, createResultType())
, num_args(argument_types_.size())
{
}
String getName() const override { return "entropy"; }
- DataTypePtr getReturnType() const override
+ static DataTypePtr createResultType()
{
return std::make_shared>();
}
diff --git a/src/AggregateFunctions/AggregateFunctionExponentialMovingAverage.cpp b/src/AggregateFunctions/AggregateFunctionExponentialMovingAverage.cpp
index 2c055c37cca..bb48b3416be 100644
--- a/src/AggregateFunctions/AggregateFunctionExponentialMovingAverage.cpp
+++ b/src/AggregateFunctions/AggregateFunctionExponentialMovingAverage.cpp
@@ -29,7 +29,7 @@ private:
public:
AggregateFunctionExponentialMovingAverage(const DataTypes & argument_types_, const Array & params)
- : IAggregateFunctionDataHelper(argument_types_, params)
+ : IAggregateFunctionDataHelper(argument_types_, params, createResultType())
{
if (params.size() != 1)
throw Exception{"Aggregate function " + getName() + " requires exactly one parameter: half decay time.",
@@ -43,7 +43,7 @@ public:
return "exponentialMovingAverage";
}
- DataTypePtr getReturnType() const override
+ static DataTypePtr createResultType()
{
return std::make_shared>();
}
diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp
index a8385ad8b59..38cc355b857 100644
--- a/src/AggregateFunctions/AggregateFunctionFactory.cpp
+++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp
@@ -72,9 +72,12 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
{
auto types_without_low_cardinality = convertLowCardinalityTypesToNested(argument_types);
- /// If one of the types is Nullable, we apply aggregate function combinator "Null".
-
- if (std::any_of(types_without_low_cardinality.begin(), types_without_low_cardinality.end(),
+ /// If one of the types is Nullable, we apply aggregate function combinator "Null" if it's not window function.
+ /// Window functions are not real aggregate functions. Applying combinators doesn't make sense for them,
+ /// they must handle the nullability themselves
+ auto properties = tryGetPropertiesImpl(name);
+ bool is_window_function = properties.has_value() && properties->is_window_function;
+ if (!is_window_function && std::any_of(types_without_low_cardinality.begin(), types_without_low_cardinality.end(),
[](const auto & type) { return type->isNullable(); }))
{
AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix("Null");
diff --git a/src/AggregateFunctions/AggregateFunctionFlameGraph.cpp b/src/AggregateFunctions/AggregateFunctionFlameGraph.cpp
deleted file mode 100644
index 5fc6b21926e..00000000000
--- a/src/AggregateFunctions/AggregateFunctionFlameGraph.cpp
+++ /dev/null
@@ -1,647 +0,0 @@
-#include
-#include
-#include
-#include
-#include
-#include
-#include
-#include
-#include
-#include
-#include
-#include
-#include
-#include
-#include
-#include
-
-namespace DB
-{
-namespace ErrorCodes
-{
- extern const int FUNCTION_NOT_ALLOWED;
- extern const int NOT_IMPLEMENTED;
- extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
- extern const int ILLEGAL_TYPE_OF_ARGUMENT;
-}
-
-struct AggregateFunctionFlameGraphTree
-{
- struct ListNode;
-
- struct TreeNode
- {
- TreeNode * parent = nullptr;
- ListNode * children = nullptr;
- UInt64 ptr = 0;
- size_t allocated = 0;
- };
-
- struct ListNode
- {
- ListNode * next = nullptr;
- TreeNode * child = nullptr;
- };
-
- TreeNode root;
-
- static ListNode * createChild(TreeNode * parent, UInt64 ptr, Arena * arena)
- {
-
- ListNode * list_node = reinterpret_cast(arena->alloc(sizeof(ListNode)));
- TreeNode * tree_node = reinterpret_cast(arena->alloc(sizeof(TreeNode)));
-
- list_node->child = tree_node;
- list_node->next = nullptr;
-
- tree_node->parent =parent;
- tree_node->children = nullptr;
- tree_node->ptr = ptr;
- tree_node->allocated = 0;
-
- return list_node;
- }
-
- TreeNode * find(const UInt64 * stack, size_t stack_size, Arena * arena)
- {
- TreeNode * node = &root;
- for (size_t i = 0; i < stack_size; ++i)
- {
- UInt64 ptr = stack[i];
- if (ptr == 0)
- break;
-
- if (!node->children)
- {
- node->children = createChild(node, ptr, arena);
- node = node->children->child;
- }
- else
- {
- ListNode * list = node->children;
- while (list->child->ptr != ptr && list->next)
- list = list->next;
-
- if (list->child->ptr != ptr)
- {
- list->next = createChild(node, ptr, arena);
- list = list->next;
- }
-
- node = list->child;
- }
- }
-
- return node;
- }
-
- static void append(DB::PaddedPODArray & values, DB::PaddedPODArray & offsets, std::vector & frame)
- {
- UInt64 prev = offsets.empty() ? 0 : offsets.back();
- offsets.push_back(prev + frame.size());
- for (UInt64 val : frame)
- values.push_back(val);
- }
-
- struct Trace
- {
- using Frames = std::vector;
-
- Frames frames;
-
- /// The total number of bytes allocated for traces with the same prefix.
- size_t allocated_total = 0;
- /// This counter is relevant in case we want to filter some traces with small amount of bytes.
- /// It shows the total number of bytes for *filtered* traces with the same prefix.
- /// This is the value which is used in flamegraph.
- size_t allocated_self = 0;
- };
-
- using Traces = std::vector;
-
- Traces dump(size_t max_depth, size_t min_bytes) const
- {
- Traces traces;
- Trace::Frames frames;
- std::vector allocated_total;
- std::vector allocated_self;
- std::vector nodes;
-
- nodes.push_back(root.children);
- allocated_total.push_back(root.allocated);
- allocated_self.push_back(root.allocated);
-
- while (!nodes.empty())
- {
- if (nodes.back() == nullptr)
- {
- traces.push_back({frames, allocated_total.back(), allocated_self.back()});
-
- nodes.pop_back();
- allocated_total.pop_back();
- allocated_self.pop_back();
-
- /// We don't have root's frame so framers are empty in the end.
- if (!frames.empty())
- frames.pop_back();
-
- continue;
- }
-
- TreeNode * current = nodes.back()->child;
- nodes.back() = nodes.back()->next;
-
- bool enough_bytes = current->allocated >= min_bytes;
- bool enough_depth = max_depth == 0 || nodes.size() < max_depth;
-
- if (enough_bytes)
- {
- frames.push_back(current->ptr);
- allocated_self.back() -= current->allocated;
-
- if (enough_depth)
- {
- allocated_total.push_back(current->allocated);
- allocated_self.push_back(current->allocated);
- nodes.push_back(current->children);
- }
- else
- {
- traces.push_back({frames, current->allocated, current->allocated});
- frames.pop_back();
- }
- }
- }
-
- return traces;
- }
-};
-
-static void insertData(DB::PaddedPODArray & chars, DB::PaddedPODArray & offsets, const char * pos, size_t length)
-{
- const size_t old_size = chars.size();
- const size_t new_size = old_size + length + 1;
-
- chars.resize(new_size);
- if (length)
- memcpy(chars.data() + old_size, pos, length);
- chars[old_size + length] = 0;
- offsets.push_back(new_size);
-}
-
-/// Split str by line feed and write as separate row to ColumnString.
-static void fillColumn(DB::PaddedPODArray & chars, DB::PaddedPODArray & offsets, const std::string & str)
-{
- size_t start = 0;
- size_t end = 0;
- size_t size = str.size();
-
- while (end < size)
- {
- if (str[end] == '\n')
- {
- insertData(chars, offsets, str.data() + start, end - start);
- start = end + 1;
- }
-
- ++end;
- }
-
- if (start < end)
- insertData(chars, offsets, str.data() + start, end - start);
-}
-
-void dumpFlameGraph(
- const AggregateFunctionFlameGraphTree::Traces & traces,
- DB::PaddedPODArray & chars,
- DB::PaddedPODArray & offsets)
-{
- DB::WriteBufferFromOwnString out;
-
- std::unordered_map mapping;
-
-#if defined(__ELF__) && !defined(OS_FREEBSD)
- auto symbol_index_ptr = DB::SymbolIndex::instance();
- const DB::SymbolIndex & symbol_index = *symbol_index_ptr;
-#endif
-
- for (const auto & trace : traces)
- {
- if (trace.allocated_self == 0)
- continue;
-
- for (size_t i = 0; i < trace.frames.size(); ++i)
- {
- if (i)
- out << ";";
-
- const void * ptr = reinterpret_cast(trace.frames[i]);
-
-#if defined(__ELF__) && !defined(OS_FREEBSD)
- if (const auto * symbol = symbol_index.findSymbol(ptr))
- writeString(demangle(symbol->name), out);
- else
- DB::writePointerHex(ptr, out);
-#else
- DB::writePointerHex(ptr, out);
-#endif
- }
-
- out << ' ' << trace.allocated_self << "\n";
- }
-
- fillColumn(chars, offsets, out.str());
-}
-
-struct AggregateFunctionFlameGraphData
-{
- struct Entry
- {
- AggregateFunctionFlameGraphTree::TreeNode * trace;
- UInt64 size;
- Entry * next = nullptr;
- };
-
- struct Pair
- {
- Entry * allocation = nullptr;
- Entry * deallocation = nullptr;
- };
-
- using Entries = HashMap;
-
- AggregateFunctionFlameGraphTree tree;
- Entries entries;
- Entry * free_list = nullptr;
-
- Entry * alloc(Arena * arena)
- {
- if (free_list)
- {
- auto * res = free_list;
- free_list = free_list->next;
- return res;
- }
-
- return reinterpret_cast(arena->alloc(sizeof(Entry)));
- }
-
- void release(Entry * entry)
- {
- entry->next = free_list;
- free_list = entry;
- }
-
- static void track(Entry * allocation)
- {
- auto * node = allocation->trace;
- while (node)
- {
- node->allocated += allocation->size;
- node = node->parent;
- }
- }
-
- static void untrack(Entry * allocation)
- {
- auto * node = allocation->trace;
- while (node)
- {
- node->allocated -= allocation->size;
- node = node->parent;
- }
- }
-
- static Entry * tryFindMatchAndRemove(Entry *& list, UInt64 size)
- {
- if (!list)
- return nullptr;
-
- if (list->size == size)
- {
- Entry * entry = list;
- list = list->next;
- return entry;
- }
- else
- {
- Entry * parent = list;
- while (parent->next && parent->next->size != size)
- parent = parent->next;
-
- if (parent->next && parent->next->size == size)
- {
- Entry * entry = parent->next;
- parent->next = entry->next;
- return entry;
- }
-
- return nullptr;
- }
- }
-
- void add(UInt64 ptr, Int64 size, const UInt64 * stack, size_t stack_size, Arena * arena)
- {
- /// In case if argument is nullptr, only track allocations.
- if (ptr == 0)
- {
- if (size > 0)
- {
- auto * node = tree.find(stack, stack_size, arena);
- Entry entry{.trace = node, .size = UInt64(size)};
- track(&entry);
- }
-
- return;
- }
-
- auto & place = entries[ptr];
- if (size > 0)
- {
- if (auto * deallocation = tryFindMatchAndRemove(place.deallocation, size))
- {
- release(deallocation);
- }
- else
- {
- auto * node = tree.find(stack, stack_size, arena);
-
- auto * allocation = alloc(arena);
- allocation->size = UInt64(size);
- allocation->trace = node;
-
- track(allocation);
-
- allocation->next = place.allocation;
- place.allocation = allocation;
- }
- }
- else if (size < 0)
- {
- UInt64 abs_size = -size;
- if (auto * allocation = tryFindMatchAndRemove(place.allocation, abs_size))
- {
- untrack(allocation);
- release(allocation);
- }
- else
- {
- auto * deallocation = alloc(arena);
- deallocation->size = abs_size;
-
- deallocation->next = place.deallocation;
- place.deallocation = deallocation;
- }
- }
- }
-
- void merge(const AggregateFunctionFlameGraphTree & other_tree, Arena * arena)
- {
- AggregateFunctionFlameGraphTree::Trace::Frames frames;
- std::vector nodes;
-
- nodes.push_back(other_tree.root.children);
-
- while (!nodes.empty())
- {
- if (nodes.back() == nullptr)
- {
- nodes.pop_back();
-
- /// We don't have root's frame so framers are empty in the end.
- if (!frames.empty())
- frames.pop_back();
-
- continue;
- }
-
- AggregateFunctionFlameGraphTree::TreeNode * current = nodes.back()->child;
- nodes.back() = nodes.back()->next;
-
- frames.push_back(current->ptr);
-
- if (current->children)
- nodes.push_back(current->children);
- else
- {
- if (current->allocated)
- add(0, current->allocated, frames.data(), frames.size(), arena);
-
- frames.pop_back();
- }
- }
- }
-
- void merge(const AggregateFunctionFlameGraphData & other, Arena * arena)
- {
- AggregateFunctionFlameGraphTree::Trace::Frames frames;
- for (const auto & entry : other.entries)
- {
- for (auto * allocation = entry.value.second.allocation; allocation; allocation = allocation->next)
- {
- frames.clear();
- const auto * node = allocation->trace;
- while (node->ptr)
- {
- frames.push_back(node->ptr);
- node = node->parent;
- }
-
- std::reverse(frames.begin(), frames.end());
- add(entry.value.first, allocation->size, frames.data(), frames.size(), arena);
- untrack(allocation);
- }
-
- for (auto * deallocation = entry.value.second.deallocation; deallocation; deallocation = deallocation->next)
- {
- add(entry.value.first, -Int64(deallocation->size), nullptr, 0, arena);
- }
- }
-
- merge(other.tree, arena);
- }
-
- void dumpFlameGraph(
- DB::PaddedPODArray & chars,
- DB::PaddedPODArray & offsets,
- size_t max_depth, size_t min_bytes) const
- {
- DB::dumpFlameGraph(tree.dump(max_depth, min_bytes), chars, offsets);
- }
-};
-
-/// Aggregate function which builds a flamegraph using the list of stacktraces.
-/// The output is an array of strings which can be used by flamegraph.pl util.
-/// See https://github.com/brendangregg/FlameGraph
-///
-/// Syntax: flameGraph(traces, [size = 1], [ptr = 0])
-/// - trace : Array(UInt64), a stacktrace
-/// - size : Int64, an allocation size (for memory profiling)
-/// - ptr : UInt64, an allocation address
-/// In case if ptr != 0, a flameGraph will map allocations (size > 0) and deallocations (size < 0) with the same size and ptr.
-/// Only allocations which were not freed are shown. Not mapped deallocations are ignored.
-///
-/// Usage:
-///
-/// * Build a flamegraph based on CPU query profiler
-/// set query_profiler_cpu_time_period_ns=10000000;
-/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;
-/// clickhouse client --allow_introspection_functions=1
-/// -q "select arrayJoin(flameGraph(arrayReverse(trace))) from system.trace_log where trace_type = 'CPU' and query_id = 'xxx'"
-/// | ~/dev/FlameGraph/flamegraph.pl > flame_cpu.svg
-///
-/// * Build a flamegraph based on memory query profiler, showing all allocations
-/// set memory_profiler_sample_probability=1, max_untracked_memory=1;
-/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;
-/// clickhouse client --allow_introspection_functions=1
-/// -q "select arrayJoin(flameGraph(trace, size)) from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx'"
-/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem.svg
-///
-/// * Build a flamegraph based on memory query profiler, showing allocations which were not deallocated in query context
-/// set memory_profiler_sample_probability=1, max_untracked_memory=1, use_uncompressed_cache=1, merge_tree_max_rows_to_use_cache=100000000000, merge_tree_max_bytes_to_use_cache=1000000000000;
-/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;
-/// clickhouse client --allow_introspection_functions=1
-/// -q "select arrayJoin(flameGraph(trace, size, ptr)) from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx'"
-/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_untracked.svg
-///
-/// * Build a flamegraph based on memory query profiler, showing active allocations at the fixed point of time
-/// set memory_profiler_sample_probability=1, max_untracked_memory=1;
-/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;
-/// 1. Memory usage per second
-/// select event_time, m, formatReadableSize(max(s) as m) from (select event_time, sum(size) over (order by event_time) as s from system.trace_log where query_id = 'xxx' and trace_type = 'MemorySample') group by event_time order by event_time;
-/// 2. Find a time point with maximal memory usage
-/// select argMax(event_time, s), max(s) from (select event_time, sum(size) over (order by event_time) as s from system.trace_log where query_id = 'xxx' and trace_type = 'MemorySample');
-/// 3. Fix active allocations at fixed point of time
-/// clickhouse client --allow_introspection_functions=1
-/// -q "select arrayJoin(flameGraph(trace, size, ptr)) from (select * from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx' and event_time <= 'yyy' order by event_time)"
-/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_time_point_pos.svg
-/// 4. Find deallocations at fixed point of time
-/// clickhouse client --allow_introspection_functions=1
-/// -q "select arrayJoin(flameGraph(trace, -size, ptr)) from (select * from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx' and event_time > 'yyy' order by event_time desc)"
-/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_time_point_neg.svg
-class AggregateFunctionFlameGraph final : public IAggregateFunctionDataHelper
-{
-public:
- explicit AggregateFunctionFlameGraph(const DataTypes & argument_types_)
- : IAggregateFunctionDataHelper(argument_types_, {})
- {}
-
- String getName() const override { return "flameGraph"; }
-
- DataTypePtr getReturnType() const override
- {
- return std::make_shared(std::make_shared());
- }
-
- bool allocatesMemoryInArena() const override { return true; }
-
- void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
- {
- const auto * trace = typeid_cast