Merge remote-tracking branch 'upstream/master' into randomize-fs-read-settings

This commit is contained in:
kssenii 2023-04-20 18:22:55 +02:00
commit 0a34f3bcea
614 changed files with 16741 additions and 8971 deletions

View File

@ -7,11 +7,11 @@ tests/ci/run_check.py
### Changelog category (leave one):
- New Feature
- Improvement
- Bug Fix (user-visible misbehavior in an official stable release)
- Performance Improvement
- Backward Incompatible Change
- Build/Testing/Packaging Improvement
- Documentation (changelog entry is not required)
- Bug Fix (user-visible misbehavior in an official stable release)
- Not for changelog (changelog entry is not required)

View File

@ -4846,3 +4846,41 @@ jobs:
cd "$GITHUB_WORKSPACE/tests/ci"
python3 finish_check.py
python3 merge_pr.py --check-approved
##############################################################################################
########################### SQLLOGIC TEST ###################################################
##############################################################################################
SQLLogicTestRelease:
needs: [BuilderDebRelease]
runs-on: [self-hosted, func-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/sqllogic_debug
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Sqllogic test (release)
REPO_COPY=${{runner.temp}}/sqllogic_debug/ClickHouse
KILL_TIMEOUT=10800
EOF
- name: Download json reports
uses: actions/download-artifact@v2
with:
path: ${{ env.REPORTS_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
- name: Sqllogic test
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
python3 sqllogic_test.py "$CHECK_NAME" "$KILL_TIMEOUT"
- name: Cleanup
if: always()
run: |
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"

View File

@ -393,7 +393,11 @@ else()
endif ()
option (ENABLE_GWP_ASAN "Enable Gwp-Asan" ON)
if (NOT OS_LINUX AND NOT OS_ANDROID)
# We use mmap for allocations more heavily in debug builds,
# but GWP-ASan also wants to use mmap frequently,
# and due to a large number of memory mappings,
# it does not work together well.
if ((NOT OS_LINUX AND NOT OS_ANDROID) OR (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG"))
set(ENABLE_GWP_ASAN OFF)
endif ()

View File

@ -22,7 +22,7 @@ curl https://clickhouse.com/ | sh
## Upcoming Events
* [**ClickHouse Spring Meetup in Manhattan**](https://www.meetup.com/clickhouse-new-york-user-group/events/292517734) - April 26 - It's spring, and it's time to meet again in the city! Talks include: "Building a domain specific query language on top of Clickhouse", "A Galaxy of Information", "Our Journey to ClickHouse Cloud from Redshift", and a ClickHouse update!
* [**v23.4 Release Webinar**](https://clickhouse.com/company/events/v23-4-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-04) - April 27 - 23.4 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release.
* [**v23.4 Release Webinar**](https://clickhouse.com/company/events/v23-4-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-04) - April 26 - 23.4 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release.
## Recent Recordings
* **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments"

View File

@ -51,3 +51,15 @@ namespace DB
};
}
namespace std
{
template <>
struct hash<DB::IPv6>
{
size_t operator()(const DB::IPv6 & x) const
{
return std::hash<DB::IPv6::UnderlyingType>()(x.toUnderType());
}
};
}

@ -1 +1 @@
Subproject commit e0accd517933ebb44aff84bc8db448ffd8ef1929
Subproject commit 2aedf7598a4040b23881dbe05b6afaca25a337ef

View File

@ -151,5 +151,9 @@
"name": "clickhouse/docs-builder",
"dependent": [
]
},
"docker/test/sqllogic": {
"name": "clickhouse/sqllogic-test",
"dependent": []
}
}

View File

@ -194,7 +194,12 @@ function build
{
(
cd "$FASTTEST_BUILD"
time ninja clickhouse-bundle 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/build_log.txt"
TIMEFORMAT=$'\nreal\t%3R\nuser\t%3U\nsys\t%3S'
( time ninja clickhouse-bundle) |& ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/build_log.txt"
BUILD_SECONDS_ELAPSED=$(awk '/^....-..-.. ..:..:.. real\t[0-9]/ {print $4}' < "$FASTTEST_OUTPUT/build_log.txt")
echo "build_clickhouse_fasttest_binary: [ OK ] $BUILD_SECONDS_ELAPSED sec." \
| ts '%Y-%m-%d %H:%M:%S' \
| tee "$FASTTEST_OUTPUT/test_result.txt"
if [ "$COPY_CLICKHOUSE_BINARY_TO_OUTPUT" -eq "1" ]; then
cp programs/clickhouse "$FASTTEST_OUTPUT/clickhouse"
@ -251,7 +256,7 @@ function run_tests
)
time clickhouse-test "${test_opts[@]}" -- "$FASTTEST_FOCUS" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee "$FASTTEST_OUTPUT/test_result.txt"
| tee -a "$FASTTEST_OUTPUT/test_result.txt"
set -e
clickhouse stop --pid-path "$FASTTEST_DATA"

View File

@ -47,10 +47,9 @@ ENV TZ=Etc/UTC
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
ENV DOCKER_CHANNEL stable
RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add -
RUN add-apt-repository "deb https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}"
RUN apt-get update \
RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \
&& add-apt-repository "deb https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}" \
&& apt-get update \
&& env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
docker-ce \
&& rm -rf \
@ -61,7 +60,7 @@ RUN apt-get update \
RUN dockerd --version; docker --version
RUN python3 -m pip install \
RUN python3 -m pip install --no-cache-dir \
PyMySQL \
aerospike==4.0.0 \
avro==1.10.2 \
@ -104,8 +103,9 @@ COPY dockerd-entrypoint.sh /usr/local/bin/
COPY compose/ /compose/
COPY misc/ /misc/
RUN wget https://dlcdn.apache.org/spark/spark-3.3.2/spark-3.3.2-bin-hadoop3.tgz \
&& tar xzvf spark-3.3.2-bin-hadoop3.tgz -C /
RUN curl -fsSL -O https://dlcdn.apache.org/spark/spark-3.3.2/spark-3.3.2-bin-hadoop3.tgz \
&& tar xzvf spark-3.3.2-bin-hadoop3.tgz -C / \
&& rm spark-3.3.2-bin-hadoop3.tgz
# download spark and packages
# if you change packages, don't forget to update them in tests/integration/helpers/cluster.py

View File

@ -16,7 +16,9 @@ echo '{
# 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 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log &
# Binding to an IP address without --tlsverify is deprecated. Startup is intentionally being slowed
# unless --tls=false or --tlsverify=false is set
dockerd --host=unix:///var/run/docker.sock --tls=false --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log &
set +e
reties=0

View File

@ -0,0 +1,45 @@
# docker build -t clickhouse/sqllogic-test .
ARG FROM_TAG=latest
FROM clickhouse/test-base:$FROM_TAG
RUN apt-get update --yes \
&& env DEBIAN_FRONTEND=noninteractive \
apt-get install --yes --no-install-recommends \
wget \
git \
python3 \
python3-dev \
python3-pip \
sqlite3 \
unixodbc \
unixodbc-dev \
sudo \
&& apt-get clean
RUN pip3 install \
numpy \
pyodbc \
deepdiff
ARG odbc_repo="https://github.com/ClickHouse/clickhouse-odbc.git"
RUN git clone --recursive ${odbc_repo} \
&& mkdir -p /clickhouse-odbc/build \
&& cmake -S /clickhouse-odbc -B /clickhouse-odbc/build \
&& ls /clickhouse-odbc/build/driver \
&& make -j 10 -C /clickhouse-odbc/build \
&& ls /clickhouse-odbc/build/driver \
&& mkdir -p /usr/local/lib64/ && cp /clickhouse-odbc/build/driver/lib*.so /usr/local/lib64/ \
&& odbcinst -i -d -f /clickhouse-odbc/packaging/odbcinst.ini.sample \
&& odbcinst -i -s -l -f /clickhouse-odbc/packaging/odbc.ini.sample
ENV TZ=Europe/Amsterdam
ENV MAX_RUN_TIME=900
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
ARG sqllogic_test_repo="https://github.com/gregrahn/sqllogictest.git"
RUN git clone --recursive ${sqllogic_test_repo}
COPY run.sh /
CMD ["/bin/bash", "/run.sh"]

100
docker/test/sqllogic/run.sh Executable file
View File

@ -0,0 +1,100 @@
#!/bin/bash
set -exu
trap "exit" INT TERM
echo "ENV"
env
# fail on errors, verbose and export all env variables
set -e -x -a
echo "Current directory"
pwd
echo "Files in current directory"
ls -la ./
echo "Files in root directory"
ls -la /
echo "Files in /clickhouse-tests directory"
ls -la /clickhouse-tests
echo "Files in /clickhouse-tests/sqllogic directory"
ls -la /clickhouse-tests/sqllogic
echo "Files in /package_folder directory"
ls -la /package_folder
echo "Files in /test_output"
ls -la /test_output
echo "File in /sqllogictest"
ls -la /sqllogictest
dpkg -i package_folder/clickhouse-common-static_*.deb
dpkg -i package_folder/clickhouse-common-static-dbg_*.deb
dpkg -i package_folder/clickhouse-server_*.deb
dpkg -i package_folder/clickhouse-client_*.deb
# install test configs
# /clickhouse-tests/config/install.sh
sudo clickhouse start
sleep 5
for _ in $(seq 1 60); do if [[ $(wget --timeout=1 -q 'localhost:8123' -O-) == 'Ok.' ]]; then break ; else sleep 1; fi ; done
function run_tests()
{
set -x
cd /test_output
/clickhouse-tests/sqllogic/runner.py --help 2>&1 \
| ts '%Y-%m-%d %H:%M:%S'
mkdir -p /test_output/self-test
/clickhouse-tests/sqllogic/runner.py --log-file /test_output/runner-self-test.log \
self-test \
--self-test-dir /clickhouse-tests/sqllogic/self-test \
--out-dir /test_output/self-test \
2>&1 \
| ts '%Y-%m-%d %H:%M:%S'
cat /test_output/self-test/check_status.tsv >> /test_output/check_status.tsv
cat /test_output/self-test/test_results.tsv >> /test_output/test_results.tsv ||:
tar -zcvf self-test.tar.gz self-test 1>/dev/null
if [ -d /sqllogictest ]
then
mkdir -p /test_output/statements-test
/clickhouse-tests/sqllogic/runner.py \
--log-file /test_output/runner-statements-test.log \
--log-level info \
statements-test \
--input-dir /sqllogictest \
--out-dir /test_output/statements-test \
2>&1 \
| ts '%Y-%m-%d %H:%M:%S'
cat /test_output/statements-test/check_status.tsv >> /test_output/check_status.tsv
cat /test_output/statements-test/test_results.tsv >> /test_output/test_results.tsv
tar -zcvf statements-check.tar.gz statements-test 1>/dev/null
fi
}
export -f run_tests
timeout "${MAX_RUN_TIME:-9000}" bash -c run_tests || echo "timeout reached" >&2
#/process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv
clickhouse-client -q "system flush logs" ||:
# Stop server so we can safely read data with clickhouse-local.
# Why do we read data with clickhouse-local?
# Because it's the simplest way to read it when server has crashed.
sudo clickhouse stop ||:
for _ in $(seq 1 60); do if [[ $(wget --timeout=1 -q 'localhost:8123' -O-) == 'Ok.' ]]; then sleep 1 ; else break; fi ; done
grep -Fa "Fatal" /var/log/clickhouse-server/clickhouse-server.log ||:
pigz < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.gz &
# Compressed (FIXME: remove once only github actions will be left)
rm /var/log/clickhouse-server/clickhouse-server.log
mv /var/log/clickhouse-server/stderr.log /test_output/ ||:

View File

@ -79,8 +79,8 @@ In most cases, the read method is only responsible for reading the specified col
But there are notable exceptions:
- The AST query is passed to the `read` method, and the table engine can use it to derive index usage and to read fewer data from a table.
- Sometimes the table engine can process data itself to a specific stage. For example, `StorageDistributed` can send a query to remote servers, ask them to process data to a stage where data from different remote servers can be merged, and return that preprocessed data. The query interpreter then finishes processing the data.
- The AST query is passed to the `read` method, and the table engine can use it to derive index usage and to read fewer data from a table.
- Sometimes the table engine can process data itself to a specific stage. For example, `StorageDistributed` can send a query to remote servers, ask them to process data to a stage where data from different remote servers can be merged, and return that preprocessed data. The query interpreter then finishes processing the data.
The tables `read` method can return multiple `IBlockInputStream` objects to allow parallel data processing. These multiple block input streams can read from a table in parallel. Then you can wrap these streams with various transformations (such as expression evaluation or filtering) that can be calculated independently and create a `UnionBlockInputStream` on top of them, to read from multiple streams in parallel.
@ -132,9 +132,9 @@ Aggregation states can be serialized and deserialized to pass over the network d
The server implements several different interfaces:
- An HTTP interface for any foreign clients.
- A TCP interface for the native ClickHouse client and for cross-server communication during distributed query execution.
- An interface for transferring data for replication.
- An HTTP interface for any foreign clients.
- A TCP interface for the native ClickHouse client and for cross-server communication during distributed query execution.
- An interface for transferring data for replication.
Internally, it is just a primitive multithread server without coroutines or fibers. Since the server is not designed to process a high rate of simple queries but to process a relatively low rate of complex queries, each of them can process a vast amount of data for analytics.

View File

@ -9,9 +9,9 @@ description: How to build ClickHouse on Linux
Supported platforms:
- x86_64
- AArch64
- Power9 (experimental)
- x86_64
- AArch64
- Power9 (experimental)
## Normal Build for Development on Ubuntu
@ -82,13 +82,13 @@ This will create the `programs/clickhouse` executable, which can be used with `c
The build requires the following components:
- Git (is used only to checkout the sources, its not needed for the build)
- CMake 3.15 or newer
- Ninja
- C++ compiler: clang-15 or newer
- Linker: lld
- Yasm
- Gawk
- Git (is used only to checkout the sources, its not needed for the build)
- CMake 3.15 or newer
- Ninja
- C++ compiler: clang-15 or newer
- Linker: lld
- Yasm
- Gawk
If all the components are installed, you may build in the same way as the steps above.

View File

@ -370,8 +370,8 @@ If the file contains a single function, name the file the same way as the functi
**11.** If the name contains an abbreviation, then:
- For variable names, the abbreviation should use lowercase letters `mysql_connection` (not `mySQL_connection`).
- For names of classes and functions, keep the uppercase letters in the abbreviation`MySQLConnection` (not `MySqlConnection`).
- For variable names, the abbreviation should use lowercase letters `mysql_connection` (not `mySQL_connection`).
- For names of classes and functions, keep the uppercase letters in the abbreviation`MySQLConnection` (not `MySqlConnection`).
**12.** Constructor arguments that are used just to initialize the class members should be named the same way as the class members, but with an underscore at the end.
@ -434,9 +434,9 @@ In application code, memory must be freed by the object that owns it.
Examples:
- The easiest way is to place an object on the stack, or make it a member of another class.
- For a large number of small objects, use containers.
- For automatic deallocation of a small number of objects that reside in the heap, use `shared_ptr/unique_ptr`.
- The easiest way is to place an object on the stack, or make it a member of another class.
- For a large number of small objects, use containers.
- For automatic deallocation of a small number of objects that reside in the heap, use `shared_ptr/unique_ptr`.
**2.** Resource management.
@ -504,10 +504,10 @@ This is not recommended, but it is allowed.
Use the following options:
- Create a function (`done()` or `finalize()`) that will do all the work in advance that might lead to an exception. If that function was called, there should be no exceptions in the destructor later.
- Tasks that are too complex (such as sending messages over the network) can be put in separate method that the class user will have to call before destruction.
- If there is an exception in the destructor, its better to log it than to hide it (if the logger is available).
- In simple applications, it is acceptable to rely on `std::terminate` (for cases of `noexcept` by default in C++11) to handle exceptions.
- Create a function (`done()` or `finalize()`) that will do all the work in advance that might lead to an exception. If that function was called, there should be no exceptions in the destructor later.
- Tasks that are too complex (such as sending messages over the network) can be put in separate method that the class user will have to call before destruction.
- If there is an exception in the destructor, its better to log it than to hide it (if the logger is available).
- In simple applications, it is acceptable to rely on `std::terminate` (for cases of `noexcept` by default in C++11) to handle exceptions.
**6.** Anonymous code blocks.
@ -529,11 +529,11 @@ ready_any.set();
In offline data processing programs:
- Try to get the best possible performance on a single CPU core. You can then parallelize your code if necessary.
- Try to get the best possible performance on a single CPU core. You can then parallelize your code if necessary.
In server applications:
- Use the thread pool to process requests. At this point, we havent had any tasks that required userspace context switching.
- Use the thread pool to process requests. At this point, we havent had any tasks that required userspace context switching.
Fork is not used for parallelization.

View File

@ -194,11 +194,11 @@ If the system clickhouse-server is already running and you do not want to stop i
Build tests allow to check that build is not broken on various alternative configurations and on some foreign systems. These tests are automated as well.
Examples:
- cross-compile for Darwin x86_64 (macOS)
- cross-compile for FreeBSD x86_64
- cross-compile for Linux AArch64
- build on Ubuntu with libraries from system packages (discouraged)
- build with shared linking of libraries (discouraged)
- cross-compile for Darwin x86_64 (macOS)
- cross-compile for FreeBSD x86_64
- cross-compile for Linux AArch64
- build on Ubuntu with libraries from system packages (discouraged)
- build with shared linking of libraries (discouraged)
For example, build with system packages is bad practice, because we cannot guarantee what exact version of packages a system will have. But this is really needed by Debian maintainers. For this reason we at least have to support this variant of build. Another example: shared linking is a common source of trouble, but it is needed for some enthusiasts.

View File

@ -59,4 +59,4 @@ For [ReplicatedMergeTree](../table-engines/mergetree-family/replication.md#table
## See Also
- [system.databases](../../operations/system-tables/databases.md) system table
- [system.databases](../../operations/system-tables/databases.md) system table

View File

@ -11,18 +11,18 @@ Database engines allow you to work with tables. By default, ClickHouse uses the
Here is a complete list of available database engines. Follow the links for more details:
- [Atomic](../../engines/database-engines/atomic.md)
- [Atomic](../../engines/database-engines/atomic.md)
- [MySQL](../../engines/database-engines/mysql.md)
- [MySQL](../../engines/database-engines/mysql.md)
- [MaterializedMySQL](../../engines/database-engines/materialized-mysql.md)
- [MaterializedMySQL](../../engines/database-engines/materialized-mysql.md)
- [Lazy](../../engines/database-engines/lazy.md)
- [Lazy](../../engines/database-engines/lazy.md)
- [PostgreSQL](../../engines/database-engines/postgresql.md)
- [PostgreSQL](../../engines/database-engines/postgresql.md)
- [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md)
- [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md)
- [Replicated](../../engines/database-engines/replicated.md)
- [Replicated](../../engines/database-engines/replicated.md)
- [SQLite](../../engines/database-engines/sqlite.md)
- [SQLite](../../engines/database-engines/sqlite.md)

View File

@ -22,10 +22,10 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo
**Engine Parameters**
- `host:port` — MySQL server endpoint.
- `database` — MySQL database name.
- `user` — MySQL user.
- `password` — User password.
- `host:port` — MySQL server endpoint.
- `database` — MySQL database name.
- `user` — MySQL user.
- `password` — User password.
## Engine Settings

View File

@ -26,10 +26,10 @@ ENGINE = MaterializedPostgreSQL('host:port', 'database', 'user', 'password') [SE
**Engine Parameters**
- `host:port` — PostgreSQL server endpoint.
- `database` — PostgreSQL database name.
- `user` — PostgreSQL user.
- `password` — User password.
- `host:port` — PostgreSQL server endpoint.
- `database` — PostgreSQL database name.
- `user` — PostgreSQL user.
- `password` — User password.
## Example of Use {#example-of-use}
@ -120,9 +120,9 @@ Warning: for this case dots in table name are not allowed.
2. Each replicated table must have one of the following [replica identity](https://www.postgresql.org/docs/10/sql-altertable.html#SQL-CREATETABLE-REPLICA-IDENTITY):
- primary key (by default)
- primary key (by default)
- index
- index
``` bash
postgres# CREATE TABLE postgres_table (a Integer NOT NULL, b Integer, c Integer NOT NULL, d Integer, e Integer NOT NULL);
@ -171,7 +171,7 @@ Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.htm
Possible values:
- Positive integer.
- Positive integer.
Default value: `65536`.

View File

@ -12,9 +12,9 @@ The `MySQL` database engine translate queries to the MySQL server so you can per
You cannot perform the following queries:
- `RENAME`
- `CREATE TABLE`
- `ALTER`
- `RENAME`
- `CREATE TABLE`
- `ALTER`
## Creating a Database {#creating-a-database}
@ -25,10 +25,10 @@ ENGINE = MySQL('host:port', ['database' | database], 'user', 'password')
**Engine Parameters**
- `host:port` — MySQL server address.
- `database` — Remote database name.
- `user` — MySQL user.
- `password` — User password.
- `host:port` — MySQL server address.
- `database` — Remote database name.
- `user` — MySQL user.
- `password` — User password.
## Data Types Support {#data_types-support}

View File

@ -21,12 +21,12 @@ ENGINE = PostgreSQL('host:port', 'database', 'user', 'password'[, `schema`, `use
**Engine Parameters**
- `host:port` — PostgreSQL server address.
- `database` — Remote database name.
- `user` — PostgreSQL user.
- `password` — User password.
- `schema` — PostgreSQL schema.
- `use_table_cache` — Defines if the database table structure is cached or not. Optional. Default value: `0`.
- `host:port` — PostgreSQL server address.
- `database` — Remote database name.
- `user` — PostgreSQL user.
- `password` — User password.
- `schema` — PostgreSQL schema.
- `use_table_cache` — Defines if the database table structure is cached or not. Optional. Default value: `0`.
## Data Types Support {#data_types-support}

View File

@ -17,9 +17,9 @@ CREATE DATABASE testdb ENGINE = Replicated('zoo_path', 'shard_name', 'replica_na
**Engine Parameters**
- `zoo_path` — ZooKeeper path. The same ZooKeeper path corresponds to the same database.
- `shard_name` — Shard name. Database replicas are grouped into shards by `shard_name`.
- `replica_name` — Replica name. Replica names must be different for all replicas of the same shard.
- `zoo_path` — ZooKeeper path. The same ZooKeeper path corresponds to the same database.
- `shard_name` — Shard name. Database replicas are grouped into shards by `shard_name`.
- `replica_name` — Replica name. Replica names must be different for all replicas of the same shard.
For [ReplicatedMergeTree](../table-engines/mergetree-family/replication.md#table_engines-replication) tables if no arguments provided, then default arguments are used: `/clickhouse/tables/{uuid}/{shard}` and `{replica}`. These can be changed in the server settings [default_replica_path](../../operations/server-configuration-parameters/settings.md#default_replica_path) and [default_replica_name](../../operations/server-configuration-parameters/settings.md#default_replica_name). Macro `{uuid}` is unfolded to table's uuid, `{shard}` and `{replica}` are unfolded to values from server config, not from database engine arguments. But in the future, it will be possible to use `shard_name` and `replica_name` of Replicated database.

View File

@ -17,7 +17,7 @@ Allows to connect to [SQLite](https://www.sqlite.org/index.html) database and pe
**Engine Parameters**
- `db_path` — Path to a file with SQLite database.
- `db_path` — Path to a file with SQLite database.
## Data Types Support {#data_types-support}

View File

@ -9,12 +9,12 @@ toc_title: Introduction
The table engine (type of table) determines:
- How and where data is stored, where to write it to, and where to read it from.
- Which queries are supported, and how.
- Concurrent data access.
- Use of indexes, if present.
- Whether multithread request execution is possible.
- Data replication parameters.
- How and where data is stored, where to write it to, and where to read it from.
- Which queries are supported, and how.
- Concurrent data access.
- Use of indexes, if present.
- Whether multithread request execution is possible.
- Data replication parameters.
## Engine Families {#engine-families}
@ -24,13 +24,13 @@ The most universal and functional table engines for high-load tasks. The propert
Engines in the family:
- [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#mergetree)
- [ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md#replacingmergetree)
- [SummingMergeTree](../../engines/table-engines/mergetree-family/summingmergetree.md#summingmergetree)
- [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md#aggregatingmergetree)
- [CollapsingMergeTree](../../engines/table-engines/mergetree-family/collapsingmergetree.md#table_engine-collapsingmergetree)
- [VersionedCollapsingMergeTree](../../engines/table-engines/mergetree-family/versionedcollapsingmergetree.md#versionedcollapsingmergetree)
- [GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md#graphitemergetree)
- [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#mergetree)
- [ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md#replacingmergetree)
- [SummingMergeTree](../../engines/table-engines/mergetree-family/summingmergetree.md#summingmergetree)
- [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md#aggregatingmergetree)
- [CollapsingMergeTree](../../engines/table-engines/mergetree-family/collapsingmergetree.md#table_engine-collapsingmergetree)
- [VersionedCollapsingMergeTree](../../engines/table-engines/mergetree-family/versionedcollapsingmergetree.md#versionedcollapsingmergetree)
- [GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md#graphitemergetree)
### Log {#log}
@ -38,9 +38,9 @@ Lightweight [engines](../../engines/table-engines/log-family/index.md) with mini
Engines in the family:
- [TinyLog](../../engines/table-engines/log-family/tinylog.md#tinylog)
- [StripeLog](../../engines/table-engines/log-family/stripelog.md#stripelog)
- [Log](../../engines/table-engines/log-family/log.md#log)
- [TinyLog](../../engines/table-engines/log-family/tinylog.md#tinylog)
- [StripeLog](../../engines/table-engines/log-family/stripelog.md#stripelog)
- [Log](../../engines/table-engines/log-family/log.md#log)
### Integration Engines {#integration-engines}
@ -49,34 +49,34 @@ Engines for communicating with other data storage and processing systems.
Engines in the family:
- [ODBC](../../engines/table-engines/integrations/odbc.md)
- [JDBC](../../engines/table-engines/integrations/jdbc.md)
- [MySQL](../../engines/table-engines/integrations/mysql.md)
- [MongoDB](../../engines/table-engines/integrations/mongodb.md)
- [HDFS](../../engines/table-engines/integrations/hdfs.md)
- [S3](../../engines/table-engines/integrations/s3.md)
- [Kafka](../../engines/table-engines/integrations/kafka.md)
- [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md)
- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md)
- [PostgreSQL](../../engines/table-engines/integrations/postgresql.md)
- [ODBC](../../engines/table-engines/integrations/odbc.md)
- [JDBC](../../engines/table-engines/integrations/jdbc.md)
- [MySQL](../../engines/table-engines/integrations/mysql.md)
- [MongoDB](../../engines/table-engines/integrations/mongodb.md)
- [HDFS](../../engines/table-engines/integrations/hdfs.md)
- [S3](../../engines/table-engines/integrations/s3.md)
- [Kafka](../../engines/table-engines/integrations/kafka.md)
- [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md)
- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md)
- [PostgreSQL](../../engines/table-engines/integrations/postgresql.md)
### Special Engines {#special-engines}
Engines in the family:
- [Distributed](../../engines/table-engines/special/distributed.md#distributed)
- [MaterializedView](../../engines/table-engines/special/materializedview.md#materializedview)
- [Dictionary](../../engines/table-engines/special/dictionary.md#dictionary)
- [Merge](../../engines/table-engines/special/merge.md#merge)
- [File](../../engines/table-engines/special/file.md#file)
- [Null](../../engines/table-engines/special/null.md#null)
- [Set](../../engines/table-engines/special/set.md#set)
- [Join](../../engines/table-engines/special/join.md#join)
- [URL](../../engines/table-engines/special/url.md#table_engines-url)
- [View](../../engines/table-engines/special/view.md#table_engines-view)
- [Memory](../../engines/table-engines/special/memory.md#memory)
- [Buffer](../../engines/table-engines/special/buffer.md#buffer)
- [KeeperMap](../../engines/table-engines/special/keepermap.md)
- [Distributed](../../engines/table-engines/special/distributed.md#distributed)
- [MaterializedView](../../engines/table-engines/special/materializedview.md#materializedview)
- [Dictionary](../../engines/table-engines/special/dictionary.md#dictionary)
- [Merge](../../engines/table-engines/special/merge.md#merge)
- [File](../../engines/table-engines/special/file.md#file)
- [Null](../../engines/table-engines/special/null.md#null)
- [Set](../../engines/table-engines/special/set.md#set)
- [Join](../../engines/table-engines/special/join.md#join)
- [URL](../../engines/table-engines/special/url.md#table_engines-url)
- [View](../../engines/table-engines/special/view.md#table_engines-view)
- [Memory](../../engines/table-engines/special/memory.md#memory)
- [Buffer](../../engines/table-engines/special/buffer.md#buffer)
- [KeeperMap](../../engines/table-engines/special/keepermap.md)
## Virtual Columns {#table_engines-virtual_columns}

View File

@ -22,17 +22,17 @@ See a detailed description of the [CREATE TABLE](../../../sql-reference/statemen
The table structure can differ from the original table structure:
- Column names should be the same as in the original table, but you can use just some of these columns and in any order.
- Column types may differ from those in the original table. ClickHouse tries to [cast](../../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) values to the ClickHouse data types.
- Column names should be the same as in the original table, but you can use just some of these columns and in any order.
- Column types may differ from those in the original table. ClickHouse tries to [cast](../../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) values to the ClickHouse data types.
**Engine Parameters**
- `engine` — The table engine `MySQL` or `PostgreSQL`.
- `host:port` — MySQL or PostgreSQL server address.
- `database` — Remote database name.
- `table` — Remote table name.
- `user` — User name.
- `password` — User password.
- `engine` — The table engine `MySQL` or `PostgreSQL`.
- `host:port` — MySQL or PostgreSQL server address.
- `database` — Remote database name.
- `table` — Remote table name.
- `user` — User name.
- `password` — User password.
## Implementation Details {#implementation-details}
@ -48,6 +48,6 @@ You can specify any number of shards and any number of replicas for each shard.
**See Also**
- [MySQL table engine](../../../engines/table-engines/integrations/mysql.md)
- [PostgreSQL table engine](../../../engines/table-engines/integrations/postgresql.md)
- [Distributed table engine](../../../engines/table-engines/special/distributed.md)
- [MySQL table engine](../../../engines/table-engines/integrations/mysql.md)
- [PostgreSQL table engine](../../../engines/table-engines/integrations/postgresql.md)
- [Distributed table engine](../../../engines/table-engines/special/distributed.md)

View File

@ -18,8 +18,8 @@ CREATE TABLE deltalake
**Engine parameters**
- `url` — Bucket url with path to the existing Delta Lake table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
- `url` — Bucket url with path to the existing Delta Lake table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md)
@ -49,4 +49,4 @@ CREATE TABLE deltalake ENGINE=DeltaLake(deltalake_conf, filename = 'test_table')
## See also
- [deltaLake table function](../../../sql-reference/table-functions/deltalake.md)
- [deltaLake table function](../../../sql-reference/table-functions/deltalake.md)

View File

@ -17,7 +17,7 @@ ENGINE = HDFS(URI, format)
**Engine Parameters**
- `URI` - whole file URI in HDFS. The path part of `URI` may contain globs. In this case the table would be readonly.
- `format` - specifies one of the available file formats. To perform
- `format` - specifies one of the available file formats. To perform
`SELECT` queries, the format must be supported for input, and to perform
`INSERT` queries for output. The available formats are listed in the
[Formats](../../../interfaces/formats.md#formats) section.
@ -58,11 +58,11 @@ SELECT * FROM hdfs_engine_table LIMIT 2
## Implementation Details {#implementation-details}
- Reads and writes can be parallel.
- Not supported:
- `ALTER` and `SELECT...SAMPLE` operations.
- Indexes.
- [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is possible, but not recommended.
- Reads and writes can be parallel.
- Not supported:
- `ALTER` and `SELECT...SAMPLE` operations.
- Indexes.
- [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is possible, but not recommended.
:::note Zero-copy replication is not ready for production
Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use.
@ -72,10 +72,10 @@ SELECT * FROM hdfs_engine_table LIMIT 2
Multiple path components can have globs. For being processed file should exists and matches to the whole path pattern. Listing of files determines during `SELECT` (not at `CREATE` moment).
- `*` — Substitutes any number of any characters except `/` including empty string.
- `?` — Substitutes any single character.
- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`.
- `{N..M}` — Substitutes any number in range from N to M including both borders.
- `*` — Substitutes any number of any characters except `/` including empty string.
- `?` — Substitutes any single character.
- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`.
- `{N..M}` — Substitutes any number in range from N to M including both borders.
Constructions with `{}` are similar to the [remote](../../../sql-reference/table-functions/remote.md) table function.
@ -83,12 +83,12 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table
1. Suppose we have several files in TSV format with the following URIs on HDFS:
- 'hdfs://hdfs1:9000/some_dir/some_file_1'
- 'hdfs://hdfs1:9000/some_dir/some_file_2'
- 'hdfs://hdfs1:9000/some_dir/some_file_3'
- 'hdfs://hdfs1:9000/another_dir/some_file_1'
- 'hdfs://hdfs1:9000/another_dir/some_file_2'
- 'hdfs://hdfs1:9000/another_dir/some_file_3'
- 'hdfs://hdfs1:9000/some_dir/some_file_1'
- 'hdfs://hdfs1:9000/some_dir/some_file_2'
- 'hdfs://hdfs1:9000/some_dir/some_file_3'
- 'hdfs://hdfs1:9000/another_dir/some_file_1'
- 'hdfs://hdfs1:9000/another_dir/some_file_2'
- 'hdfs://hdfs1:9000/another_dir/some_file_3'
1. There are several ways to make a table consisting of all six files:
@ -145,7 +145,7 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us
| **parameter** | **default value** |
| - | - |
| - | - |
| rpc\_client\_connect\_tcpnodelay | true |
| dfs\_client\_read\_shortcircuit | true |
| output\_replace-datanode-on-failure | true |
@ -195,7 +195,7 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us
#### ClickHouse extras {#clickhouse-extras}
| **parameter** | **default value** |
| - | - |
| - | - |
|hadoop\_kerberos\_keytab | "" |
|hadoop\_kerberos\_principal | "" |
|libhdfs3\_conf | "" |
@ -230,9 +230,9 @@ libhdfs3 support HDFS namenode HA.
## Virtual Columns {#virtual-columns}
- `_path` — Path to the file.
- `_file` — Name of the file.
- `_path` — Path to the file.
- `_file` — Name of the file.
**See Also**
- [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns)
- [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns)

View File

@ -28,17 +28,17 @@ PARTITION BY expr
See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query.
The table structure can differ from the original Hive table structure:
- Column names should be the same as in the original Hive table, but you can use just some of these columns and in any order, also you can use some alias columns calculated from other columns.
- Column types should be the same from those in the original Hive table.
- Partition by expression should be consistent with the original Hive table, and columns in partition by expression should be in the table structure.
- Column names should be the same as in the original Hive table, but you can use just some of these columns and in any order, also you can use some alias columns calculated from other columns.
- Column types should be the same from those in the original Hive table.
- Partition by expression should be consistent with the original Hive table, and columns in partition by expression should be in the table structure.
**Engine Parameters**
- `thrift://host:port` — Hive Metastore address
- `thrift://host:port` — Hive Metastore address
- `database` — Remote database name.
- `database` — Remote database name.
- `table` — Remote table name.
- `table` — Remote table name.
## Usage Example {#usage-example}

View File

@ -18,8 +18,8 @@ CREATE TABLE hudi_table
**Engine parameters**
- `url` — Bucket url with the path to an existing Hudi table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
- `url` — Bucket url with the path to an existing Hudi table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md)
@ -49,4 +49,4 @@ CREATE TABLE hudi_table ENGINE=Hudi(hudi_conf, filename = 'test_table')
## See also
- [hudi table function](/docs/en/sql-reference/table-functions/hudi.md)
- [hudi table function](/docs/en/sql-reference/table-functions/hudi.md)

View File

@ -18,8 +18,8 @@ CREATE TABLE iceberg_table
**Engine parameters**
- `url` — url with the path to an existing Iceberg table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
- `url` — url with the path to an existing Iceberg table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md)
@ -49,4 +49,4 @@ CREATE TABLE iceberg_table ENGINE=Iceberg(iceberg_conf, filename = 'test_table')
## See also
- [iceberg table function](/docs/en/sql-reference/table-functions/iceberg.md)
- [iceberg table function](/docs/en/sql-reference/table-functions/iceberg.md)

View File

@ -10,20 +10,20 @@ ClickHouse provides various means for integrating with external systems, includi
List of supported integrations:
- [ODBC](../../../engines/table-engines/integrations/odbc.md)
- [JDBC](../../../engines/table-engines/integrations/jdbc.md)
- [MySQL](../../../engines/table-engines/integrations/mysql.md)
- [MongoDB](../../../engines/table-engines/integrations/mongodb.md)
- [HDFS](../../../engines/table-engines/integrations/hdfs.md)
- [S3](../../../engines/table-engines/integrations/s3.md)
- [Kafka](../../../engines/table-engines/integrations/kafka.md)
- [EmbeddedRocksDB](../../../engines/table-engines/integrations/embedded-rocksdb.md)
- [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md)
- [PostgreSQL](../../../engines/table-engines/integrations/postgresql.md)
- [SQLite](../../../engines/table-engines/integrations/sqlite.md)
- [Hive](../../../engines/table-engines/integrations/hive.md)
- [ExternalDistributed](../../../engines/table-engines/integrations/ExternalDistributed.md)
- [MaterializedPostgreSQL](../../../engines/table-engines/integrations/materialized-postgresql.md)
- [NATS](../../../engines/table-engines/integrations/nats.md)
- [DeltaLake](../../../engines/table-engines/integrations/deltalake.md)
- [Hudi](../../../engines/table-engines/integrations/hudi.md)
- [ODBC](../../../engines/table-engines/integrations/odbc.md)
- [JDBC](../../../engines/table-engines/integrations/jdbc.md)
- [MySQL](../../../engines/table-engines/integrations/mysql.md)
- [MongoDB](../../../engines/table-engines/integrations/mongodb.md)
- [HDFS](../../../engines/table-engines/integrations/hdfs.md)
- [S3](../../../engines/table-engines/integrations/s3.md)
- [Kafka](../../../engines/table-engines/integrations/kafka.md)
- [EmbeddedRocksDB](../../../engines/table-engines/integrations/embedded-rocksdb.md)
- [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md)
- [PostgreSQL](../../../engines/table-engines/integrations/postgresql.md)
- [SQLite](../../../engines/table-engines/integrations/sqlite.md)
- [Hive](../../../engines/table-engines/integrations/hive.md)
- [ExternalDistributed](../../../engines/table-engines/integrations/ExternalDistributed.md)
- [MaterializedPostgreSQL](../../../engines/table-engines/integrations/materialized-postgresql.md)
- [NATS](../../../engines/table-engines/integrations/nats.md)
- [DeltaLake](../../../engines/table-engines/integrations/deltalake.md)
- [Hudi](../../../engines/table-engines/integrations/hudi.md)

View File

@ -25,14 +25,14 @@ ENGINE = JDBC(datasource_uri, external_database, external_table)
**Engine Parameters**
- `datasource_uri` — URI or name of an external DBMS.
- `datasource_uri` — URI or name of an external DBMS.
URI Format: `jdbc:<driver_name>://<host_name>:<port>/?user=<username>&password=<password>`.
Example for MySQL: `jdbc:mysql://localhost:3306/?user=root&password=root`.
- `external_database` — Database in an external DBMS.
- `external_database` — Database in an external DBMS.
- `external_table` — Name of the table in `external_database` or a select query like `select * from table1 where column1=1`.
- `external_table` — Name of the table in `external_database` or a select query like `select * from table1 where column1=1`.
## Usage Example {#usage-example}
@ -91,4 +91,4 @@ FROM system.numbers
## See Also {#see-also}
- [JDBC table function](../../../sql-reference/table-functions/jdbc.md).
- [JDBC table function](../../../sql-reference/table-functions/jdbc.md).

View File

@ -10,9 +10,9 @@ This engine works with [Apache Kafka](http://kafka.apache.org/).
Kafka lets you:
- Publish or subscribe to data flows.
- Organize fault-tolerant storage.
- Process streams as they become available.
- Publish or subscribe to data flows.
- Organize fault-tolerant storage.
- Process streams as they become available.
## Creating a Table {#table_engine-kafka-creating-a-table}
@ -46,27 +46,27 @@ SETTINGS
Required parameters:
- `kafka_broker_list` — A comma-separated list of brokers (for example, `localhost:9092`).
- `kafka_topic_list` — A list of Kafka topics.
- `kafka_group_name` — A group of Kafka consumers. Reading margins are tracked for each group separately. If you do not want messages to be duplicated in the cluster, use the same group name everywhere.
- `kafka_format` — Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section.
- `kafka_broker_list` — A comma-separated list of brokers (for example, `localhost:9092`).
- `kafka_topic_list` — A list of Kafka topics.
- `kafka_group_name` — A group of Kafka consumers. Reading margins are tracked for each group separately. If you do not want messages to be duplicated in the cluster, use the same group name everywhere.
- `kafka_format` — Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section.
Optional parameters:
- `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, [Capn 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. 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`.
- `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, [Capn 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. 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:
@ -239,14 +239,14 @@ Example:
## Virtual Columns {#virtual-columns}
- `_topic` — Kafka topic.
- `_key` — Key of the message.
- `_offset` — Offset of the message.
- `_timestamp` — Timestamp of the message.
- `_timestamp_ms` — Timestamp in milliseconds of the message.
- `_partition` — Partition of Kafka topic.
- `_headers.name` — Array of message's headers keys.
- `_headers.value` — Array of message's headers values.
- `_topic` — Kafka topic.
- `_key` — Key of the message.
- `_offset` — Offset of the message.
- `_timestamp` — Timestamp of the message.
- `_timestamp_ms` — Timestamp in milliseconds of the message.
- `_partition` — Partition of Kafka topic.
- `_headers.name` — Array of message's headers keys.
- `_headers.value` — Array of message's headers values.
## Data formats support {#data-formats-support}
@ -258,5 +258,5 @@ The number of rows in one Kafka message depends on whether the format is row-bas
**See Also**
- [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns)
- [background_message_broker_schedule_pool_size](../../../operations/server-configuration-parameters/settings.md#background_message_broker_schedule_pool_size)
- [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns)
- [background_message_broker_schedule_pool_size](../../../operations/server-configuration-parameters/settings.md#background_message_broker_schedule_pool_size)

View File

@ -19,11 +19,11 @@ PRIMARY KEY key;
**Engine Parameters**
- `host:port` — PostgreSQL server address.
- `database` — Remote database name.
- `table` — Remote table name.
- `user` — PostgreSQL user.
- `password` — User password.
- `host:port` — PostgreSQL server address.
- `database` — Remote database name.
- `table` — Remote table name.
- `user` — PostgreSQL user.
- `password` — User password.
## Requirements {#requirements}
@ -33,11 +33,13 @@ PRIMARY KEY key;
3. Only database [Atomic](https://en.wikipedia.org/wiki/Atomicity_(database_systems)) is allowed.
4. The `MaterializedPostgreSQL` table engine only works for PostgreSQL versions >= 11 as the implementation requires the [pg_replication_slot_advance](https://pgpedia.info/p/pg_replication_slot_advance.html) PostgreSQL function.
## Virtual columns {#virtual-columns}
- `_version` — Transaction counter. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
- `_version` — Transaction counter. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
- `_sign` — Deletion mark. Type: [Int8](../../../sql-reference/data-types/int-uint.md). Possible values:
- `_sign` — Deletion mark. Type: [Int8](../../../sql-reference/data-types/int-uint.md). Possible values:
- `1` — Row is not deleted,
- `-1` — Row is deleted.

View File

@ -21,17 +21,17 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name
**Engine Parameters**
- `host:port` — MongoDB server address.
- `host:port` — MongoDB server address.
- `database` — Remote database name.
- `database` — Remote database name.
- `collection` — Remote collection name.
- `collection` — Remote collection name.
- `user` — MongoDB user.
- `user` — MongoDB user.
- `password` — User password.
- `password` — User password.
- `options` — MongoDB connection string options (optional parameter).
- `options` — MongoDB connection string options (optional parameter).
## Usage Example {#usage-example}

View File

@ -31,25 +31,25 @@ See a detailed description of the [CREATE TABLE](../../../sql-reference/statemen
The table structure can differ from the original MySQL table structure:
- Column names should be the same as in the original MySQL table, but you can use just some of these columns and in any order.
- Column types may differ from those in the original MySQL table. ClickHouse tries to [cast](../../../engines/database-engines/mysql.md#data_types-support) values to the ClickHouse data types.
- The [external_table_functions_use_nulls](../../../operations/settings/settings.md#external-table-functions-use-nulls) setting defines how to handle Nullable columns. Default value: 1. If 0, the table function does not make Nullable columns and inserts default values instead of nulls. This is also applicable for NULL values inside arrays.
- Column names should be the same as in the original MySQL table, but you can use just some of these columns and in any order.
- Column types may differ from those in the original MySQL table. ClickHouse tries to [cast](../../../engines/database-engines/mysql.md#data_types-support) values to the ClickHouse data types.
- The [external_table_functions_use_nulls](../../../operations/settings/settings.md#external-table-functions-use-nulls) setting defines how to handle Nullable columns. Default value: 1. If 0, the table function does not make Nullable columns and inserts default values instead of nulls. This is also applicable for NULL values inside arrays.
**Engine Parameters**
- `host:port` — MySQL server address.
- `host:port` — MySQL server address.
- `database` — Remote database name.
- `database` — Remote database name.
- `table` — Remote table name.
- `table` — Remote table name.
- `user` — MySQL user.
- `user` — MySQL user.
- `password` — User password.
- `password` — User password.
- `replace_query` — Flag that converts `INSERT INTO` queries to `REPLACE INTO`. If `replace_query=1`, the query is substituted.
- `replace_query` — Flag that converts `INSERT INTO` queries to `REPLACE INTO`. If `replace_query=1`, the query is substituted.
- `on_duplicate_clause` — The `ON DUPLICATE KEY on_duplicate_clause` expression that is added to the `INSERT` query.
- `on_duplicate_clause` — The `ON DUPLICATE KEY on_duplicate_clause` expression that is added to the `INSERT` query.
Example: `INSERT INTO t (c1,c2) VALUES ('a', 2) ON DUPLICATE KEY UPDATE c2 = c2 + 1`, where `on_duplicate_clause` is `UPDATE c2 = c2 + 1`. See the [MySQL documentation](https://dev.mysql.com/doc/refman/8.0/en/insert-on-duplicate.html) to find which `on_duplicate_clause` you can use with the `ON DUPLICATE KEY` clause.
@ -121,8 +121,8 @@ Allows to automatically close the connection after query execution, i.e. disable
Possible values:
- 1 — Auto-close connection is allowed, so the connection reuse is disabled
- 0 — Auto-close connection is not allowed, so the connection reuse is enabled
- 1 — Auto-close connection is allowed, so the connection reuse is disabled
- 0 — Auto-close connection is not allowed, so the connection reuse is enabled
Default value: `1`.
@ -132,8 +132,8 @@ Sets the number of retries for pool with failover.
Possible values:
- Positive integer.
- 0 — There are no retries for pool with failover.
- Positive integer.
- 0 — There are no retries for pool with failover.
Default value: `3`.
@ -143,7 +143,7 @@ Size of connection pool (if all connections are in use, the query will wait unti
Possible values:
- Positive integer.
- Positive integer.
Default value: `16`.
@ -153,7 +153,7 @@ Timeout (in seconds) for waiting for free connection (in case of there is alread
Possible values:
- Positive integer.
- Positive integer.
Default value: `5`.
@ -163,7 +163,7 @@ Connect timeout (in seconds).
Possible values:
- Positive integer.
- Positive integer.
Default value: `10`.
@ -173,11 +173,11 @@ Read/write timeout (in seconds).
Possible values:
- Positive integer.
- Positive integer.
Default value: `300`.
## See Also {#see-also}
- [The mysql table function](../../../sql-reference/table-functions/mysql.md)
- [Using MySQL as a dictionary source](../../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-mysql)
- [The mysql table function](../../../sql-reference/table-functions/mysql.md)
- [Using MySQL as a dictionary source](../../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-mysql)

View File

@ -45,9 +45,9 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
Required parameters:
- `nats_url` host:port (for example, `localhost:5672`)..
- `nats_subjects` List of subject for NATS table to subscribe/publsh to. Supports wildcard subjects like `foo.*.bar` or `baz.>`
- `nats_format` Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section.
- `nats_url` host:port (for example, `localhost:5672`)..
- `nats_subjects` List of subject for NATS table to subscribe/publsh to. Supports wildcard subjects like `foo.*.bar` or `baz.>`
- `nats_format` Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section.
Optional parameters:

View File

@ -28,15 +28,15 @@ See a detailed description of the [CREATE TABLE](../../../sql-reference/statemen
The table structure can differ from the source table structure:
- Column names should be the same as in the source table, but you can use just some of these columns and in any order.
- Column types may differ from those in the source table. ClickHouse tries to [cast](../../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) values to the ClickHouse data types.
- The [external_table_functions_use_nulls](../../../operations/settings/settings.md#external-table-functions-use-nulls) setting defines how to handle Nullable columns. Default value: 1. If 0, the table function does not make Nullable columns and inserts default values instead of nulls. This is also applicable for NULL values inside arrays.
- Column names should be the same as in the source table, but you can use just some of these columns and in any order.
- Column types may differ from those in the source table. ClickHouse tries to [cast](../../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) values to the ClickHouse data types.
- The [external_table_functions_use_nulls](../../../operations/settings/settings.md#external-table-functions-use-nulls) setting defines how to handle Nullable columns. Default value: 1. If 0, the table function does not make Nullable columns and inserts default values instead of nulls. This is also applicable for NULL values inside arrays.
**Engine Parameters**
- `connection_settings` — Name of the section with connection settings in the `odbc.ini` file.
- `external_database` — Name of a database in an external DBMS.
- `external_table` — Name of a table in the `external_database`.
- `connection_settings` — Name of the section with connection settings in the `odbc.ini` file.
- `external_database` — Name of a database in an external DBMS.
- `external_table` — Name of a table in the `external_database`.
## Usage Example {#usage-example}
@ -126,5 +126,5 @@ SELECT * FROM odbc_t
## See Also {#see-also}
- [ODBC dictionaries](../../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-odbc)
- [ODBC table function](../../../sql-reference/table-functions/odbc.md)
- [ODBC dictionaries](../../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-odbc)
- [ODBC table function](../../../sql-reference/table-functions/odbc.md)

View File

@ -23,19 +23,19 @@ See a detailed description of the [CREATE TABLE](../../../sql-reference/statemen
The table structure can differ from the original PostgreSQL table structure:
- Column names should be the same as in the original PostgreSQL table, but you can use just some of these columns and in any order.
- Column types may differ from those in the original PostgreSQL table. ClickHouse tries to [cast](../../../engines/database-engines/postgresql.md#data_types-support) values to the ClickHouse data types.
- The [external_table_functions_use_nulls](../../../operations/settings/settings.md#external-table-functions-use-nulls) setting defines how to handle Nullable columns. Default value: 1. If 0, the table function does not make Nullable columns and inserts default values instead of nulls. This is also applicable for NULL values inside arrays.
- Column names should be the same as in the original PostgreSQL table, but you can use just some of these columns and in any order.
- Column types may differ from those in the original PostgreSQL table. ClickHouse tries to [cast](../../../engines/database-engines/postgresql.md#data_types-support) values to the ClickHouse data types.
- The [external_table_functions_use_nulls](../../../operations/settings/settings.md#external-table-functions-use-nulls) setting defines how to handle Nullable columns. Default value: 1. If 0, the table function does not make Nullable columns and inserts default values instead of nulls. This is also applicable for NULL values inside arrays.
**Engine Parameters**
- `host:port` — PostgreSQL server address.
- `database` — Remote database name.
- `table` — Remote table name.
- `user` — PostgreSQL user.
- `password` — User password.
- `schema` — Non-default table schema. Optional.
- `on conflict ...` — example: `ON CONFLICT DO NOTHING`. Optional. Note: adding this option will make insertion less efficient.
- `host:port` — PostgreSQL server address.
- `database` — Remote database name.
- `table` — Remote table name.
- `user` — PostgreSQL user.
- `password` — User password.
- `schema` — Non-default table schema. Optional.
- `on conflict ...` — example: `ON CONFLICT DO NOTHING`. Optional. Note: adding this option will make insertion less efficient.
or via config (since version 21.11):
@ -111,7 +111,7 @@ In the example below replica `example01-1` has the highest priority:
## Usage Example {#usage-example}
Table in PostgreSQL:
### Table in PostgreSQL
``` text
postgres=# CREATE TABLE "public"."test" (
@ -134,7 +134,9 @@ postgresql> SELECT * FROM test;
(1 row)
```
Table in ClickHouse, retrieving data from the PostgreSQL table created above:
### Creating Table in ClickHouse, and connecting to PostgreSQL table created above
This example uses the [PostgreSQL table engine](/docs/en/engines/table-engines/integrations/postgresql.md) to connect the ClickHouse table to the PostgreSQL table:
``` sql
CREATE TABLE default.postgresql_table
@ -146,6 +148,35 @@ CREATE TABLE default.postgresql_table
ENGINE = PostgreSQL('localhost:5432', 'public', 'test', 'postges_user', 'postgres_password');
```
### Inserting initial data from PostgreSQL table into ClickHouse table, using a SELECT query
The [postgresql table function](/docs/en/sql-reference/table-functions/postgresql.md) copies the data from PostgreSQL to ClickHouse, which is often used for improving the query performance of the data by querying or performing analytics in ClickHouse rather than in PostgreSQL, or can also be used for migrating data from PostgreSQL to ClickHouse:
``` sql
INSERT INTO default.postgresql_table
SELECT * FROM postgresql('localhost:5432', 'public', 'test', 'postges_user', 'postgres_password');
```
### Inserting incremental data from PostgreSQL table into ClickHouse table
If then performing ongoing synchronization between the PostgreSQL table and ClickHouse table after the initial insert, you can use a WHERE clause in ClickHouse to insert only data added to PostgreSQL based on a timestamp or unique sequence ID.
This would require keeping track of the max ID or timestamp previously added, such as the following:
``` sql
SELECT max(`int_id`) AS maxIntID FROM default.postgresql_table;
```
Then inserting values from PostgreSQL table greater than the max
``` sql
INSERT INTO default.postgresql_table
SELECT * FROM postgresql('localhost:5432', 'public', 'test', 'postges_user', 'postgres_password');
WHERE int_id > maxIntID;
```
### Selecting data from the resulting ClickHouse table
``` sql
SELECT * FROM postgresql_table WHERE str IN ('test');
```
@ -156,7 +187,7 @@ SELECT * FROM postgresql_table WHERE str IN ('test');
└────────────────┴──────┴────────┘
```
Using Non-default Schema:
### Using Non-default Schema
```text
postgres=# CREATE SCHEMA "nice.schema";
@ -173,8 +204,8 @@ CREATE TABLE pg_table_schema_with_dots (a UInt32)
**See Also**
- [The `postgresql` table function](../../../sql-reference/table-functions/postgresql.md)
- [Using PostgreSQL as a dictionary source](../../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-postgresql)
- [The `postgresql` table function](../../../sql-reference/table-functions/postgresql.md)
- [Using PostgreSQL as a dictionary source](../../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-postgresql)
## Related content

View File

@ -10,8 +10,8 @@ This engine allows integrating ClickHouse with [RabbitMQ](https://www.rabbitmq.c
`RabbitMQ` lets you:
- Publish or subscribe to data flows.
- Process streams as they become available.
- Publish or subscribe to data flows.
- Process streams as they become available.
## Creating a Table {#table_engine-rabbitmq-creating-a-table}
@ -51,9 +51,9 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
Required parameters:
- `rabbitmq_host_port` host:port (for example, `localhost:5672`).
- `rabbitmq_exchange_name` RabbitMQ exchange name.
- `rabbitmq_format` Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section.
- `rabbitmq_host_port` host:port (for example, `localhost:5672`).
- `rabbitmq_exchange_name` RabbitMQ exchange name.
- `rabbitmq_format` Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section.
Optional parameters:
@ -138,24 +138,24 @@ There can be no more than one exchange per table. One exchange can be shared bet
Exchange type options:
- `direct` - Routing is based on the exact matching of keys. Example table key list: `key1,key2,key3,key4,key5`, message key can equal any of them.
- `fanout` - Routing to all tables (where exchange name is the same) regardless of the keys.
- `topic` - Routing is based on patterns with dot-separated keys. Examples: `*.logs`, `records.*.*.2020`, `*.2018,*.2019,*.2020`.
- `headers` - Routing is based on `key=value` matches with a setting `x-match=all` or `x-match=any`. Example table key list: `x-match=all,format=logs,type=report,year=2020`.
- `consistent_hash` - Data is evenly distributed between all bound tables (where the exchange name is the same). Note that this exchange type must be enabled with RabbitMQ plugin: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`.
- `direct` - Routing is based on the exact matching of keys. Example table key list: `key1,key2,key3,key4,key5`, message key can equal any of them.
- `fanout` - Routing to all tables (where exchange name is the same) regardless of the keys.
- `topic` - Routing is based on patterns with dot-separated keys. Examples: `*.logs`, `records.*.*.2020`, `*.2018,*.2019,*.2020`.
- `headers` - Routing is based on `key=value` matches with a setting `x-match=all` or `x-match=any`. Example table key list: `x-match=all,format=logs,type=report,year=2020`.
- `consistent_hash` - Data is evenly distributed between all bound tables (where the exchange name is the same). Note that this exchange type must be enabled with RabbitMQ plugin: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`.
Setting `rabbitmq_queue_base` may be used for the following cases:
- to let different tables share queues, so that multiple consumers could be registered for the same queues, which makes a better performance. If using `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings, the exact match of queues is achieved in case these parameters are the same.
- to be able to restore reading from certain durable queues when not all messages were successfully consumed. To resume consumption from one specific queue - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. By default, queue names will be unique to tables.
- to reuse queues as they are declared durable and not auto-deleted. (Can be deleted via any of RabbitMQ CLI tools.)
- to let different tables share queues, so that multiple consumers could be registered for the same queues, which makes a better performance. If using `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings, the exact match of queues is achieved in case these parameters are the same.
- to be able to restore reading from certain durable queues when not all messages were successfully consumed. To resume consumption from one specific queue - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. By default, queue names will be unique to tables.
- to reuse queues as they are declared durable and not auto-deleted. (Can be deleted via any of RabbitMQ CLI tools.)
To improve performance, received messages are grouped into blocks the size of [max_insert_block_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). If the block wasnt formed within [stream_flush_interval_ms](../../../operations/server-configuration-parameters/settings.md) milliseconds, the data will be flushed to the table regardless of the completeness of the block.
If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings are specified along with `rabbitmq_exchange_type`, then:
- `rabbitmq-consistent-hash-exchange` plugin must be enabled.
- `message_id` property of the published messages must be specified (unique for each message/batch).
- `rabbitmq-consistent-hash-exchange` plugin must be enabled.
- `message_id` property of the published messages must be specified (unique for each message/batch).
For insert query there is message metadata, which is added for each published message: `messageID` and `republished` flag (true, if published more than once) - can be accessed via message headers.

View File

@ -19,11 +19,11 @@ CREATE TABLE s3_engine_table (name String, value UInt32)
**Engine parameters**
- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path).
- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed.
- `format` — The [format](../../../interfaces/formats.md#formats) of the file.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3).
- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension.
- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path).
- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed.
- `format` — The [format](../../../interfaces/formats.md#formats) of the file.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3).
- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension.
### PARTITION BY
@ -51,18 +51,18 @@ SELECT * FROM s3_engine_table LIMIT 2;
```
## Virtual columns {#virtual-columns}
- `_path` — Path to the file.
- `_file` — Name of the file.
- `_path` — Path to the file.
- `_file` — Name of the file.
For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns).
## Implementation Details {#implementation-details}
- Reads and writes can be parallel
- Not supported:
- `ALTER` and `SELECT...SAMPLE` operations.
- Indexes.
- [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is possible, but not supported.
- Reads and writes can be parallel
- Not supported:
- `ALTER` and `SELECT...SAMPLE` operations.
- Indexes.
- [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is possible, but not supported.
:::note Zero-copy replication is not ready for production
Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use.
@ -72,10 +72,10 @@ For more information about virtual columns see [here](../../../engines/table-eng
`path` argument can specify multiple files using bash-like wildcards. For being processed file should exist and match to the whole path pattern. Listing of files is determined during `SELECT` (not at `CREATE` moment).
- `*` — Substitutes any number of any characters except `/` including empty string.
- `?` — Substitutes any single character.
- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`.
- `{N..M}` — Substitutes any number in range from N to M including both borders. N and M can have leading zeroes e.g. `000..078`.
- `*` — Substitutes any number of any characters except `/` including empty string.
- `?` — Substitutes any single character.
- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`.
- `{N..M}` — Substitutes any number in range from N to M including both borders. N and M can have leading zeroes e.g. `000..078`.
Constructions with `{}` are similar to the [remote](../../../sql-reference/table-functions/remote.md) table function.
@ -96,12 +96,12 @@ CREATE TABLE big_table (name String, value UInt32)
Suppose we have several files in CSV format with the following URIs on S3:
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/some_folder/some_file_1.csv'
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/some_folder/some_file_2.csv'
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/some_folder/some_file_3.csv'
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/another_folder/some_file_1.csv'
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/another_folder/some_file_2.csv'
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/another_folder/some_file_3.csv'
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/some_folder/some_file_1.csv'
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/some_folder/some_file_2.csv'
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/some_folder/some_file_3.csv'
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/another_folder/some_file_1.csv'
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/another_folder/some_file_2.csv'
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/another_folder/some_file_3.csv'
There are several ways to make a table consisting of all six files:
@ -131,14 +131,14 @@ CREATE TABLE table_with_asterisk (name String, value UInt32)
The following settings can be set before query execution or placed into configuration file.
- `s3_max_single_part_upload_size` — The maximum size of object to upload using singlepart upload to S3. Default value is `64Mb`.
- `s3_min_upload_part_size` — The minimum size of part to upload during multipart upload to [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Default value is `512Mb`.
- `s3_max_redirects` — Max number of S3 redirects hops allowed. Default value is `10`.
- `s3_single_read_retries` — The maximum number of attempts during single read. Default value is `4`.
- `s3_max_put_rps` — Maximum PUT requests per second rate before throttling. Default value is `0` (unlimited).
- `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`.
- `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited).
- `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`.
- `s3_max_single_part_upload_size` — The maximum size of object to upload using singlepart upload to S3. Default value is `64Mb`.
- `s3_min_upload_part_size` — The minimum size of part to upload during multipart upload to [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Default value is `512Mb`.
- `s3_max_redirects` — Max number of S3 redirects hops allowed. Default value is `10`.
- `s3_single_read_retries` — The maximum number of attempts during single read. Default value is `4`.
- `s3_max_put_rps` — Maximum PUT requests per second rate before throttling. Default value is `0` (unlimited).
- `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`.
- `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited).
- `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`.
Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration.
@ -146,17 +146,17 @@ Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max
The following settings can be specified in configuration file for given endpoint (which will be matched by exact prefix of a URL):
- `endpoint` — Specifies prefix of an endpoint. Mandatory.
- `access_key_id` and `secret_access_key` — Specifies credentials to use with given endpoint. Optional.
- `use_environment_credentials` — If set to `true`, S3 client will try to obtain credentials from environment variables and [Amazon EC2](https://en.wikipedia.org/wiki/Amazon_Elastic_Compute_Cloud) metadata for given endpoint. Optional, default value is `false`.
- `region` — Specifies S3 region name. Optional.
- `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Optional, default value is `false`.
- `expiration_window_seconds` — Grace period for checking if expiration-based credentials have expired. Optional, default value is `120`.
- `no_sign_request` - Ignore all the credentials so requests are not signed. Useful for accessing public buckets.
- `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be specified multiple times.
- `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional.
- `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional.
- `max_put_rps`, `max_put_burst`, `max_get_rps` and `max_get_burst` - Throttling settings (see description above) to use for specific endpoint instead of per query. Optional.
- `endpoint` — Specifies prefix of an endpoint. Mandatory.
- `access_key_id` and `secret_access_key` — Specifies credentials to use with given endpoint. Optional.
- `use_environment_credentials` — If set to `true`, S3 client will try to obtain credentials from environment variables and [Amazon EC2](https://en.wikipedia.org/wiki/Amazon_Elastic_Compute_Cloud) metadata for given endpoint. Optional, default value is `false`.
- `region` — Specifies S3 region name. Optional.
- `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Optional, default value is `false`.
- `expiration_window_seconds` — Grace period for checking if expiration-based credentials have expired. Optional, default value is `120`.
- `no_sign_request` - Ignore all the credentials so requests are not signed. Useful for accessing public buckets.
- `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be specified multiple times.
- `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional.
- `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional.
- `max_put_rps`, `max_put_burst`, `max_get_rps` and `max_get_burst` - Throttling settings (see description above) to use for specific endpoint instead of per query. Optional.
**Example:**
@ -191,4 +191,4 @@ CREATE TABLE big_table (name String, value UInt32)
## See also
- [s3 table function](../../../sql-reference/table-functions/s3.md)
- [s3 table function](../../../sql-reference/table-functions/s3.md)

View File

@ -20,8 +20,8 @@ The engine allows to import and export data to SQLite and supports queries to SQ
**Engine Parameters**
- `db_path` — Path to SQLite file with a database.
- `table` — Name of a table in the SQLite database.
- `db_path` — Path to SQLite file with a database.
- `table` — Name of a table in the SQLite database.
## Usage Example {#usage-example}
@ -56,5 +56,5 @@ SELECT * FROM sqlite_db.table2 ORDER BY col1;
**See Also**
- [SQLite](../../../engines/database-engines/sqlite.md) engine
- [sqlite](../../../sql-reference/table-functions/sqlite.md) table function
- [SQLite](../../../engines/database-engines/sqlite.md) engine
- [sqlite](../../../sql-reference/table-functions/sqlite.md) table function

View File

@ -10,9 +10,9 @@ These engines were developed for scenarios when you need to quickly write many s
Engines of the family:
- [StripeLog](/docs/en/engines/table-engines/log-family/stripelog.md)
- [Log](/docs/en/engines/table-engines/log-family/log.md)
- [TinyLog](/docs/en/engines/table-engines/log-family/tinylog.md)
- [StripeLog](/docs/en/engines/table-engines/log-family/stripelog.md)
- [Log](/docs/en/engines/table-engines/log-family/log.md)
- [TinyLog](/docs/en/engines/table-engines/log-family/tinylog.md)
`Log` family table engines can store data to [HDFS](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-hdfs) or [S3](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3) distributed file systems.
@ -20,21 +20,21 @@ Engines of the family:
Engines:
- Store data on a disk.
- Store data on a disk.
- Append data to the end of file when writing.
- Append data to the end of file when writing.
- Support locks for concurrent data access.
- Support locks for concurrent data access.
During `INSERT` queries, the table is locked, and other queries for reading and writing data both wait for the table to unlock. If there are no data writing queries, any number of data reading queries can be performed concurrently.
- Do not support [mutations](/docs/en/sql-reference/statements/alter/index.md#alter-mutations).
- Do not support [mutations](/docs/en/sql-reference/statements/alter/index.md#alter-mutations).
- Do not support indexes.
- Do not support indexes.
This means that `SELECT` queries for ranges of data are not efficient.
- Do not write data atomically.
- Do not write data atomically.
You can get a table with corrupted data if something breaks the write operation, for example, abnormal server shutdown.

View File

@ -29,8 +29,8 @@ The `StripeLog` engine stores all the columns in one file. For each `INSERT` que
For each table ClickHouse writes the files:
- `data.bin` — Data file.
- `index.mrk` — File with marks. Marks contain offsets for each column of each data block inserted.
- `data.bin` — Data file.
- `index.mrk` — File with marks. Marks contain offsets for each column of each data block inserted.
The `StripeLog` engine does not support the `ALTER UPDATE` and `ALTER DELETE` operations.

View File

@ -60,7 +60,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
All of the parameters excepting `sign` have the same meaning as in `MergeTree`.
- `sign` — Name of the column with the type of row: `1` — “state” row, `-1` — “cancel” row.
- `sign` — Name of the column with the type of row: `1` — “state” row, `-1` — “cancel” row.
Column Data Type — `Int8`.

View File

@ -77,11 +77,11 @@ The `name` column contains the names of the partition data parts. You can use th
Lets break down the name of the part: `201901_1_9_2_11`:
- `201901` is the partition name.
- `1` is the minimum number of the data block.
- `9` is the maximum number of the data block.
- `2` is the chunk level (the depth of the merge tree it is formed from).
- `11` is the mutation version (if a part mutated)
- `201901` is the partition name.
- `1` is the minimum number of the data block.
- `9` is the maximum number of the data block.
- `2` is the chunk level (the depth of the merge tree it is formed from).
- `11` is the mutation version (if a part mutated)
:::info
The parts of old-type tables have the name: `20190117_20190123_2_2_0` (minimum date - maximum date - minimum block number - maximum block number - level).
@ -165,9 +165,9 @@ Performance of such a query heavily depends on the table layout. Because of that
The key factors for a good performance:
- number of partitions involved in the query should be sufficiently large (more than `max_threads / 2`), otherwise query will underutilize the machine
- partitions shouldn't be too small, so batch processing won't degenerate into row-by-row processing
- partitions should be comparable in size, so all threads will do roughly the same amount of work
- number of partitions involved in the query should be sufficiently large (more than `max_threads / 2`), otherwise query will underutilize the machine
- partitions shouldn't be too small, so batch processing won't degenerate into row-by-row processing
- partitions should be comparable in size, so all threads will do roughly the same amount of work
:::info
It's recommended to apply some hash function to columns in `partition by` clause in order to distribute data evenly between partitions.
@ -175,6 +175,6 @@ It's recommended to apply some hash function to columns in `partition by` clause
Relevant settings are:
- `allow_aggregate_partitions_independently` - controls if the use of optimisation is enabled
- `force_aggregate_partitions_independently` - forces its use when it's applicable from the correctness standpoint, but getting disabled by internal logic that estimates its expediency
- `max_number_of_partitions_for_independent_aggregation` - hard limit on the maximal number of partitions table could have
- `allow_aggregate_partitions_independently` - controls if the use of optimisation is enabled
- `force_aggregate_partitions_independently` - forces its use when it's applicable from the correctness standpoint, but getting disabled by internal logic that estimates its expediency
- `max_number_of_partitions_for_independent_aggregation` - hard limit on the maximal number of partitions table could have

View File

@ -33,19 +33,19 @@ See a detailed description of the [CREATE TABLE](../../../sql-reference/statemen
A table for the Graphite data should have the following columns for the following data:
- Metric name (Graphite sensor). Data type: `String`.
- Metric name (Graphite sensor). Data type: `String`.
- Time of measuring the metric. Data type: `DateTime`.
- Time of measuring the metric. Data type: `DateTime`.
- Value of the metric. Data type: `Float64`.
- Value of the metric. Data type: `Float64`.
- Version of the metric. Data type: any numeric (ClickHouse saves the rows with the highest version or the last written if versions are the same. Other rows are deleted during the merge of data parts).
- Version of the metric. Data type: any numeric (ClickHouse saves the rows with the highest version or the last written if versions are the same. Other rows are deleted during the merge of data parts).
The names of these columns should be set in the rollup configuration.
**GraphiteMergeTree parameters**
- `config_section` — Name of the section in the configuration file, where are the rules of rollup set.
- `config_section` — Name of the section in the configuration file, where are the rules of rollup set.
**Query clauses**
@ -73,7 +73,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
All of the parameters excepting `config_section` have the same meaning as in `MergeTree`.
- `config_section` — Name of the section in the configuration file, where are the rules of rollup set.
- `config_section` — Name of the section in the configuration file, where are the rules of rollup set.
</details>
@ -141,18 +141,18 @@ When processing a row, ClickHouse checks the rules in the `pattern` sections. Ea
Fields for `pattern` and `default` sections:
- `rule_type` - a rule's type. It's applied only to a particular metrics. The engine use it to separate plain and tagged metrics. Optional parameter. Default value: `all`.
- `rule_type` - a rule's type. It's applied only to a particular metrics. The engine use it to separate plain and tagged metrics. Optional parameter. Default value: `all`.
It's unnecessary when performance is not critical, or only one metrics type is used, e.g. plain metrics. By default only one type of rules set is created. Otherwise, if any of special types is defined, two different sets are created. One for plain metrics (root.branch.leaf) and one for tagged metrics (root.branch.leaf;tag1=value1).
The default rules are ended up in both sets.
Valid values:
- `all` (default) - a universal rule, used when `rule_type` is omitted.
- `plain` - a rule for plain metrics. The field `regexp` is processed as regular expression.
- `tagged` - a rule for tagged metrics (metrics are stored in DB in the format of `someName?tag1=value1&tag2=value2&tag3=value3`). Regular expression must be sorted by tags' names, first tag must be `__name__` if exists. The field `regexp` is processed as regular expression.
- `tag_list` - a rule for tagged matrics, a simple DSL for easier metric description in graphite format `someName;tag1=value1;tag2=value2`, `someName`, or `tag1=value1;tag2=value2`. The field `regexp` is translated into a `tagged` rule. The sorting by tags' names is unnecessary, ti will be done automatically. A tag's value (but not a name) can be set as a regular expression, e.g. `env=(dev|staging)`.
- `regexp` A pattern for the metric name (a regular or DSL).
- `age` The minimum age of the data in seconds.
- `precision` How precisely to define the age of the data in seconds. Should be a divisor for 86400 (seconds in a day).
- `function` The name of the aggregating function to apply to data whose age falls within the range `[age, age + precision]`. Accepted functions: min / max / any / avg. The average is calculated imprecisely, like the average of the averages.
- `all` (default) - a universal rule, used when `rule_type` is omitted.
- `plain` - a rule for plain metrics. The field `regexp` is processed as regular expression.
- `tagged` - a rule for tagged metrics (metrics are stored in DB in the format of `someName?tag1=value1&tag2=value2&tag3=value3`). Regular expression must be sorted by tags' names, first tag must be `__name__` if exists. The field `regexp` is processed as regular expression.
- `tag_list` - a rule for tagged matrics, a simple DSL for easier metric description in graphite format `someName;tag1=value1;tag2=value2`, `someName`, or `tag1=value1;tag2=value2`. The field `regexp` is translated into a `tagged` rule. The sorting by tags' names is unnecessary, ti will be done automatically. A tag's value (but not a name) can be set as a regular expression, e.g. `env=(dev|staging)`.
- `regexp` A pattern for the metric name (a regular or DSL).
- `age` The minimum age of the data in seconds.
- `precision` How precisely to define the age of the data in seconds. Should be a divisor for 86400 (seconds in a day).
- `function` The name of the aggregating function to apply to data whose age falls within the range `[age, age + precision]`. Accepted functions: min / max / any / avg. The average is calculated imprecisely, like the average of the averages.
### Configuration Example without rules types {#configuration-example}

View File

@ -12,19 +12,19 @@ Engines in the `MergeTree` family are designed for inserting a very large amount
Main features:
- Stores data sorted by primary key.
- Stores data sorted by primary key.
This allows you to create a small sparse index that helps find data faster.
- Partitions can be used if the [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md) is specified.
- Partitions can be used if the [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md) is specified.
ClickHouse supports certain operations with partitions that are more efficient than general operations on the same data with the same result. ClickHouse also automatically cuts off the partition data where the partitioning key is specified in the query.
- Data replication support.
- Data replication support.
The family of `ReplicatedMergeTree` tables provides data replication. For more information, see [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md).
- Data sampling support.
- Data sampling support.
If necessary, you can set the data sampling method in the table.
@ -207,10 +207,10 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
**MergeTree() Parameters**
- `date-column` — The name of a column of the [Date](/docs/en/sql-reference/data-types/date.md) type. ClickHouse automatically creates partitions by month based on this column. The partition names are in the `"YYYYMM"` format.
- `sampling_expression` — An expression for sampling.
- `(primary, key)` — Primary key. Type: [Tuple()](/docs/en/sql-reference/data-types/tuple.md)
- `index_granularity` — The granularity of an index. The number of data rows between the “marks” of an index. The value 8192 is appropriate for most tasks.
- `date-column` — The name of a column of the [Date](/docs/en/sql-reference/data-types/date.md) type. ClickHouse automatically creates partitions by month based on this column. The partition names are in the `"YYYYMM"` format.
- `sampling_expression` — An expression for sampling.
- `(primary, key)` — Primary key. Type: [Tuple()](/docs/en/sql-reference/data-types/tuple.md)
- `index_granularity` — The granularity of an index. The number of data rows between the “marks” of an index. The value 8192 is appropriate for most tasks.
**Example**
@ -250,9 +250,9 @@ Take the `(CounterID, Date)` primary key as an example. In this case, the sortin
If the data query specifies:
- `CounterID in ('a', 'h')`, the server reads the data in the ranges of marks `[0, 3)` and `[6, 8)`.
- `CounterID IN ('a', 'h') AND Date = 3`, the server reads the data in the ranges of marks `[1, 3)` and `[7, 8)`.
- `Date = 3`, the server reads the data in the range of marks `[1, 10]`.
- `CounterID in ('a', 'h')`, the server reads the data in the ranges of marks `[0, 3)` and `[6, 8)`.
- `CounterID IN ('a', 'h') AND Date = 3`, the server reads the data in the ranges of marks `[1, 3)` and `[7, 8)`.
- `Date = 3`, the server reads the data in the range of marks `[1, 10]`.
The examples above show that it is always more effective to use an index than a full scan.
@ -268,18 +268,18 @@ You can use `Nullable`-typed expressions in the `PRIMARY KEY` and `ORDER BY` cla
The number of columns in the primary key is not explicitly limited. Depending on the data structure, you can include more or fewer columns in the primary key. This may:
- Improve the performance of an index.
- Improve the performance of an index.
If the primary key is `(a, b)`, then adding another column `c` will improve the performance if the following conditions are met:
- There are queries with a condition on column `c`.
- Long data ranges (several times longer than the `index_granularity`) with identical values for `(a, b)` are common. In other words, when adding another column allows you to skip quite long data ranges.
- There are queries with a condition on column `c`.
- Long data ranges (several times longer than the `index_granularity`) with identical values for `(a, b)` are common. In other words, when adding another column allows you to skip quite long data ranges.
- Improve data compression.
- Improve data compression.
ClickHouse sorts data by primary key, so the higher the consistency, the better the compression.
- Provide additional logic when merging data parts in the [CollapsingMergeTree](/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md/#table_engine-collapsingmergetree) and [SummingMergeTree](/docs/en/engines/table-engines/mergetree-family/summingmergetree.md) engines.
- Provide additional logic when merging data parts in the [CollapsingMergeTree](/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md/#table_engine-collapsingmergetree) and [SummingMergeTree](/docs/en/engines/table-engines/mergetree-family/summingmergetree.md) engines.
In this case it makes sense to specify the *sorting key* that is different from the primary key.
@ -483,25 +483,25 @@ Indexes of type `set` can be utilized by all functions. The other index types ar
Functions with a constant argument that is less than ngram size cant be used by `ngrambf_v1` for query optimization.
(*) For `hasTokenCaseInsensitve` and `hasTokenCaseInsensitive` to be effective, the `tokenbf_v1` index must be created on lowercased data, for example `INDEX idx (lower(str_col)) TYPE tokenbf_v1(512, 3, 0)`.
(*) For `hasTokenCaseInsensitive` and `hasTokenCaseInsensitiveOrNull` to be effective, the `tokenbf_v1` index must be created on lowercased data, for example `INDEX idx (lower(str_col)) TYPE tokenbf_v1(512, 3, 0)`.
:::note
Bloom filters can have false positive matches, so the `ngrambf_v1`, `tokenbf_v1`, and `bloom_filter` indexes can not be used for optimizing queries where the result of a function is expected to be false.
For example:
- Can be optimized:
- `s LIKE '%test%'`
- `NOT s NOT LIKE '%test%'`
- `s = 1`
- `NOT s != 1`
- `startsWith(s, 'test')`
- Can not be optimized:
- `NOT s LIKE '%test%'`
- `s NOT LIKE '%test%'`
- `NOT s = 1`
- `s != 1`
- `NOT startsWith(s, 'test')`
- Can be optimized:
- `s LIKE '%test%'`
- `NOT s NOT LIKE '%test%'`
- `s = 1`
- `NOT s != 1`
- `startsWith(s, 'test')`
- Can not be optimized:
- `NOT s LIKE '%test%'`
- `s NOT LIKE '%test%'`
- `NOT s = 1`
- `s != 1`
- `NOT startsWith(s, 'test')`
:::
@ -614,11 +614,11 @@ TTL expr
Type of TTL rule may follow each TTL expression. It affects an action which is to be done once the expression is satisfied (reaches current time):
- `DELETE` - delete expired rows (default action);
- `RECOMPRESS codec_name` - recompress data part with the `codec_name`;
- `TO DISK 'aaa'` - move part to the disk `aaa`;
- `TO VOLUME 'bbb'` - move part to the disk `bbb`;
- `GROUP BY` - aggregate expired rows.
- `DELETE` - delete expired rows (default action);
- `RECOMPRESS codec_name` - recompress data part with the `codec_name`;
- `TO DISK 'aaa'` - move part to the disk `aaa`;
- `TO VOLUME 'bbb'` - move part to the disk `bbb`;
- `GROUP BY` - aggregate expired rows.
`DELETE` action can be used together with `WHERE` clause to delete only some of the expired rows based on a filtering condition:
``` sql
@ -722,10 +722,10 @@ Data part is the minimum movable unit for `MergeTree`-engine tables. The data be
### Terms {#terms}
- Disk — Block device mounted to the filesystem.
- Default disk — Disk that stores the path specified in the [path](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-path) server setting.
- Volume — Ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)).
- Storage policy — Set of volumes and the rules for moving data between them.
- Disk — Block device mounted to the filesystem.
- Default disk — Disk that stores the path specified in the [path](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-path) server setting.
- Volume — Ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)).
- Storage policy — Set of volumes and the rules for moving data between them.
The names given to the described entities can be found in the system tables, [system.storage_policies](/docs/en/operations/system-tables/storage_policies.md/#system_tables-storage_policies) and [system.disks](/docs/en/operations/system-tables/disks.md/#system_tables-disks). To apply one of the configured storage policies for a table, use the `storage_policy` setting of `MergeTree`-engine family tables.
@ -759,9 +759,9 @@ Configuration structure:
Tags:
- `<disk_name_N>` — Disk name. Names must be different for all disks.
- `path` — path under which a server will store data (`data` and `shadow` folders), should be terminated with /.
- `keep_free_space_bytes` — the amount of free disk space to be reserved.
- `<disk_name_N>` — Disk name. Names must be different for all disks.
- `path` — path under which a server will store data (`data` and `shadow` folders), should be terminated with /.
- `keep_free_space_bytes` — the amount of free disk space to be reserved.
The order of the disk definition is not important.
@ -797,14 +797,14 @@ Storage policies configuration markup:
Tags:
- `policy_name_N` — Policy name. Policy names must be unique.
- `volume_name_N` — Volume name. Volume names must be unique.
- `disk` — a disk within a volume.
- `max_data_part_size_bytes` — the maximum size of a part that can be stored on any of the volumes disks. If the a size of a merged part estimated to be bigger than `max_data_part_size_bytes` then this part will be written to a next volume. Basically this feature allows to keep new/small parts on a hot (SSD) volume and move them to a cold (HDD) volume when they reach large size. Do not use this setting if your policy has only one volume.
- `move_factor` — when the amount of available space gets lower than this factor, data automatically starts to move on the next volume if any (by default, 0.1). ClickHouse sorts existing parts by size from largest to smallest (in descending order) and selects parts with the total size that is sufficient to meet the `move_factor` condition. If the total size of all parts is insufficient, all parts will be moved.
- `prefer_not_to_merge` — Disables merging of data parts on this volume. When this setting is enabled, merging data on this volume is not allowed. This allows controlling how ClickHouse works with slow disks.
- `perform_ttl_move_on_insert` — Disables TTL move on data part INSERT. By default if we insert a data part that already expired by the TTL move rule it immediately goes to a volume/disk declared in move rule. This can significantly slowdown insert in case if destination volume/disk is slow (e.g. S3).
- `load_balancing` - Policy for disk balancing, `round_robin` or `least_used`.
- `policy_name_N` — Policy name. Policy names must be unique.
- `volume_name_N` — Volume name. Volume names must be unique.
- `disk` — a disk within a volume.
- `max_data_part_size_bytes` — the maximum size of a part that can be stored on any of the volumes disks. If the a size of a merged part estimated to be bigger than `max_data_part_size_bytes` then this part will be written to a next volume. Basically this feature allows to keep new/small parts on a hot (SSD) volume and move them to a cold (HDD) volume when they reach large size. Do not use this setting if your policy has only one volume.
- `move_factor` — when the amount of available space gets lower than this factor, data automatically starts to move on the next volume if any (by default, 0.1). ClickHouse sorts existing parts by size from largest to smallest (in descending order) and selects parts with the total size that is sufficient to meet the `move_factor` condition. If the total size of all parts is insufficient, all parts will be moved.
- `prefer_not_to_merge` — Disables merging of data parts on this volume. When this setting is enabled, merging data on this volume is not allowed. This allows controlling how ClickHouse works with slow disks.
- `perform_ttl_move_on_insert` — Disables TTL move on data part INSERT. By default if we insert a data part that already expired by the TTL move rule it immediately goes to a volume/disk declared in move rule. This can significantly slowdown insert in case if destination volume/disk is slow (e.g. S3).
- `load_balancing` - Policy for disk balancing, `round_robin` or `least_used`.
Cofiguration examples:
@ -880,10 +880,10 @@ The number of threads performing background moves of data parts can be changed b
In the case of `MergeTree` tables, data is getting to disk in different ways:
- As a result of an insert (`INSERT` query).
- During background merges and [mutations](/docs/en/sql-reference/statements/alter/index.md#alter-mutations).
- When downloading from another replica.
- As a result of partition freezing [ALTER TABLE … FREEZE PARTITION](/docs/en/sql-reference/statements/alter/partition.md/#alter_freeze-partition).
- As a result of an insert (`INSERT` query).
- During background merges and [mutations](/docs/en/sql-reference/statements/alter/index.md#alter-mutations).
- When downloading from another replica.
- As a result of partition freezing [ALTER TABLE … FREEZE PARTITION](/docs/en/sql-reference/statements/alter/partition.md/#alter_freeze-partition).
In all these cases except for mutations and partition freezing, a part is stored on a volume and a disk according to the given storage policy:
@ -956,30 +956,30 @@ ClickHouse versions 22.3 through 22.7 use a different cache configuration, see [
Required parameters:
- `endpoint` — S3 endpoint URL in `path` or `virtual hosted` [styles](https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html). Endpoint URL should contain a bucket and root path to store data.
- `access_key_id` — S3 access key id.
- `secret_access_key` — S3 secret access key.
- `endpoint` — S3 endpoint URL in `path` or `virtual hosted` [styles](https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html). Endpoint URL should contain a bucket and root path to store data.
- `access_key_id` — S3 access key id.
- `secret_access_key` — S3 secret access key.
Optional parameters:
- `region` — S3 region name.
- `support_batch_delete` — This controls the check to see if batch deletes are supported. Set this to `false` when using Google Cloud Storage (GCS) as GCS does not support batch deletes and preventing the checks will prevent error messages in the logs.
- `use_environment_credentials` — Reads AWS credentials from the Environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY and AWS_SESSION_TOKEN if they exist. Default value is `false`.
- `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Default value is `false`.
- `expiration_window_seconds` — Grace period for checking if expiration-based credentials have expired. Optional, default value is `120`.
- `proxy` — Proxy configuration for S3 endpoint. Each `uri` element inside `proxy` block should contain a proxy URL.
- `connect_timeout_ms` — Socket connect timeout in milliseconds. Default value is `10 seconds`.
- `request_timeout_ms` — Request timeout in milliseconds. Default value is `5 seconds`.
- `retry_attempts` — Number of retry attempts in case of failed request. Default value is `10`.
- `single_read_retries` — Number of retry attempts in case of connection drop during read. Default value is `4`.
- `min_bytes_for_seek` — Minimal number of bytes to use seek operation instead of sequential read. Default value is `1 Mb`.
- `metadata_path` — Path on local FS to store metadata files for S3. Default value is `/var/lib/clickhouse/disks/<disk_name>/`.
- `skip_access_check` — If true, disk access checks will not be performed on disk start-up. Default value is `false`.
- `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set.
- `s3_max_put_rps` — Maximum PUT requests per second rate before throttling. Default value is `0` (unlimited).
- `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`.
- `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited).
- `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`.
- `region` — S3 region name.
- `support_batch_delete` — This controls the check to see if batch deletes are supported. Set this to `false` when using Google Cloud Storage (GCS) as GCS does not support batch deletes and preventing the checks will prevent error messages in the logs.
- `use_environment_credentials` — Reads AWS credentials from the Environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY and AWS_SESSION_TOKEN if they exist. Default value is `false`.
- `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Default value is `false`.
- `expiration_window_seconds` — Grace period for checking if expiration-based credentials have expired. Optional, default value is `120`.
- `proxy` — Proxy configuration for S3 endpoint. Each `uri` element inside `proxy` block should contain a proxy URL.
- `connect_timeout_ms` — Socket connect timeout in milliseconds. Default value is `10 seconds`.
- `request_timeout_ms` — Request timeout in milliseconds. Default value is `5 seconds`.
- `retry_attempts` — Number of retry attempts in case of failed request. Default value is `10`.
- `single_read_retries` — Number of retry attempts in case of connection drop during read. Default value is `4`.
- `min_bytes_for_seek` — Minimal number of bytes to use seek operation instead of sequential read. Default value is `1 Mb`.
- `metadata_path` — Path on local FS to store metadata files for S3. Default value is `/var/lib/clickhouse/disks/<disk_name>/`.
- `skip_access_check` — If true, disk access checks will not be performed on disk start-up. Default value is `false`.
- `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set.
- `s3_max_put_rps` — Maximum PUT requests per second rate before throttling. Default value is `0` (unlimited).
- `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`.
- `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited).
- `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`.
### Configuring the cache
@ -994,12 +994,12 @@ This is the cache configuration from above:
```
These parameters define the cache layer:
- `type` — If a disk is of type `cache` it caches mark and index files in memory.
- `disk` — The name of the disk that will be cached.
- `type` — If a disk is of type `cache` it caches mark and index files in memory.
- `disk` — The name of the disk that will be cached.
Cache parameters:
- `path` — The path where metadata for the cache is stored.
- `max_size` — The size (amount of memory) that the cache can grow to.
- `path` — The path where metadata for the cache is stored.
- `max_size` — The size (amount of memory) that the cache can grow to.
:::tip
There are several other cache parameters that you can use to tune your storage, see [using local cache](/docs/en/operations/storing-data.md/#using-local-cache) for the details.
@ -1100,9 +1100,9 @@ Examples of working configurations can be found in integration tests directory (
## Virtual Columns {#virtual-columns}
- `_part` — Name of a part.
- `_part_index` — Sequential index of the part in the query result.
- `_partition_id` — Name of a partition.
- `_part_uuid` — Unique part identifier (if enabled MergeTree setting `assign_part_uuids`).
- `_partition_value` — Values (a tuple) of a `partition by` expression.
- `_sample_factor` — Sample factor (from the query).
- `_part` — Name of a part.
- `_part_index` — Sequential index of the part in the query result.
- `_partition_id` — Name of a partition.
- `_part_uuid` — Unique part identifier (if enabled MergeTree setting `assign_part_uuids`).
- `_partition_value` — Values (a tuple) of a `partition by` expression.
- `_sample_factor` — Sample factor (from the query).

View File

@ -20,12 +20,12 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2],
...
) ENGINE = ReplacingMergeTree([ver])
) ENGINE = ReplacingMergeTree([ver [, is_deleted]])
[PARTITION BY expr]
[ORDER BY expr]
[PRIMARY KEY expr]
[SAMPLE BY expr]
[SETTINGS name=value, ...]
[SETTINGS name=value, clean_deleted_rows=value, ...]
```
For a description of request parameters, see [statement description](../../../sql-reference/statements/create/table.md).
@ -88,6 +88,18 @@ SELECT * FROM mySecondReplacingMT FINAL;
└─────┴─────────┴─────────────────────┘
```
### is_deleted
`is_deleted` — Name of the column with the type of row: `1` is a “deleted“ row, `0` is a “state“ row.
Column data type — `Int8`.
Can only be enabled when `ver` is used.
The row is deleted when use the `OPTIMIZE ... FINAL CLEANUP`, or `OPTIMIZE ... FINAL` if the engine settings `clean_deleted_rows` has been set to `Always`.
No matter the operation on the data, the version must be increased. If two inserted rows have the same version number, the last inserted one is the one kept.
## Query clauses
When creating a `ReplacingMergeTree` table the same [clauses](../../../engines/table-engines/mergetree-family/mergetree.md) are required, as when creating a `MergeTree` table.
@ -111,6 +123,6 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
All of the parameters excepting `ver` have the same meaning as in `MergeTree`.
- `ver` - column with the version. Optional parameter. For a description, see the text above.
- `ver` - column with the version. Optional parameter. For a description, see the text above.
</details>

View File

@ -26,13 +26,13 @@ ENGINE = ReplicatedReplacingMergeTree
Replication is only supported for tables in the MergeTree family:
- ReplicatedMergeTree
- ReplicatedSummingMergeTree
- ReplicatedReplacingMergeTree
- ReplicatedAggregatingMergeTree
- ReplicatedCollapsingMergeTree
- ReplicatedVersionedCollapsingMergeTree
- ReplicatedGraphiteMergeTree
- ReplicatedMergeTree
- ReplicatedSummingMergeTree
- ReplicatedReplacingMergeTree
- ReplicatedAggregatingMergeTree
- ReplicatedCollapsingMergeTree
- ReplicatedVersionedCollapsingMergeTree
- ReplicatedGraphiteMergeTree
Replication works at the level of an individual table, not the entire server. A server can store both replicated and non-replicated tables at the same time.
@ -42,9 +42,9 @@ Compressed data for `INSERT` and `ALTER` queries is replicated (for more informa
`CREATE`, `DROP`, `ATTACH`, `DETACH` and `RENAME` queries are executed on a single server and are not replicated:
- The `CREATE TABLE` query creates a new replicatable table on the server where the query is run. If this table already exists on other servers, it adds a new replica.
- The `DROP TABLE` query deletes the replica located on the server where the query is run.
- The `RENAME` query renames the table on one of the replicas. In other words, replicated tables can have different names on different replicas.
- The `CREATE TABLE` query creates a new replicatable table on the server where the query is run. If this table already exists on other servers, it adds a new replica.
- The `DROP TABLE` query deletes the replica located on the server where the query is run.
- The `RENAME` query renames the table on one of the replicas. In other words, replicated tables can have different names on different replicas.
ClickHouse uses [ClickHouse Keeper](/docs/en/guides/sre/keeper/index.md) for storing replicas meta information. It is possible to use ZooKeeper version 3.4.5 or newer, but ClickHouse Keeper is recommended.
@ -316,8 +316,8 @@ Create a MergeTree table with a different name. Move all the data from the direc
If you want to get rid of a `ReplicatedMergeTree` table without launching the server:
- Delete the corresponding `.sql` file in the metadata directory (`/var/lib/clickhouse/metadata/`).
- Delete the corresponding path in ClickHouse Keeper (`/path_to_table/replica_name`).
- Delete the corresponding `.sql` file in the metadata directory (`/var/lib/clickhouse/metadata/`).
- Delete the corresponding path in ClickHouse Keeper (`/path_to_table/replica_name`).
After this, you can launch the server, create a `MergeTree` table, move the data to its directory, and then restart the server.
@ -327,8 +327,8 @@ If the data in ClickHouse Keeper was lost or damaged, you can save data by movin
**See Also**
- [background_schedule_pool_size](/docs/en/operations/server-configuration-parameters/settings.md/#background_schedule_pool_size)
- [background_fetches_pool_size](/docs/en/operations/server-configuration-parameters/settings.md/#background_fetches_pool_size)
- [execute_merges_on_single_replica_time_threshold](/docs/en/operations/settings/settings.md/#execute-merges-on-single-replica-time-threshold)
- [max_replicated_fetches_network_bandwidth](/docs/en/operations/settings/merge-tree-settings.md/#max_replicated_fetches_network_bandwidth)
- [max_replicated_sends_network_bandwidth](/docs/en/operations/settings/merge-tree-settings.md/#max_replicated_sends_network_bandwidth)
- [background_schedule_pool_size](/docs/en/operations/server-configuration-parameters/settings.md/#background_schedule_pool_size)
- [background_fetches_pool_size](/docs/en/operations/server-configuration-parameters/settings.md/#background_fetches_pool_size)
- [execute_merges_on_single_replica_time_threshold](/docs/en/operations/settings/settings.md/#execute-merges-on-single-replica-time-threshold)
- [max_replicated_fetches_network_bandwidth](/docs/en/operations/settings/merge-tree-settings.md/#max_replicated_fetches_network_bandwidth)
- [max_replicated_sends_network_bandwidth](/docs/en/operations/settings/merge-tree-settings.md/#max_replicated_sends_network_bandwidth)

View File

@ -59,7 +59,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
All of the parameters excepting `columns` have the same meaning as in `MergeTree`.
- `columns` — tuple with names of columns values of which will be summarized. Optional parameter. For a description, see the text above.
- `columns` — tuple with names of columns values of which will be summarized. Optional parameter. For a description, see the text above.
</details>
@ -122,8 +122,8 @@ Table can have nested data structures that are processed in a special way.
If the name of a nested table ends with `Map` and it contains at least two columns that meet the following criteria:
- the first column is numeric `(*Int*, Date, DateTime)` or a string `(String, FixedString)`, lets call it `key`,
- the other columns are arithmetic `(*Int*, Float32/64)`, lets call it `(values...)`,
- the first column is numeric `(*Int*, Date, DateTime)` or a string `(String, FixedString)`, lets call it `key`,
- the other columns are arithmetic `(*Int*, Float32/64)`, lets call it `(values...)`,
then this nested table is interpreted as a mapping of `key => (values...)`, and when merging its rows, the elements of two data sets are merged by `key` with a summation of the corresponding `(values...)`.

View File

@ -8,8 +8,8 @@ sidebar_label: VersionedCollapsingMergeTree
This engine:
- Allows quick writing of object states that are continually changing.
- Deletes old object states in the background. This significantly reduces the volume of storage.
- Allows quick writing of object states that are continually changing.
- Deletes old object states in the background. This significantly reduces the volume of storage.
See the section [Collapsing](#table_engines_versionedcollapsingmergetree) for details.
@ -73,11 +73,11 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
All of the parameters except `sign` and `version` have the same meaning as in `MergeTree`.
- `sign` — Name of the column with the type of row: `1` is a “state” row, `-1` is a “cancel” row.
- `sign` — Name of the column with the type of row: `1` is a “state” row, `-1` is a “cancel” row.
Column Data Type — `Int8`.
- `version` — Name of the column with the version of the object state.
- `version` — Name of the column with the version of the object state.
The column data type should be `UInt*`.

View File

@ -97,4 +97,4 @@ select * from products limit 1;
**See Also**
- [Dictionary function](../../../sql-reference/table-functions/dictionary.md#dictionary-function)
- [Dictionary function](../../../sql-reference/table-functions/dictionary.md#dictionary-function)

View File

@ -229,8 +229,8 @@ A simple remainder from the division is a limited solution for sharding and isn
You should be concerned about the sharding scheme in the following cases:
- Queries are used that require joining data (`IN` or `JOIN`) by a specific key. If data is sharded by this key, you can use local `IN` or `JOIN` instead of `GLOBAL IN` or `GLOBAL JOIN`, which is much more efficient.
- A large number of servers is used (hundreds or more) with a large number of small queries, for example, queries for data of individual clients (e.g. websites, advertisers, or partners). In order for the small queries to not affect the entire cluster, it makes sense to locate data for a single client on a single shard. Alternatively, you can set up bi-level sharding: divide the entire cluster into “layers”, where a layer may consist of multiple shards. Data for a single client is located on a single layer, but shards can be added to a layer as necessary, and data is randomly distributed within them. `Distributed` tables are created for each layer, and a single shared distributed table is created for global queries.
- Queries are used that require joining data (`IN` or `JOIN`) by a specific key. If data is sharded by this key, you can use local `IN` or `JOIN` instead of `GLOBAL IN` or `GLOBAL JOIN`, which is much more efficient.
- A large number of servers is used (hundreds or more) with a large number of small queries, for example, queries for data of individual clients (e.g. websites, advertisers, or partners). In order for the small queries to not affect the entire cluster, it makes sense to locate data for a single client on a single shard. Alternatively, you can set up bi-level sharding: divide the entire cluster into “layers”, where a layer may consist of multiple shards. Data for a single client is located on a single layer, but shards can be added to a layer as necessary, and data is randomly distributed within them. `Distributed` tables are created for each layer, and a single shared distributed table is created for global queries.
Data is written asynchronously. When inserted in the table, the data block is just written to the local file system. The data is sent to the remote servers in the background as soon as possible. The periodicity for sending data is managed by the [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) and [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) settings. The `Distributed` engine sends each file with inserted data separately, but you can enable batch sending of files with the [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) setting. This setting improves cluster performance by better utilizing local server and network resources. You should check whether data is sent successfully by checking the list of files (data waiting to be sent) in the table directory: `/var/lib/clickhouse/data/database/table/`. The number of threads performing background tasks can be set by [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting.
@ -256,6 +256,6 @@ Since [remote](../../../sql-reference/table-functions/remote.md) and [cluster](.
**See Also**
- [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) description
- [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting
- [shardNum()](../../../sql-reference/functions/other-functions.md#shard-num) and [shardCount()](../../../sql-reference/functions/other-functions.md#shard-count) functions
- [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) description
- [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting
- [shardNum()](../../../sql-reference/functions/other-functions.md#shard-num) and [shardCount()](../../../sql-reference/functions/other-functions.md#shard-count) functions

View File

@ -10,9 +10,9 @@ The File table engine keeps the data in a file in one of the supported [file for
Usage scenarios:
- Data export from ClickHouse to file.
- Convert data from one format to another.
- Updating data in ClickHouse via editing a file on a disk.
- Data export from ClickHouse to file.
- Convert data from one format to another.
- Updating data in ClickHouse via editing a file on a disk.
## Usage in ClickHouse Server {#usage-in-clickhouse-server}
@ -78,14 +78,14 @@ $ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64
## Details of Implementation {#details-of-implementation}
- Multiple `SELECT` queries can be performed concurrently, but `INSERT` queries will wait each other.
- Supported creating new file by `INSERT` query.
- If file exists, `INSERT` would append new values in it.
- Not supported:
- `ALTER`
- `SELECT ... SAMPLE`
- Indices
- Replication
- Multiple `SELECT` queries can be performed concurrently, but `INSERT` queries will wait each other.
- Supported creating new file by `INSERT` query.
- If file exists, `INSERT` would append new values in it.
- Not supported:
- `ALTER`
- `SELECT ... SAMPLE`
- Indices
- Replication
## PARTITION BY

View File

@ -9,8 +9,8 @@ The GenerateRandom table engine produces random data for given table schema.
Usage examples:
- Use in test to populate reproducible large table.
- Generate random input for fuzzing tests.
- Use in test to populate reproducible large table.
- Generate random input for fuzzing tests.
## Usage in ClickHouse Server {#usage-in-clickhouse-server}
@ -49,9 +49,9 @@ SELECT * FROM generate_engine_table LIMIT 3
## Details of Implementation {#details-of-implementation}
- Not supported:
- `ALTER`
- `SELECT ... SAMPLE`
- `INSERT`
- Indices
- Replication
- Not supported:
- `ALTER`
- `SELECT ... SAMPLE`
- `INSERT`
- Indices
- Replication

View File

@ -8,8 +8,8 @@ sidebar_label: Special
There are three main categories of table engines:
- [MergeTree engine family](../../../engines/table-engines/mergetree-family/index.md) for main production use.
- [Log engine family](../../../engines/table-engines/log-family/index.md) for small temporary data.
- [Table engines for integrations](../../../engines/table-engines/integrations/index.md).
- [MergeTree engine family](../../../engines/table-engines/mergetree-family/index.md) for main production use.
- [Log engine family](../../../engines/table-engines/log-family/index.md) for small temporary data.
- [Table engines for integrations](../../../engines/table-engines/integrations/index.md).
The remaining engines are unique in their purpose and are not grouped into families yet, thus they are placed in this “special” category.

View File

@ -54,8 +54,8 @@ You can use `INSERT` queries to add data to the `Join`-engine tables. If the tab
Main use-cases for `Join`-engine tables are following:
- Place the table to the right side in a `JOIN` clause.
- Call the [joinGet](/docs/en/sql-reference/functions/other-functions.md/#joinget) function, which lets you extract data from the table the same way as from a dictionary.
- Place the table to the right side in a `JOIN` clause.
- Call the [joinGet](/docs/en/sql-reference/functions/other-functions.md/#joinget) function, which lets you extract data from the table the same way as from a dictionary.
### Deleting Data {#deleting-data}

View File

@ -78,11 +78,11 @@ SELECT * FROM WatchLog;
## Virtual Columns {#virtual-columns}
- `_table` — Contains the name of the table from which data was read. Type: [String](../../../sql-reference/data-types/string.md).
- `_table` — Contains the name of the table from which data was read. Type: [String](../../../sql-reference/data-types/string.md).
You can set the constant conditions on `_table` in the `WHERE/PREWHERE` clause (for example, `WHERE _table='xyz'`). In this case the read operation is performed only for that tables where the condition on `_table` is satisfied, so the `_table` column acts as an index.
**See Also**
- [Virtual columns](../../../engines/table-engines/special/index.md#table_engines-virtual_columns)
- [merge](../../../sql-reference/table-functions/merge.md) table function
- [Virtual columns](../../../engines/table-engines/special/index.md#table_engines-virtual_columns)
- [merge](../../../sql-reference/table-functions/merge.md) table function

View File

@ -91,11 +91,11 @@ SELECT * FROM url_engine_table
## Details of Implementation {#details-of-implementation}
- Reads and writes can be parallel
- Not supported:
- `ALTER` and `SELECT...SAMPLE` operations.
- Indexes.
- Replication.
- Reads and writes can be parallel
- Not supported:
- `ALTER` and `SELECT...SAMPLE` operations.
- Indexes.
- Replication.
## PARTITION BY

View File

@ -43,7 +43,7 @@ The rows look like:
```response
┌─marketplace─┬─customer_id─┬─review_id──────┬─product_id─┬─product_parent─┬─product_title──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─product_category─┬─star_rating─┬─helpful_votes─┬─total_votes─┬─vine──┬─verified_purchase─┬─review_headline───────────┬─review_body────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─review_date─┐
│ US │ 16414143 │ R3W4P9UBGNGH1U │ B00YL0EKWE │ 852431543 │ LG G4 Case Hard Transparent Slim Clear Cover for LG G4 │ Wireless │ 2 │ 1 │ 3 │ false │ true │ Looks good, functions meh │ 2 issues - Once I turned on the circle apps and installed this case, my battery drained twice as fast as usual. I ended up turning off the circle apps, which kind of makes the case just a case... with a hole in it. Second, the wireless charging doesn't work. I have a Motorola 360 watch and a Qi charging pad. The watch charges fine but this case doesn't. But hey, it looks nice. │ 2015-08-31 │
│ US │ 16414143 │ R3W4P9UBGNGH1U │ B00YL0EKWE │ 852431543 │ LG G4 Case Hard Transparent Slim Clear Cover for LG G4 │ Wireless │ 2 │ 1 │ 3 │ false │ true │ Looks good, functions meh │ 2 issues - Once I turned on the circle apps and installed this case, my battery drained twice as fast as usual. I ended up turning off the circle apps, which kind of makes the case just a case... with a hole in it. Second, the wireless charging doesn't work. I have a Motorola 360 watch and a Qi charging pad. The watch charges fine but this case doesn't. But hey, it looks nice. │ 2015-08-31 │
│ US │ 50800750 │ R15V54KBMTQWAY │ B00XK95RPQ │ 516894650 │ Selfie Stick Fiblastiq&trade; Extendable Wireless Bluetooth Selfie Stick with built-in Bluetooth Adjustable Phone Holder │ Wireless │ 4 │ 0 │ 0 │ false │ false │ A fun little gadget │ Im embarrassed to admit that until recently, I have had a very negative opinion about “selfie sticks” aka “monopods” aka “narcissticks.” But having reviewed a number of them recently, theyre growing on me. This one is pretty nice and simple to set up and with easy instructions illustrated on the back of the box (not sure why some reviewers have stated that there are no instructions when they are clearly printed on the box unless they received different packaging than I did). Once assembled, the pairing via bluetooth and use of the stick are easy and intuitive. Nothing to it.<br /><br />The stick comes with a USB charging cable but arrived with a charge so you can use it immediately, though its probably a good idea to charge it right away so that you have no interruption of use out of the box. Make sure the stick is switched to on (it will light up) and extend your stick to the length you desire up to about a yards length and snap away.<br /><br />The phone clamp held the phone sturdily so I wasnt worried about it slipping out. But the longer you extend the stick, the harder it is to maneuver. But that will happen with any stick and is not specific to this one in particular.<br /><br />Two things that could improve this: 1) add the option to clamp this in portrait orientation instead of having to try and hold the stick at the portrait angle, which makes it feel unstable; 2) add the opening for a tripod so that this can be used to sit upright on a table for skyping and facetime eliminating the need to hold the phone up with your hand, causing fatigue.<br /><br />But other than that, this is a nice quality monopod for a variety of picture taking opportunities.<br /><br />I received a sample in exchange for my honest opinion. │ 2015-08-31 │
│ US │ 15184378 │ RY8I449HNXSVF │ B00SXRXUKO │ 984297154 │ Tribe AB40 Water Resistant Sports Armband with Key Holder for 4.7-Inch iPhone 6S/6/5/5S/5C, Galaxy S4 + Screen Protector - Dark Pink │ Wireless │ 5 │ 0 │ 0 │ false │ true │ Five Stars │ Fits iPhone 6 well │ 2015-08-31 │
│ US │ 10203548 │ R18TLJYCKJFLSR │ B009V5X1CE │ 279912704 │ RAVPower® Element 10400mAh External Battery USB Portable Charger (Dual USB Outputs, Ultra Compact Design), Travel Charger for iPhone 6,iPhone 6 plus,iPhone 5, 5S, 5C, 4S, 4, iPad Air, 4, 3, 2, Mini 2 (Apple adapters not included); Samsung Galaxy S5, S4, S3, S2, Note 3, Note 2; HTC One, EVO, Thunderbolt, Incredible, Droid DNA, Motorola ATRIX, Droid, Moto X, Google Glass, Nexus 4, Nexus 5, Nexus 7, │ Wireless │ 5 │ 0 │ 0 │ false │ true │ Great charger │ Great charger. I easily get 3+ charges on a Samsung Galaxy 3. Works perfectly for camping trips or long days on the boat. │ 2015-08-31 │

View File

@ -380,7 +380,7 @@ decide that we would look at the types of crimes reported over time in the five
New York City. These fields might be then included in the `ORDER BY`:
| Column | Description (from the data dictionary) |
| ----------- | --------------------------------------------------- |
| ----------- | --------------------------------------------------- |
| OFNS_DESC | Description of offense corresponding with key code |
| RPT_DT | Date event was reported to police |
| BORO_NM | The name of the borough in which the incident occurred |

View File

@ -390,9 +390,9 @@ You can also play with the data in Playground, [example](https://play.clickhouse
This performance test was created by Vadim Tkachenko. See:
- https://www.percona.com/blog/2009/10/02/analyzing-air-traffic-performance-with-infobright-and-monetdb/
- https://www.percona.com/blog/2009/10/26/air-traffic-queries-in-luciddb/
- https://www.percona.com/blog/2009/11/02/air-traffic-queries-in-infinidb-early-alpha/
- https://www.percona.com/blog/2014/04/21/using-apache-hadoop-and-impala-together-with-mysql-for-data-analysis/
- https://www.percona.com/blog/2016/01/07/apache-spark-with-air-ontime-performance-data/
- http://nickmakos.blogspot.ru/2012/08/analyzing-air-traffic-performance-with.html
- https://www.percona.com/blog/2009/10/02/analyzing-air-traffic-performance-with-infobright-and-monetdb/
- https://www.percona.com/blog/2009/10/26/air-traffic-queries-in-luciddb/
- https://www.percona.com/blog/2009/11/02/air-traffic-queries-in-infinidb-early-alpha/
- https://www.percona.com/blog/2014/04/21/using-apache-hadoop-and-impala-together-with-mysql-for-data-analysis/
- https://www.percona.com/blog/2016/01/07/apache-spark-with-air-ontime-performance-data/
- http://nickmakos.blogspot.ru/2012/08/analyzing-air-traffic-performance-with.html

View File

@ -50,13 +50,13 @@ clickhouse-client --query "
This is a showcase how to parse custom CSV, as it requires multiple tunes.
Explanation:
- The dataset is in CSV format, but it requires some preprocessing on insertion; we use table function [input](../../sql-reference/table-functions/input.md) to perform preprocessing;
- The structure of CSV file is specified in the argument of the table function `input`;
- The field `num` (row number) is unneeded - we parse it from file and ignore;
- We use `FORMAT CSVWithNames` but the header in CSV will be ignored (by command line parameter `--input_format_with_names_use_header 0`), because the header does not contain the name for the first field;
- File is using only double quotes to enclose CSV strings; some strings are not enclosed in double quotes, and single quote must not be parsed as the string enclosing - that's why we also add the `--format_csv_allow_single_quote 0` parameter;
- Some strings from CSV cannot parse, because they contain `\M/` sequence at the beginning of the value; the only value starting with backslash in CSV can be `\N` that is parsed as SQL NULL. We add `--input_format_allow_errors_num 10` parameter and up to ten malformed records can be skipped;
- There are arrays for ingredients, directions and NER fields; these arrays are represented in unusual form: they are serialized into string as JSON and then placed in CSV - we parse them as String and then use [JSONExtract](../../sql-reference/functions/json-functions.md) function to transform it to Array.
- The dataset is in CSV format, but it requires some preprocessing on insertion; we use table function [input](../../sql-reference/table-functions/input.md) to perform preprocessing;
- The structure of CSV file is specified in the argument of the table function `input`;
- The field `num` (row number) is unneeded - we parse it from file and ignore;
- We use `FORMAT CSVWithNames` but the header in CSV will be ignored (by command line parameter `--input_format_with_names_use_header 0`), because the header does not contain the name for the first field;
- File is using only double quotes to enclose CSV strings; some strings are not enclosed in double quotes, and single quote must not be parsed as the string enclosing - that's why we also add the `--format_csv_allow_single_quote 0` parameter;
- Some strings from CSV cannot parse, because they contain `\M/` sequence at the beginning of the value; the only value starting with backslash in CSV can be `\N` that is parsed as SQL NULL. We add `--input_format_allow_errors_num 10` parameter and up to ten malformed records can be skipped;
- There are arrays for ingredients, directions and NER fields; these arrays are represented in unusual form: they are serialized into string as JSON and then placed in CSV - we parse them as String and then use [JSONExtract](../../sql-reference/functions/json-functions.md) function to transform it to Array.
## Validate the Inserted Data

View File

@ -161,11 +161,11 @@ sudo systemctl status clickhouse-keeper
#### Packages {#packages}
- `clickhouse-common-static` — Installs ClickHouse compiled binary files.
- `clickhouse-server` — Creates a symbolic link for `clickhouse-server` and installs the default server configuration.
- `clickhouse-client` — Creates a symbolic link for `clickhouse-client` and other client-related tools. and installs client configuration files.
- `clickhouse-common-static-dbg` — Installs ClickHouse compiled binary files with debug info.
- `clickhouse-keeper` - Used to install ClickHouse Keeper on dedicated ClickHouse Keeper nodes. If you are running ClickHouse Keeper on the same server as ClickHouse server, then you do not need to install this package. Installs ClickHouse Keeper and the default ClickHouse Keeper configuration files.
- `clickhouse-common-static` — Installs ClickHouse compiled binary files.
- `clickhouse-server` — Creates a symbolic link for `clickhouse-server` and installs the default server configuration.
- `clickhouse-client` — Creates a symbolic link for `clickhouse-client` and other client-related tools. and installs client configuration files.
- `clickhouse-common-static-dbg` — Installs ClickHouse compiled binary files with debug info.
- `clickhouse-keeper` - Used to install ClickHouse Keeper on dedicated ClickHouse Keeper nodes. If you are running ClickHouse Keeper on the same server as ClickHouse server, then you do not need to install this package. Installs ClickHouse Keeper and the default ClickHouse Keeper configuration files.
:::info
If you need to install specific version of ClickHouse you have to install all packages with the same version:
@ -429,8 +429,8 @@ We recommend using a minimum of 4GB of RAM to perform non-trivial queries. The C
The required volume of RAM generally depends on:
- The complexity of queries.
- The amount of data that is processed in queries.
- The complexity of queries.
- The amount of data that is processed in queries.
To calculate the required volume of RAM, you may estimate the size of temporary data for [GROUP BY](/docs/en/sql-reference/statements/select/group-by.md#select-group-by-clause), [DISTINCT](/docs/en/sql-reference/statements/select/distinct.md#select-distinct), [JOIN](/docs/en/sql-reference/statements/select/join.md#select-join) and other operations you use.
@ -442,11 +442,11 @@ The ClickHouse binary requires at least 2.5 GB of disk space for installation.
The volume of storage required for your data may be calculated separately based on
- an estimation of the data volume.
- an estimation of the data volume.
You can take a sample of the data and get the average size of a row from it. Then multiply the value by the number of rows you plan to store.
- The data compression coefficient.
- The data compression coefficient.
To estimate the data compression coefficient, load a sample of your data into ClickHouse, and compare the actual size of the data with the size of the table stored. For example, clickstream data is usually compressed by 6-10 times.

View File

@ -26,8 +26,8 @@ You can make queries to Playground using any HTTP client, for example [curl](htt
The queries are executed as a read-only user. It implies some limitations:
- DDL queries are not allowed
- INSERT queries are not allowed
- DDL queries are not allowed
- INSERT queries are not allowed
The service also have quotas on its usage.

View File

@ -148,8 +148,8 @@ Format a query as usual, then place the values that you want to pass from the ap
{<name>:<data type>}
```
- `name` — Placeholder identifier. In the console client it should be used in app parameters as `--param_<name> = value`.
- `data type` — [Data type](../sql-reference/data-types/index.md) of the app parameter value. For example, a data structure like `(integer, ('string', integer))` can have the `Tuple(UInt8, Tuple(String, UInt8))` data type (you can also use another [integer](../sql-reference/data-types/int-uint.md) types). It's also possible to pass table, database, column names as a parameter, in that case you would need to use `Identifier` as a data type.
- `name` — Placeholder identifier. In the console client it should be used in app parameters as `--param_<name> = value`.
- `data type` — [Data type](../sql-reference/data-types/index.md) of the app parameter value. For example, a data structure like `(integer, ('string', integer))` can have the `Tuple(UInt8, Tuple(String, UInt8))` data type (you can also use another [integer](../sql-reference/data-types/int-uint.md) types). It's also possible to pass table, database, column names as a parameter, in that case you would need to use `Identifier` as a data type.
#### Example {#example}
@ -162,37 +162,37 @@ $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="numbe
You can pass parameters to `clickhouse-client` (all parameters have a default value) using:
- From the Command Line
- From the Command Line
Command-line options override the default values and settings in configuration files.
- Configuration files.
- Configuration files.
Settings in the configuration files override the default values.
### Command Line Options {#command-line-options}
- `--host, -h` The server name, localhost by default. You can use either the name or the IPv4 or IPv6 address.
- `--port` The port to connect to. Default value: 9000. Note that the HTTP interface and the native interface use different ports.
- `--user, -u` The username. Default value: default.
- `--password` The password. Default value: empty string.
- `--ask-password` - Prompt the user to enter a password.
- `--query, -q` The query to process when using non-interactive mode. You must specify either `query` or `queries-file` option.
- `--queries-file` file path with queries to execute. You must specify either `query` or `queries-file` option.
- `--database, -d` Select the current default database. Default value: the current database from the server settings (default by default).
- `--multiline, -m` If specified, allow multiline queries (do not send the query on Enter).
- `--multiquery, -n` If specified, allow processing multiple queries separated by semicolons.
- `--format, -f` Use the specified default format to output the result.
- `--vertical, -E` If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables.
- `--time, -t` If specified, print the query execution time to stderr in non-interactive mode.
- `--stacktrace` If specified, also print the stack trace if an exception occurs.
- `--config-file` The name of the configuration file.
- `--secure` If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl).
- `--history_file` — Path to a file containing command history.
- `--param_<name>` — Value for a [query with parameters](#cli-queries-with-parameters).
- `--hardware-utilization` — Print hardware utilization information in progress bar.
- `--print-profile-events` Print `ProfileEvents` packets.
- `--profile-events-delay-ms` Delay between printing `ProfileEvents` packets (-1 - print only totals, 0 - print every single packet).
- `--host, -h` The server name, localhost by default. You can use either the name or the IPv4 or IPv6 address.
- `--port` The port to connect to. Default value: 9000. Note that the HTTP interface and the native interface use different ports.
- `--user, -u` The username. Default value: default.
- `--password` The password. Default value: empty string.
- `--ask-password` - Prompt the user to enter a password.
- `--query, -q` The query to process when using non-interactive mode. You must specify either `query` or `queries-file` option.
- `--queries-file` file path with queries to execute. You must specify either `query` or `queries-file` option.
- `--database, -d` Select the current default database. Default value: the current database from the server settings (default by default).
- `--multiline, -m` If specified, allow multiline queries (do not send the query on Enter).
- `--multiquery, -n` If specified, allow processing multiple queries separated by semicolons.
- `--format, -f` Use the specified default format to output the result.
- `--vertical, -E` If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables.
- `--time, -t` If specified, print the query execution time to stderr in non-interactive mode.
- `--stacktrace` If specified, also print the stack trace if an exception occurs.
- `--config-file` The name of the configuration file.
- `--secure` If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl).
- `--history_file` — Path to a file containing command history.
- `--param_<name>` — Value for a [query with parameters](#cli-queries-with-parameters).
- `--hardware-utilization` — Print hardware utilization information in progress bar.
- `--print-profile-events` Print `ProfileEvents` packets.
- `--profile-events-delay-ms` Delay between printing `ProfileEvents` packets (-1 - print only totals, 0 - print every single packet).
Since version 20.5, `clickhouse-client` has automatic syntax highlighting (always enabled).
@ -200,10 +200,10 @@ Since version 20.5, `clickhouse-client` has automatic syntax highlighting (alway
`clickhouse-client` uses the first existing file of the following:
- Defined in the `--config-file` parameter.
- `./clickhouse-client.xml`
- `~/.clickhouse-client/config.xml`
- `/etc/clickhouse-client/config.xml`
- Defined in the `--config-file` parameter.
- `./clickhouse-client.xml`
- `~/.clickhouse-client/config.xml`
- `/etc/clickhouse-client/config.xml`
Example of a config file:

View File

@ -38,6 +38,7 @@ The supported formats are:
| [JSONCompactStrings](#jsoncompactstrings) | ✗ | ✔ |
| [JSONCompactColumns](#jsoncompactcolumns) | ✔ | ✔ |
| [JSONEachRow](#jsoneachrow) | ✔ | ✔ |
| [PrettyJSONEachRow](#prettyjsoneachrow) | ✗ | ✔ |
| [JSONEachRowWithProgress](#jsoneachrowwithprogress) | ✗ | ✔ |
| [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ |
| [JSONStringsEachRowWithProgress](#jsonstringseachrowwithprogress) | ✗ | ✔ |
@ -255,11 +256,11 @@ 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` (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)
- `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.
@ -275,15 +276,15 @@ The `format_template_rows_between_delimiter` setting specifies the delimiter bet
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)
- `min` is the row with minimum values in `format_template_row` format (when extremes are set to 1)
- `max` is the row with maximum values in `format_template_row` format (when extremes are set to 1)
- `rows` is the total number of output rows
- `rows_before_limit` is the minimal number of rows there would have been without LIMIT. Output only if the query contains LIMIT. If the query contains GROUP BY, rows_before_limit_at_least is the exact number of rows there would have been without a LIMIT.
- `time` is the request execution time in seconds
- `rows_read` is the number of rows has been read
- `bytes_read` is the number of bytes (uncompressed) has been read
- `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)
- `min` is the row with minimum values in `format_template_row` format (when extremes are set to 1)
- `max` is the row with maximum values in `format_template_row` format (when extremes are set to 1)
- `rows` is the total number of output rows
- `rows_before_limit` is the minimal number of rows there would have been without LIMIT. Output only if the query contains LIMIT. If the query contains GROUP BY, rows_before_limit_at_least is the exact number of rows there would have been without a LIMIT.
- `time` is the request execution time in seconds
- `rows_read` is the number of rows has been read
- `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 the default value.
@ -619,8 +620,8 @@ ClickHouse supports [NULL](/docs/en/sql-reference/syntax.md), which is displayed
**See Also**
- [JSONEachRow](#jsoneachrow) format
- [output_format_json_array_of_rows](/docs/en/operations/settings/settings-formats.md/#output_format_json_array_of_rows) setting
- [JSONEachRow](#jsoneachrow) format
- [output_format_json_array_of_rows](/docs/en/operations/settings/settings-formats.md/#output_format_json_array_of_rows) setting
For JSON input format, if setting [input_format_json_validate_types_from_metadata](/docs/en/operations/settings/settings-formats.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.
@ -917,6 +918,40 @@ Example:
While importing data columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1.
## PrettyJSONEachRow {#prettyjsoneachrow}
Differs from JSONEachRow only in that JSON is pretty formatted with new line delimiters and 4 space indents. Suitable only for output.
Example
```json
{
"num": "42",
"str": "hello",
"arr": [
"0",
"1"
],
"tuple": {
"num": 42,
"str": "world"
}
}
{
"num": "43",
"str": "hello",
"arr": [
"0",
"1",
"2"
],
"tuple": {
"num": 43,
"str": "world"
}
}
```
## JSONStringsEachRow {#jsonstringseachrow}
Differs from JSONEachRow only in that data fields are output in strings, not in typed JSON values.
@ -1096,8 +1131,8 @@ INSERT INTO UserActivity FORMAT JSONEachRow {"PageViews":5, "UserID":"4324182021
ClickHouse allows:
- Any order of key-value pairs in the object.
- Omitting some values.
- Any order of key-value pairs in the object.
- Omitting some values.
ClickHouse ignores spaces between elements and commas after the objects. You can pass all the objects in one line. You do not have to separate them with line breaks.
@ -1117,8 +1152,8 @@ CREATE TABLE IF NOT EXISTS example_table
) ENGINE = Memory;
```
- If `input_format_defaults_for_omitted_fields = 0`, then the default value for `x` and `a` equals `0` (as the default value for the `UInt32` data type).
- If `input_format_defaults_for_omitted_fields = 1`, then the default value for `x` equals `0`, but the default value of `a` equals `x * 2`.
- If `input_format_defaults_for_omitted_fields = 0`, then the default value for `x` and `a` equals `0` (as the default value for the `UInt32` data type).
- If `input_format_defaults_for_omitted_fields = 1`, then the default value for `x` equals `0`, but the default value of `a` equals `x * 2`.
:::note
When inserting data with `input_format_defaults_for_omitted_fields = 1`, ClickHouse consumes more computational resources, compared to insertion with `input_format_defaults_for_omitted_fields = 0`.
@ -1447,8 +1482,8 @@ For [NULL](/docs/en/sql-reference/syntax.md/#null-literal) support, an additiona
Similar to [RowBinary](#rowbinary), but with added header:
- [LEB128](https://en.wikipedia.org/wiki/LEB128)-encoded number of columns (N)
- N `String`s specifying column names
- [LEB128](https://en.wikipedia.org/wiki/LEB128)-encoded number of columns (N)
- N `String`s specifying column names
:::note
If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1,
@ -1460,9 +1495,9 @@ Otherwise, the first row will be skipped.
Similar to [RowBinary](#rowbinary), but with added header:
- [LEB128](https://en.wikipedia.org/wiki/LEB128)-encoded number of columns (N)
- N `String`s specifying column names
- N `String`s specifying column types
- [LEB128](https://en.wikipedia.org/wiki/LEB128)-encoded number of columns (N)
- N `String`s specifying column names
- N `String`s specifying column types
:::note
If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1,
@ -1872,8 +1907,8 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Avro" > file.avro
Column names must:
- start with `[A-Za-z_]`
- subsequently contain only `[A-Za-z0-9_]`
- 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](/docs/en/operations/settings/settings-formats.md/#output_format_avro_codec) and [output_format_avro_sync_interval](/docs/en/operations/settings/settings-formats.md/#output_format_avro_sync_interval) respectively.
@ -2171,17 +2206,17 @@ 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](/docs/en/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](/docs/en/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))
- Escaped (similarly to [TSV](#tabseparated))
- Quoted (similarly to [Values](#data-format-values))
- Raw (extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](#tabseparatedraw))
- CSV (similarly to [CSV](#csv))
- JSON (similarly to [JSONEachRow](#jsoneachrow))
- Escaped (similarly to [TSV](#tabseparated))
- 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](/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`.
- `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**

View File

@ -10,14 +10,14 @@ sidebar_label: gRPC Interface
ClickHouse supports [gRPC](https://grpc.io/) interface. It is an open source remote procedure call system that uses HTTP/2 and [Protocol Buffers](https://en.wikipedia.org/wiki/Protocol_Buffers). The implementation of gRPC in ClickHouse supports:
- SSL;
- authentication;
- sessions;
- compression;
- parallel queries through the same channel;
- cancellation of queries;
- getting progress and logs;
- external tables.
- SSL;
- authentication;
- sessions;
- compression;
- parallel queries through the same channel;
- cancellation of queries;
- getting progress and logs;
- external tables.
The specification of the interface is described in [clickhouse_grpc.proto](https://github.com/ClickHouse/ClickHouse/blob/master/src/Server/grpc_protos/clickhouse_grpc.proto).
@ -64,15 +64,15 @@ Or you can use a built-in Python client. It is placed in [utils/grpc-client/clic
The client supports the following arguments:
- `--help` Shows a help message and exits.
- `--host HOST, -h HOST` A server name. Default value: `localhost`. You can use IPv4 or IPv6 addresses also.
- `--port PORT` A port to connect to. This port should be enabled in the ClickHouse server configuration (see `grpc_port`). Default value: `9100`.
- `--user USER_NAME, -u USER_NAME` A user name. Default value: `default`.
- `--password PASSWORD` A password. Default value: empty string.
- `--query QUERY, -q QUERY` A query to process when using non-interactive mode.
- `--database DATABASE, -d DATABASE` A default database. If not specified, the current database set in the server settings is used (`default` by default).
- `--format OUTPUT_FORMAT, -f OUTPUT_FORMAT` A result output [format](formats.md). Default value for interactive mode: `PrettyCompact`.
- `--debug` Enables showing debug information.
- `--help` Shows a help message and exits.
- `--host HOST, -h HOST` A server name. Default value: `localhost`. You can use IPv4 or IPv6 addresses also.
- `--port PORT` A port to connect to. This port should be enabled in the ClickHouse server configuration (see `grpc_port`). Default value: `9100`.
- `--user USER_NAME, -u USER_NAME` A user name. Default value: `default`.
- `--password PASSWORD` A password. Default value: empty string.
- `--query QUERY, -q QUERY` A query to process when using non-interactive mode.
- `--database DATABASE, -d DATABASE` A default database. If not specified, the current database set in the server settings is used (`default` by default).
- `--format OUTPUT_FORMAT, -f OUTPUT_FORMAT` A result output [format](formats.md). Default value for interactive mode: `PrettyCompact`.
- `--debug` Enables showing debug information.
To run the client in an interactive mode call it without `--query` argument.

View File

@ -293,11 +293,11 @@ X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_ro
Possible header fields:
- `read_rows` — Number of rows read.
- `read_bytes` — Volume of data read in bytes.
- `total_rows_to_read` — Total number of rows to be read.
- `written_rows` — Number of rows written.
- `written_bytes` — Volume of data written in bytes.
- `read_rows` — Number of rows read.
- `read_bytes` — Volume of data read in bytes.
- `total_rows_to_read` — Total number of rows to be read.
- `written_rows` — Number of rows written.
- `written_bytes` — Volume of data written in bytes.
Running requests do not stop automatically if the HTTP connection is lost. Parsing and data formatting are performed on the server-side, and using the network might be ineffective.
The optional query_id parameter can be passed as the query ID (any string). For more information, see the section “Settings, replace_running_query”.
@ -374,7 +374,7 @@ ClickHouse also supports Predefined HTTP Interface which can help you more easil
Example:
- First of all, add this section to server configuration file:
- First of all, add this section to server configuration file:
<!-- -->
@ -393,7 +393,7 @@ Example:
</http_handlers>
```
- You can now request the URL directly for data in the Prometheus format:
- You can now request the URL directly for data in the Prometheus format:
<!-- -->
@ -448,22 +448,22 @@ As you can see from the example if `http_handlers` is configured in the config.x
Now `rule` can configure `method`, `headers`, `url`, `handler`:
- `method` is responsible for matching the method part of the HTTP request. `method` fully conforms to the definition of [method](https://developer.mozilla.org/en-US/docs/Web/HTTP/Methods) in the HTTP protocol. It is an optional configuration. If it is not defined in the configuration file, it does not match the method portion of the HTTP request.
- `url` is responsible for matching the URL part of the HTTP request. It is compatible with [RE2](https://github.com/google/re2)s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the URL portion of the HTTP request.
- `url` is responsible for matching the URL part of the HTTP request. It is compatible with [RE2](https://github.com/google/re2)s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the URL portion of the HTTP request.
- `headers` are responsible for matching the header part of the HTTP request. It is compatible with RE2s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the header portion of the HTTP request.
- `headers` are responsible for matching the header part of the HTTP request. It is compatible with RE2s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the header portion of the HTTP request.
- `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `response_content`, `query`, `query_param_name`.
- `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `response_content`, `query`, `query_param_name`.
`type` currently supports three types: [predefined_query_handler](#predefined_query_handler), [dynamic_query_handler](#dynamic_query_handler), [static](#static).
- `query` — use with `predefined_query_handler` type, executes query when the handler is called.
- `query` — use with `predefined_query_handler` type, executes query when the handler is called.
- `query_param_name` — use with `dynamic_query_handler` type, extracts and executes the value corresponding to the `query_param_name` value in HTTP request parameters.
- `query_param_name` — use with `dynamic_query_handler` type, extracts and executes the value corresponding to the `query_param_name` value in HTTP request parameters.
- `status` — use with `static` type, response status code.
- `status` — use with `static` type, response status code.
- `content_type` — use with any type, response [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type).
- `content_type` — use with any type, response [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type).
- `response_content` — use with `static` type, response content sent to client, when using the prefix file:// or config://, find the content from the file or configuration sends to client.
- `response_content` — use with `static` type, response content sent to client, when using the prefix file:// or config://, find the content from the file or configuration sends to client.
Next are the configuration methods for different `type`.

View File

@ -8,6 +8,6 @@ sidebar_label: JDBC Driver
Use the [official JDBC driver](https://github.com/ClickHouse/clickhouse-jdbc) (and Java client) to access ClickHouse from your Java applications.
- Third-party drivers:
- [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC)
- [clickhouse4j](https://github.com/blynkkk/clickhouse4j)
- Third-party drivers:
- [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC)
- [clickhouse4j](https://github.com/blynkkk/clickhouse4j)

View File

@ -57,9 +57,9 @@ If user password is specified using [SHA256](../operations/settings/settings-use
Restrictions:
- prepared queries are not supported
- prepared queries are not supported
- some data types are sent as strings
- some data types are sent as strings
To cancel a long query use `KILL QUERY connection_id` statement (it is replaced with `KILL QUERY WHERE query_id = connection_id` while proceeding). For example:

View File

@ -10,19 +10,19 @@ description: ClickHouse provides three network interfaces
ClickHouse provides three network interfaces (they can be optionally wrapped in TLS for additional security):
- [HTTP](http.md), which is documented and easy to use directly.
- [Native TCP](../interfaces/tcp.md), which has less overhead.
- [gRPC](grpc.md).
- [HTTP](http.md), which is documented and easy to use directly.
- [Native TCP](../interfaces/tcp.md), which has less overhead.
- [gRPC](grpc.md).
In most cases it is recommended to use an appropriate tool or library instead of interacting with those directly. The following are officially supported by ClickHouse:
- [Command-line client](../interfaces/cli.md)
- [JDBC driver](../interfaces/jdbc.md)
- [ODBC driver](../interfaces/odbc.md)
- [C++ client library](../interfaces/cpp.md)
- [Command-line client](../interfaces/cli.md)
- [JDBC driver](../interfaces/jdbc.md)
- [ODBC driver](../interfaces/odbc.md)
- [C++ client library](../interfaces/cpp.md)
There are also a wide range of third-party libraries for working with ClickHouse:
- [Client libraries](../interfaces/third-party/client-libraries.md)
- [Integrations](../interfaces/third-party/integrations.md)
- [Visual interfaces](../interfaces/third-party/gui.md)
- [Client libraries](../interfaces/third-party/client-libraries.md)
- [Integrations](../interfaces/third-party/integrations.md)
- [Visual interfaces](../interfaces/third-party/gui.md)

View File

@ -10,68 +10,68 @@ sidebar_label: Client Libraries
ClickHouse Inc does **not** maintain the libraries listed below and hasnt done any extensive testing to ensure their quality.
:::
- Python
- [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)
- [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver)
- [clickhouse-client](https://github.com/yurial/clickhouse-client)
- [aiochclient](https://github.com/maximdanilchenko/aiochclient)
- [asynch](https://github.com/long2ice/asynch)
- PHP
- [smi2/phpclickhouse](https://packagist.org/packages/smi2/phpClickHouse)
- [8bitov/clickhouse-php-client](https://packagist.org/packages/8bitov/clickhouse-php-client)
- [bozerkins/clickhouse-client](https://packagist.org/packages/bozerkins/clickhouse-client)
- [simpod/clickhouse-client](https://packagist.org/packages/simpod/clickhouse-client)
- [seva-code/php-click-house-client](https://packagist.org/packages/seva-code/php-click-house-client)
- [SeasClick C++ client](https://github.com/SeasX/SeasClick)
- [one-ck](https://github.com/lizhichao/one-ck)
- [glushkovds/phpclickhouse-laravel](https://packagist.org/packages/glushkovds/phpclickhouse-laravel)
- [kolya7k ClickHouse PHP extension](https://github.com//kolya7k/clickhouse-php)
- [hyvor/clickhouse-php](https://github.com/hyvor/clickhouse-php)
- Go
- [clickhouse](https://github.com/kshvakov/clickhouse/)
- [go-clickhouse](https://github.com/roistat/go-clickhouse)
- [chconn](https://github.com/vahid-sohrabloo/chconn)
- [mailrugo-clickhouse](https://github.com/mailru/go-clickhouse)
- [golang-clickhouse](https://github.com/leprosus/golang-clickhouse)
- [uptrace/go-clickhouse](https://clickhouse.uptrace.dev/)
- Swift
- [ClickHouseNIO](https://github.com/patrick-zippenfenig/ClickHouseNIO)
- [ClickHouseVapor ORM](https://github.com/patrick-zippenfenig/ClickHouseVapor)
- NodeJs
- [clickhouse (NodeJs)](https://github.com/TimonKK/clickhouse)
- [node-clickhouse](https://github.com/apla/node-clickhouse)
- [nestjs-clickhouse](https://github.com/depyronick/nestjs-clickhouse)
- [clickhouse-client](https://github.com/depyronick/clickhouse-client)
- [node-clickhouse-orm](https://github.com/zimv/node-clickhouse-orm)
- Perl
- [perl-DBD-ClickHouse](https://github.com/elcamlost/perl-DBD-ClickHouse)
- [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse)
- [AnyEvent-ClickHouse](https://metacpan.org/release/AnyEvent-ClickHouse)
- Ruby
- [ClickHouse (Ruby)](https://github.com/shlima/click_house)
- [clickhouse-activerecord](https://github.com/PNixx/clickhouse-activerecord)
- Rust
- [clickhouse.rs](https://github.com/loyd/clickhouse.rs)
- [clickhouse-rs](https://github.com/suharev7/clickhouse-rs)
- [Klickhouse](https://github.com/Protryon/klickhouse)
- R
- [RClickHouse](https://github.com/IMSMWU/RClickHouse)
- Java
- [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java)
- [clickhouse-client](https://github.com/Ecwid/clickhouse-client)
- Scala
- [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client)
- Kotlin
- [AORM](https://github.com/TanVD/AORM)
- C#
- [Octonica.ClickHouseClient](https://github.com/Octonica/ClickHouseClient)
- [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net)
- [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client)
- [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net)
- Elixir
- [clickhousex](https://github.com/appodeal/clickhousex/)
- [pillar](https://github.com/sofakingworld/pillar)
- Nim
- [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse)
- Haskell
- [hdbc-clickhouse](https://github.com/zaneli/hdbc-clickhouse)
- Python
- [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)
- [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver)
- [clickhouse-client](https://github.com/yurial/clickhouse-client)
- [aiochclient](https://github.com/maximdanilchenko/aiochclient)
- [asynch](https://github.com/long2ice/asynch)
- PHP
- [smi2/phpclickhouse](https://packagist.org/packages/smi2/phpClickHouse)
- [8bitov/clickhouse-php-client](https://packagist.org/packages/8bitov/clickhouse-php-client)
- [bozerkins/clickhouse-client](https://packagist.org/packages/bozerkins/clickhouse-client)
- [simpod/clickhouse-client](https://packagist.org/packages/simpod/clickhouse-client)
- [seva-code/php-click-house-client](https://packagist.org/packages/seva-code/php-click-house-client)
- [SeasClick C++ client](https://github.com/SeasX/SeasClick)
- [one-ck](https://github.com/lizhichao/one-ck)
- [glushkovds/phpclickhouse-laravel](https://packagist.org/packages/glushkovds/phpclickhouse-laravel)
- [kolya7k ClickHouse PHP extension](https://github.com//kolya7k/clickhouse-php)
- [hyvor/clickhouse-php](https://github.com/hyvor/clickhouse-php)
- Go
- [clickhouse](https://github.com/kshvakov/clickhouse/)
- [go-clickhouse](https://github.com/roistat/go-clickhouse)
- [chconn](https://github.com/vahid-sohrabloo/chconn)
- [mailrugo-clickhouse](https://github.com/mailru/go-clickhouse)
- [golang-clickhouse](https://github.com/leprosus/golang-clickhouse)
- [uptrace/go-clickhouse](https://clickhouse.uptrace.dev/)
- Swift
- [ClickHouseNIO](https://github.com/patrick-zippenfenig/ClickHouseNIO)
- [ClickHouseVapor ORM](https://github.com/patrick-zippenfenig/ClickHouseVapor)
- NodeJs
- [clickhouse (NodeJs)](https://github.com/TimonKK/clickhouse)
- [node-clickhouse](https://github.com/apla/node-clickhouse)
- [nestjs-clickhouse](https://github.com/depyronick/nestjs-clickhouse)
- [clickhouse-client](https://github.com/depyronick/clickhouse-client)
- [node-clickhouse-orm](https://github.com/zimv/node-clickhouse-orm)
- Perl
- [perl-DBD-ClickHouse](https://github.com/elcamlost/perl-DBD-ClickHouse)
- [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse)
- [AnyEvent-ClickHouse](https://metacpan.org/release/AnyEvent-ClickHouse)
- Ruby
- [ClickHouse (Ruby)](https://github.com/shlima/click_house)
- [clickhouse-activerecord](https://github.com/PNixx/clickhouse-activerecord)
- Rust
- [clickhouse.rs](https://github.com/loyd/clickhouse.rs)
- [clickhouse-rs](https://github.com/suharev7/clickhouse-rs)
- [Klickhouse](https://github.com/Protryon/klickhouse)
- R
- [RClickHouse](https://github.com/IMSMWU/RClickHouse)
- Java
- [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java)
- [clickhouse-client](https://github.com/Ecwid/clickhouse-client)
- Scala
- [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client)
- Kotlin
- [AORM](https://github.com/TanVD/AORM)
- C#
- [Octonica.ClickHouseClient](https://github.com/Octonica/ClickHouseClient)
- [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net)
- [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client)
- [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net)
- Elixir
- [clickhousex](https://github.com/appodeal/clickhousex/)
- [pillar](https://github.com/sofakingworld/pillar)
- Nim
- [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse)
- Haskell
- [hdbc-clickhouse](https://github.com/zaneli/hdbc-clickhouse)

View File

@ -14,11 +14,11 @@ Web interface for ClickHouse in the [Tabix](https://github.com/tabixio/tabix) pr
Features:
- Works with ClickHouse directly from the browser, without the need to install additional software.
- Query editor with syntax highlighting.
- Auto-completion of commands.
- Tools for graphical analysis of query execution.
- Colour scheme options.
- Works with ClickHouse directly from the browser, without the need to install additional software.
- Query editor with syntax highlighting.
- Auto-completion of commands.
- Tools for graphical analysis of query execution.
- Colour scheme options.
[Tabix documentation](https://tabix.io/doc/).
@ -28,21 +28,21 @@ Features:
Features:
- Query builder with syntax highlighting. View the response in a table or JSON view.
- Export query results as CSV or JSON.
- List of processes with descriptions. Write mode. Ability to stop (`KILL`) a process.
- Database graph. Shows all tables and their columns with additional information.
- A quick view of the column size.
- Server configuration.
- Query builder with syntax highlighting. View the response in a table or JSON view.
- Export query results as CSV or JSON.
- List of processes with descriptions. Write mode. Ability to stop (`KILL`) a process.
- Database graph. Shows all tables and their columns with additional information.
- A quick view of the column size.
- Server configuration.
The following features are planned for development:
- Database management.
- User management.
- Real-time data analysis.
- Cluster monitoring.
- Cluster management.
- Monitoring replicated and Kafka tables.
- Database management.
- User management.
- Real-time data analysis.
- Cluster monitoring.
- Cluster management.
- Monitoring replicated and Kafka tables.
### LightHouse {#lighthouse}
@ -50,9 +50,9 @@ The following features are planned for development:
Features:
- Table list with filtering and metadata.
- Table preview with filtering and sorting.
- Read-only queries execution.
- Table list with filtering and metadata.
- Table preview with filtering and sorting.
- Read-only queries execution.
### Redash {#redash}
@ -62,9 +62,9 @@ Supports for multiple data sources including ClickHouse, Redash can join results
Features:
- Powerful editor of queries.
- Database explorer.
- Visualization tools, that allow you to represent data in different forms.
- Powerful editor of queries.
- Database explorer.
- Visualization tools, that allow you to represent data in different forms.
### Grafana {#grafana}
@ -92,10 +92,10 @@ Features:
Features:
- Query development with syntax highlight and autocompletion.
- Table list with filters and metadata search.
- Table data preview.
- Full-text search.
- Query development with syntax highlight and autocompletion.
- Table list with filters and metadata search.
- Table data preview.
- Full-text search.
By default, DBeaver does not connect using a session (the CLI for example does). If you require session support (for example to set settings for your session), edit the driver connection properties and set `session_id` to a random string (it uses the http connection under the hood). Then you can use any setting from the query window.
@ -105,10 +105,10 @@ By default, DBeaver does not connect using a session (the CLI for example does).
Features:
- Autocompletion.
- Syntax highlighting for the queries and data output.
- Pager support for the data output.
- Custom PostgreSQL-like commands.
- Autocompletion.
- Syntax highlighting for the queries and data output.
- Pager support for the data output.
- Custom PostgreSQL-like commands.
### clickhouse-flamegraph {#clickhouse-flamegraph}
@ -132,15 +132,15 @@ Features:
Features:
- Support query history (pagination, clear all, etc.)
- Support selected sql clauses query
- Support terminating query
- Support table management (metadata, delete, preview)
- Support database management (delete, create)
- Support custom query
- Support multiple data sources management(connection test, monitoring)
- Support monitor (processor, connection, query)
- Support migrate data
- Support query history (pagination, clear all, etc.)
- Support selected sql clauses query
- Support terminating query
- Support table management (metadata, delete, preview)
- Support database management (delete, create)
- Support custom query
- Support multiple data sources management(connection test, monitoring)
- Support monitor (processor, connection, query)
- Support migrate data
### Bytebase {#bytebase}
@ -148,13 +148,13 @@ Features:
Features:
- Schema review between developers and DBAs.
- Database-as-Code, version control the schema in VCS such GitLab and trigger the deployment upon code commit.
- Streamlined deployment with per-environment policy.
- Full migration history.
- Schema drift detection.
- Backup and restore.
- RBAC.
- Schema review between developers and DBAs.
- Database-as-Code, version control the schema in VCS such GitLab and trigger the deployment upon code commit.
- Streamlined deployment with per-environment policy.
- Full migration history.
- Schema drift detection.
- Backup and restore.
- RBAC.
### Zeppelin-Interpreter-for-ClickHouse {#zeppelin-interpreter-for-clickhouse}
@ -166,9 +166,9 @@ Features:
Features:
- An online SQL editor which can run your SQL code without any installing.
- You can observe all processes and mutations. For those unfinished processes, you can kill them in ui.
- The Metrics contains Cluster Analysis,Data Analysis,Query Analysis.
- An online SQL editor which can run your SQL code without any installing.
- You can observe all processes and mutations. For those unfinished processes, you can kill them in ui.
- The Metrics contains Cluster Analysis,Data Analysis,Query Analysis.
### ClickVisual {#clickvisual}
@ -218,12 +218,12 @@ Features:
Features:
- Very fast code completion.
- ClickHouse syntax highlighting.
- Support for features specific to ClickHouse, for example, nested columns, table engines.
- Data Editor.
- Refactorings.
- Search and Navigation.
- Very fast code completion.
- ClickHouse syntax highlighting.
- Support for features specific to ClickHouse, for example, nested columns, table engines.
- Data Editor.
- Refactorings.
- Search and Navigation.
### Yandex DataLens {#yandex-datalens}
@ -231,15 +231,15 @@ Features:
Features:
- Wide range of available visualizations, from simple bar charts to complex dashboards.
- Dashboards could be made publicly available.
- Support for multiple data sources including ClickHouse.
- Storage for materialized data based on ClickHouse.
- Wide range of available visualizations, from simple bar charts to complex dashboards.
- Dashboards could be made publicly available.
- Support for multiple data sources including ClickHouse.
- Storage for materialized data based on ClickHouse.
DataLens is [available for free](https://cloud.yandex.com/docs/datalens/pricing) for low-load projects, even for commercial use.
- [DataLens documentation](https://cloud.yandex.com/docs/datalens/).
- [Tutorial](https://cloud.yandex.com/docs/solutions/datalens/data-from-ch-visualization) on visualizing data from a ClickHouse database.
- [DataLens documentation](https://cloud.yandex.com/docs/datalens/).
- [Tutorial](https://cloud.yandex.com/docs/solutions/datalens/data-from-ch-visualization) on visualizing data from a ClickHouse database.
### Holistics Software {#holistics-software}
@ -247,11 +247,11 @@ DataLens is [available for free](https://cloud.yandex.com/docs/datalens/pricing)
Features:
- Automated email, Slack and Google Sheet schedules of reports.
- SQL editor with visualizations, version control, auto-completion, reusable query components and dynamic filters.
- Embedded analytics of reports and dashboards via iframe.
- Data preparation and ETL capabilities.
- SQL data modelling support for relational mapping of data.
- Automated email, Slack and Google Sheet schedules of reports.
- SQL editor with visualizations, version control, auto-completion, reusable query components and dynamic filters.
- Embedded analytics of reports and dashboards via iframe.
- Data preparation and ETL capabilities.
- SQL data modelling support for relational mapping of data.
### Looker {#looker}
@ -260,9 +260,9 @@ to integrate data with other applications.
Features:
- Easy and agile development using LookML, a language which supports curated
- Easy and agile development using LookML, a language which supports curated
[Data Modeling](https://looker.com/platform/data-modeling) to support report writers and end-users.
- Powerful workflow integration via Lookers [Data Actions](https://looker.com/platform/actions).
- Powerful workflow integration via Lookers [Data Actions](https://looker.com/platform/actions).
[How to configure ClickHouse in Looker.](https://docs.looker.com/setup-and-management/database-config/clickhouse)
@ -272,12 +272,12 @@ Features:
Features:
- Business users-friendly reports builder.
- Powerful report parameters for SQL filtering and report-specific query customizations.
- Can connect to ClickHouse both with a native TCP/IP endpoint and a HTTP(S) interface (2 different drivers).
- It is possible to use all power of ClickHouse SQL dialect in dimensions/measures definitions.
- [Web API](https://www.seektable.com/help/web-api-integration) for automated reports generation.
- Supports reports development flow with account data [backup/restore](https://www.seektable.com/help/self-hosted-backup-restore); data models (cubes) / reports configuration is a human-readable XML and can be stored under version control system.
- Business users-friendly reports builder.
- Powerful report parameters for SQL filtering and report-specific query customizations.
- Can connect to ClickHouse both with a native TCP/IP endpoint and a HTTP(S) interface (2 different drivers).
- It is possible to use all power of ClickHouse SQL dialect in dimensions/measures definitions.
- [Web API](https://www.seektable.com/help/web-api-integration) for automated reports generation.
- Supports reports development flow with account data [backup/restore](https://www.seektable.com/help/self-hosted-backup-restore); data models (cubes) / reports configuration is a human-readable XML and can be stored under version control system.
SeekTable is [free](https://www.seektable.com/help/cloud-pricing) for personal/individual usage.

View File

@ -8,10 +8,10 @@ sidebar_position: 24
This is a collection of links to third-party tools that provide some sort of interface to ClickHouse. It can be either visual interface, command-line interface or an API:
- [Client libraries](../../interfaces/third-party/client-libraries.md)
- [Integrations](../../interfaces/third-party/integrations.md)
- [GUI](../../interfaces/third-party/gui.md)
- [Proxies](../../interfaces/third-party/proxy.md)
- [Client libraries](../../interfaces/third-party/client-libraries.md)
- [Integrations](../../interfaces/third-party/integrations.md)
- [GUI](../../interfaces/third-party/gui.md)
- [Proxies](../../interfaces/third-party/proxy.md)
:::note
Generic tools that support common API like [ODBC](../../interfaces/odbc.md) or [JDBC](../../interfaces/jdbc.md) usually can work with ClickHouse as well, but are not listed here because there are way too many of them.

View File

@ -12,104 +12,104 @@ ClickHouse, Inc. does **not** maintain the tools and libraries listed below and
## Infrastructure Products {#infrastructure-products}
- Relational database management systems
- [MySQL](https://www.mysql.com)
- [mysql2ch](https://github.com/long2ice/mysql2ch)
- [ProxySQL](https://github.com/sysown/proxysql/wiki/ClickHouse-Support)
- [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader)
- [horgh-replicator](https://github.com/larsnovikov/horgh-replicator)
- [PostgreSQL](https://www.postgresql.org)
- [clickhousedb_fdw](https://github.com/Percona-Lab/clickhousedb_fdw)
- [infi.clickhouse_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm))
- [pg2ch](https://github.com/mkabilov/pg2ch)
- [clickhouse_fdw](https://github.com/adjust/clickhouse_fdw)
- [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server)
- [ClickHouseMigrator](https://github.com/zlzforever/ClickHouseMigrator)
- Message queues
- [Kafka](https://kafka.apache.org)
- [clickhouse_sinker](https://github.com/housepower/clickhouse_sinker) (uses [Go client](https://github.com/ClickHouse/clickhouse-go/))
- [stream-loader-clickhouse](https://github.com/adform/stream-loader)
- Batch processing
- [Spark](https://spark.apache.org)
- [spark-clickhouse-connector](https://github.com/housepower/spark-clickhouse-connector)
- Stream processing
- [Flink](https://flink.apache.org)
- [flink-clickhouse-sink](https://github.com/ivi-ru/flink-clickhouse-sink)
- Object storages
- [S3](https://en.wikipedia.org/wiki/Amazon_S3)
- [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup)
- Container orchestration
- [Kubernetes](https://kubernetes.io)
- [clickhouse-operator](https://github.com/Altinity/clickhouse-operator)
- Configuration management
- [puppet](https://puppet.com)
- [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse)
- [mfedotov/clickhouse](https://forge.puppet.com/mfedotov/clickhouse)
- Monitoring
- [Graphite](https://graphiteapp.org)
- [graphouse](https://github.com/ClickHouse/graphouse)
- [carbon-clickhouse](https://github.com/lomik/carbon-clickhouse)
- [graphite-clickhouse](https://github.com/lomik/graphite-clickhouse)
- [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer) - optimizes staled partitions in [\*GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md#graphitemergetree) if rules from [rollup configuration](../../engines/table-engines/mergetree-family/graphitemergetree.md#rollup-configuration) could be applied
- [Grafana](https://grafana.com/)
- [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana)
- [Prometheus](https://prometheus.io/)
- [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter)
- [PromHouse](https://github.com/Percona-Lab/PromHouse)
- [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (uses [Go client](https://github.com/kshvakov/clickhouse/))
- [Nagios](https://www.nagios.org/)
- [check_clickhouse](https://github.com/exogroup/check_clickhouse/)
- [check_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py)
- [Zabbix](https://www.zabbix.com)
- [clickhouse-zabbix-template](https://github.com/Altinity/clickhouse-zabbix-template)
- [Sematext](https://sematext.com/)
- [clickhouse integration](https://github.com/sematext/sematext-agent-integrations/tree/master/clickhouse)
- Logging
- [rsyslog](https://www.rsyslog.com/)
- [omclickhouse](https://www.rsyslog.com/doc/master/configuration/modules/omclickhouse.html)
- [fluentd](https://www.fluentd.org)
- [loghouse](https://github.com/flant/loghouse) (for [Kubernetes](https://kubernetes.io))
- [logagent](https://www.sematext.com/logagent)
- [logagent output-plugin-clickhouse](https://sematext.com/docs/logagent/output-plugin-clickhouse/)
- Geo
- [MaxMind](https://dev.maxmind.com/geoip/)
- [clickhouse-maxmind-geoip](https://github.com/AlexeyKupershtokh/clickhouse-maxmind-geoip)
- AutoML
- [MindsDB](https://mindsdb.com/)
- [MindsDB](https://github.com/mindsdb/mindsdb) - Predictive AI layer for ClickHouse database.
- Relational database management systems
- [MySQL](https://www.mysql.com)
- [mysql2ch](https://github.com/long2ice/mysql2ch)
- [ProxySQL](https://github.com/sysown/proxysql/wiki/ClickHouse-Support)
- [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader)
- [horgh-replicator](https://github.com/larsnovikov/horgh-replicator)
- [PostgreSQL](https://www.postgresql.org)
- [clickhousedb_fdw](https://github.com/Percona-Lab/clickhousedb_fdw)
- [infi.clickhouse_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm))
- [pg2ch](https://github.com/mkabilov/pg2ch)
- [clickhouse_fdw](https://github.com/adjust/clickhouse_fdw)
- [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server)
- [ClickHouseMigrator](https://github.com/zlzforever/ClickHouseMigrator)
- Message queues
- [Kafka](https://kafka.apache.org)
- [clickhouse_sinker](https://github.com/housepower/clickhouse_sinker) (uses [Go client](https://github.com/ClickHouse/clickhouse-go/))
- [stream-loader-clickhouse](https://github.com/adform/stream-loader)
- Batch processing
- [Spark](https://spark.apache.org)
- [spark-clickhouse-connector](https://github.com/housepower/spark-clickhouse-connector)
- Stream processing
- [Flink](https://flink.apache.org)
- [flink-clickhouse-sink](https://github.com/ivi-ru/flink-clickhouse-sink)
- Object storages
- [S3](https://en.wikipedia.org/wiki/Amazon_S3)
- [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup)
- Container orchestration
- [Kubernetes](https://kubernetes.io)
- [clickhouse-operator](https://github.com/Altinity/clickhouse-operator)
- Configuration management
- [puppet](https://puppet.com)
- [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse)
- [mfedotov/clickhouse](https://forge.puppet.com/mfedotov/clickhouse)
- Monitoring
- [Graphite](https://graphiteapp.org)
- [graphouse](https://github.com/ClickHouse/graphouse)
- [carbon-clickhouse](https://github.com/lomik/carbon-clickhouse)
- [graphite-clickhouse](https://github.com/lomik/graphite-clickhouse)
- [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer) - optimizes staled partitions in [\*GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md#graphitemergetree) if rules from [rollup configuration](../../engines/table-engines/mergetree-family/graphitemergetree.md#rollup-configuration) could be applied
- [Grafana](https://grafana.com/)
- [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana)
- [Prometheus](https://prometheus.io/)
- [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter)
- [PromHouse](https://github.com/Percona-Lab/PromHouse)
- [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (uses [Go client](https://github.com/kshvakov/clickhouse/))
- [Nagios](https://www.nagios.org/)
- [check_clickhouse](https://github.com/exogroup/check_clickhouse/)
- [check_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py)
- [Zabbix](https://www.zabbix.com)
- [clickhouse-zabbix-template](https://github.com/Altinity/clickhouse-zabbix-template)
- [Sematext](https://sematext.com/)
- [clickhouse integration](https://github.com/sematext/sematext-agent-integrations/tree/master/clickhouse)
- Logging
- [rsyslog](https://www.rsyslog.com/)
- [omclickhouse](https://www.rsyslog.com/doc/master/configuration/modules/omclickhouse.html)
- [fluentd](https://www.fluentd.org)
- [loghouse](https://github.com/flant/loghouse) (for [Kubernetes](https://kubernetes.io))
- [logagent](https://www.sematext.com/logagent)
- [logagent output-plugin-clickhouse](https://sematext.com/docs/logagent/output-plugin-clickhouse/)
- Geo
- [MaxMind](https://dev.maxmind.com/geoip/)
- [clickhouse-maxmind-geoip](https://github.com/AlexeyKupershtokh/clickhouse-maxmind-geoip)
- AutoML
- [MindsDB](https://mindsdb.com/)
- [MindsDB](https://github.com/mindsdb/mindsdb) - Predictive AI layer for ClickHouse database.
## Programming Language Ecosystems {#programming-language-ecosystems}
- Python
- [SQLAlchemy](https://www.sqlalchemy.org)
- [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm))
- [pandas](https://pandas.pydata.org)
- [pandahouse](https://github.com/kszucs/pandahouse)
- PHP
- [Doctrine](https://www.doctrine-project.org/)
- [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse)
- R
- [dplyr](https://db.rstudio.com/dplyr/)
- [RClickHouse](https://github.com/IMSMWU/RClickHouse) (uses [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp))
- Java
- [Hadoop](http://hadoop.apache.org)
- [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (uses [JDBC](../../sql-reference/table-functions/jdbc.md))
- Scala
- [Akka](https://akka.io)
- [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client)
- C#
- [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview)
- [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net)
- [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client)
- [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net)
- [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations)
- [Linq To DB](https://github.com/linq2db/linq2db)
- Elixir
- [Ecto](https://github.com/elixir-ecto/ecto)
- [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto)
- Ruby
- [Ruby on Rails](https://rubyonrails.org/)
- [activecube](https://github.com/bitquery/activecube)
- [ActiveRecord](https://github.com/PNixx/clickhouse-activerecord)
- [GraphQL](https://github.com/graphql)
- [activecube-graphql](https://github.com/bitquery/activecube-graphql)
- Python
- [SQLAlchemy](https://www.sqlalchemy.org)
- [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm))
- [pandas](https://pandas.pydata.org)
- [pandahouse](https://github.com/kszucs/pandahouse)
- PHP
- [Doctrine](https://www.doctrine-project.org/)
- [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse)
- R
- [dplyr](https://db.rstudio.com/dplyr/)
- [RClickHouse](https://github.com/IMSMWU/RClickHouse) (uses [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp))
- Java
- [Hadoop](http://hadoop.apache.org)
- [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (uses [JDBC](../../sql-reference/table-functions/jdbc.md))
- Scala
- [Akka](https://akka.io)
- [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client)
- C#
- [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview)
- [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net)
- [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client)
- [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net)
- [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations)
- [Linq To DB](https://github.com/linq2db/linq2db)
- Elixir
- [Ecto](https://github.com/elixir-ecto/ecto)
- [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto)
- Ruby
- [Ruby on Rails](https://rubyonrails.org/)
- [activecube](https://github.com/bitquery/activecube)
- [ActiveRecord](https://github.com/PNixx/clickhouse-activerecord)
- [GraphQL](https://github.com/graphql)
- [activecube-graphql](https://github.com/bitquery/activecube-graphql)

View File

@ -12,9 +12,9 @@ sidebar_label: Proxies
Features:
- Per-user routing and response caching.
- Flexible limits.
- Automatic SSL certificate renewal.
- Per-user routing and response caching.
- Flexible limits.
- Automatic SSL certificate renewal.
Implemented in Go.
@ -24,9 +24,9 @@ Implemented in Go.
Features:
- In-memory and on-disk data buffering.
- Per-table routing.
- Load-balancing and health checking.
- In-memory and on-disk data buffering.
- Per-table routing.
- Load-balancing and health checking.
Implemented in Go.
@ -36,8 +36,8 @@ Implemented in Go.
Features:
- Group requests and send by threshold or interval.
- Multiple remote servers.
- Basic authentication.
- Group requests and send by threshold or interval.
- Multiple remote servers.
- Basic authentication.
Implemented in Go.

View File

@ -1,17 +1,17 @@
[//]: # (This file is included in FAQ > Troubleshooting)
- [Installation](#troubleshooting-installation-errors)
- [Connecting to the server](#troubleshooting-accepts-no-connections)
- [Query processing](#troubleshooting-does-not-process-queries)
- [Efficiency of query processing](#troubleshooting-too-slow)
- [Installation](#troubleshooting-installation-errors)
- [Connecting to the server](#troubleshooting-accepts-no-connections)
- [Query processing](#troubleshooting-does-not-process-queries)
- [Efficiency of query processing](#troubleshooting-too-slow)
## Installation {#troubleshooting-installation-errors}
### You Cannot Get Deb Packages from ClickHouse Repository with Apt-get {#you-cannot-get-deb-packages-from-clickhouse-repository-with-apt-get}
- Check firewall settings.
- If you cannot access the repository for any reason, download packages as described in the [install guide](../getting-started/install.md) article and install them manually using the `sudo dpkg -i <packages>` command. You will also need the `tzdata` package.
- Check firewall settings.
- If you cannot access the repository for any reason, download packages as described in the [install guide](../getting-started/install.md) article and install them manually using the `sudo dpkg -i <packages>` command. You will also need the `tzdata` package.
### You Cannot Update Deb Packages from ClickHouse Repository with Apt-get {#you-cannot-update-deb-packages-from-clickhouse-repository-with-apt-get}
@ -73,8 +73,8 @@ After that follow the [install guide](../getting-started/install.md#from-rpm-pac
Possible issues:
- The server is not running.
- Unexpected or wrong configuration parameters.
- The server is not running.
- Unexpected or wrong configuration parameters.
### Server Is Not Running {#server-is-not-running}
@ -98,8 +98,8 @@ The main log of `clickhouse-server` is in `/var/log/clickhouse-server/clickhouse
If the server started successfully, you should see the strings:
- `<Information> Application: starting up.` — Server started.
- `<Information> Application: Ready for connections.` — Server is running and ready for connections.
- `<Information> Application: starting up.` — Server started.
- `<Information> Application: Ready for connections.` — Server is running and ready for connections.
If `clickhouse-server` start failed with a configuration error, you should see the `<Error>` string with an error description. For example:
@ -149,30 +149,30 @@ This command starts the server as an interactive app with standard parameters of
Check:
- Docker settings.
- Docker settings.
If you run ClickHouse in Docker in an IPv6 network, make sure that `network=host` is set.
- Endpoint settings.
- Endpoint settings.
Check [listen_host](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) and [tcp_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) settings.
ClickHouse server accepts localhost connections only by default.
- HTTP protocol settings.
- HTTP protocol settings.
Check protocol settings for the HTTP API.
- Secure connection settings.
- Secure connection settings.
Check:
- The [tcp_port_secure](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) setting.
- Settings for [SSL certificates](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl).
- The [tcp_port_secure](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) setting.
- Settings for [SSL certificates](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl).
Use proper parameters while connecting. For example, use the `port_secure` parameter with `clickhouse_client`.
- User settings.
- User settings.
You might be using the wrong user name or password.

View File

@ -14,10 +14,10 @@ All XML files should have the same root element, usually `<clickhouse>`. As for
Some settings specified in the main configuration file can be overridden in other configuration files:
- The `replace` or `remove` attributes can be specified for the elements of these configuration files.
- If neither is specified, it combines the contents of elements recursively, replacing values of duplicate children.
- If `replace` is specified, it replaces the entire element with the specified one.
- If `remove` is specified, it deletes the element.
- The `replace` or `remove` attributes can be specified for the elements of these configuration files.
- If neither is specified, it combines the contents of elements recursively, replacing values of duplicate children.
- If `replace` is specified, it replaces the entire element with the specified one.
- If `remove` is specified, it deletes the element.
You can also declare attributes as coming from environment variables by using `from_env="VARIABLE_NAME"`:

View File

@ -12,15 +12,15 @@ import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_automated.md
You can monitor:
- Utilization of hardware resources.
- ClickHouse server metrics.
- Utilization of hardware resources.
- ClickHouse server metrics.
## Resource Utilization {#resource-utilization}
ClickHouse also monitors the state of hardware resources by itself such as:
- Load and temperature on processors.
- Utilization of storage system, RAM and network.
- Load and temperature on processors.
- Utilization of storage system, RAM and network.
This data is collected in the `system.asynchronous_metric_log` table.
@ -32,8 +32,8 @@ To track server events use server logs. See the [logger](../operations/server-co
ClickHouse collects:
- Different metrics of how the server uses computational resources.
- Common statistics on query processing.
- Different metrics of how the server uses computational resources.
- Common statistics on query processing.
You can find metrics in the [system.metrics](../operations/system-tables/metrics.md#system_tables-metrics), [system.events](../operations/system-tables/events.md#system_tables-events), and [system.asynchronous_metrics](../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) tables.

View File

@ -6,4 +6,4 @@ sidebar_position: 52
# Optimizing Performance
- [Sampling query profiler](../../operations/optimizing-performance/sampling-query-profiler.md)
- [Sampling query profiler](../../operations/optimizing-performance/sampling-query-profiler.md)

View File

@ -25,11 +25,11 @@ SETTINGS allow_introspection_functions = 1
In self-managed deployments, to use query profiler:
- Setup the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) section of the server configuration.
- Setup the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) section of the server configuration.
This section configures the [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table containing the results of the profiler functioning. It is configured by default. Remember that data in this table is valid only for a running server. After the server restart, ClickHouse does not clean up the table and all the stored virtual memory address may become invalid.
- Setup the [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) or [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) settings. Both settings can be used simultaneously.
- Setup the [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) or [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) settings. Both settings can be used simultaneously.
These settings allow you to configure profiler timers. As these are the session settings, you can get different sampling frequency for the whole server, individual users or user profiles, for your interactive session, and for each individual query.
@ -37,13 +37,13 @@ The default sampling frequency is one sample per second and both CPU and real ti
To analyze the `trace_log` system table:
- Install the `clickhouse-common-static-dbg` package. See [Install from DEB Packages](../../getting-started/install.md#install-from-deb-packages).
- Install the `clickhouse-common-static-dbg` package. See [Install from DEB Packages](../../getting-started/install.md#install-from-deb-packages).
- Allow introspection functions by the [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) setting.
- Allow introspection functions by the [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) setting.
For security reasons, introspection functions are disabled by default.
- Use the `addressToLine`, `addressToLineWithInlines`, `addressToSymbol` and `demangle` [introspection functions](../../sql-reference/functions/introspection.md) to get function names and their positions in ClickHouse code. To get a profile for some query, you need to aggregate data from the `trace_log` table. You can aggregate data by individual functions or by the whole stack traces.
- Use the `addressToLine`, `addressToLineWithInlines`, `addressToSymbol` and `demangle` [introspection functions](../../sql-reference/functions/introspection.md) to get function names and their positions in ClickHouse code. To get a profile for some query, you need to aggregate data from the `trace_log` table. You can aggregate data by individual functions or by the whole stack traces.
If you need to visualize `trace_log` info, try [flamegraph](../../interfaces/third-party/gui.md#clickhouse-flamegraph-clickhouse-flamegraph) and [speedscope](https://github.com/laplab/clickhouse-speedscope).
@ -51,14 +51,14 @@ If you need to visualize `trace_log` info, try [flamegraph](../../interfaces/thi
In this example we:
- Filtering `trace_log` data by a query identifier and the current date.
- Filtering `trace_log` data by a query identifier and the current date.
- Aggregating by stack trace.
- Aggregating by stack trace.
- Using introspection functions, we will get a report of:
- Using introspection functions, we will get a report of:
- Names of symbols and corresponding source code functions.
- Source code locations of these functions.
- Names of symbols and corresponding source code functions.
- Source code locations of these functions.
<!-- -->

View File

@ -12,8 +12,8 @@ The system also has a feature for limiting the complexity of a single query. See
In contrast to query complexity restrictions, quotas:
- Place restrictions on a set of queries that can be run over a period of time, instead of limiting a single query.
- Account for resources spent on all remote servers for distributed query processing.
- Place restrictions on a set of queries that can be run over a period of time, instead of limiting a single query.
- Account for resources spent on all remote servers for distributed query processing.
Lets look at the section of the users.xml file that defines quotas.

View File

@ -45,17 +45,17 @@ Configuration template:
`<case>` fields:
- `min_part_size` The minimum size of a data part.
- `min_part_size_ratio` The ratio of the data part size to the table size.
- `method` Compression method. Acceptable values: `lz4`, `lz4hc`, `zstd`,`deflate_qpl`.
- `level` Compression level. See [Codecs](../../sql-reference/statements/create/table.md#create-query-general-purpose-codecs).
- `min_part_size` The minimum size of a data part.
- `min_part_size_ratio` The ratio of the data part size to the table size.
- `method` Compression method. Acceptable values: `lz4`, `lz4hc`, `zstd`,`deflate_qpl`.
- `level` Compression level. See [Codecs](../../sql-reference/statements/create/table.md#create-query-general-purpose-codecs).
You can configure multiple `<case>` sections.
Actions when conditions are met:
- If a data part matches a condition set, ClickHouse uses the specified compression method.
- If a data part matches multiple condition sets, ClickHouse uses the first matched condition set.
- If a data part matches a condition set, ClickHouse uses the specified compression method.
- If a data part matches multiple condition sets, ClickHouse uses the first matched condition set.
If no conditions met for a data part, ClickHouse uses the `lz4` compression.
@ -165,7 +165,7 @@ List of prefixes for [custom settings](../../operations/settings/index.md#custom
**See Also**
- [Custom settings](../../operations/settings/index.md#custom_settings)
- [Custom settings](../../operations/settings/index.md#custom_settings)
## core_dump {#server_configuration_parameters-core_dump}
@ -173,7 +173,7 @@ Configures soft limit for core dump file size.
Possible values:
- Positive integer.
- Positive integer.
Default value: `1073741824` (1 GB).
@ -274,8 +274,8 @@ The path to the config file for dictionaries.
Path:
- Specify the absolute path or the path relative to the server config file.
- The path can contain wildcards \* and ?.
- Specify the absolute path or the path relative to the server config file.
- The path can contain wildcards \* and ?.
See also “[Dictionaries](../../sql-reference/dictionaries/index.md)”.
@ -291,8 +291,8 @@ The path to the config file for executable user defined functions.
Path:
- Specify the absolute path or the path relative to the server config file.
- The path can contain wildcards \* and ?.
- Specify the absolute path or the path relative to the server config file.
- The path can contain wildcards \* and ?.
See also “[Executable User Defined Functions](../../sql-reference/functions/index.md#executable-user-defined-functions).”.
@ -335,15 +335,15 @@ Sending data to [Graphite](https://github.com/graphite-project).
Settings:
- host The Graphite server.
- port The port on the Graphite server.
- interval The interval for sending, in seconds.
- timeout The timeout for sending data, in seconds.
- root_path Prefix for keys.
- metrics Sending data from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table.
- events Sending deltas data accumulated for the time period from the [system.events](../../operations/system-tables/events.md#system_tables-events) table.
- events_cumulative Sending cumulative data from the [system.events](../../operations/system-tables/events.md#system_tables-events) table.
- asynchronous_metrics Sending data from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table.
- host The Graphite server.
- port The port on the Graphite server.
- interval The interval for sending, in seconds.
- timeout The timeout for sending data, in seconds.
- root_path Prefix for keys.
- metrics Sending data from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table.
- events Sending deltas data accumulated for the time period from the [system.events](../../operations/system-tables/events.md#system_tables-events) table.
- events_cumulative Sending cumulative data from the [system.events](../../operations/system-tables/events.md#system_tables-events) table.
- asynchronous_metrics Sending data from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table.
You can configure multiple `<graphite>` clauses. For instance, you can use this for sending different data at different intervals.
@ -516,10 +516,10 @@ These credentials are common for replication via `HTTP` and `HTTPS`.
The section contains the following parameters:
- `user` — Username.
- `password` — Password.
- `allow_empty` — If `true`, then other replicas are allowed to connect without authentication even if credentials are set. If `false`, then connections without authentication are refused. Default value: `false`.
- `old` — Contains old `user` and `password` used during credential rotation. Several `old` sections can be specified.
- `user` — Username.
- `password` — Password.
- `allow_empty` — If `true`, then other replicas are allowed to connect without authentication even if credentials are set. If `false`, then connections without authentication are refused. Default value: `false`.
- `old` — Contains old `user` and `password` used during credential rotation. Several `old` sections can be specified.
**Credentials Rotation**
@ -584,12 +584,12 @@ Backlog (queue size of pending connections) of the listen socket.
Default value: `4096` (as in linux [5.4+](https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/commit/?id=19f92a030ca6d772ab44b22ee6a01378a8cb32d4)).
Usually this value does not need to be changed, since:
- default value is large enough,
- and for accepting client's connections server has separate thread.
- default value is large enough,
- and for accepting client's connections server has separate thread.
So even if you have `TcpExtListenOverflows` (from `nstat`) non zero and this counter grows for ClickHouse server it does not mean that this value need to be increased, since:
- usually if 4096 is not enough it shows some internal ClickHouse scaling issue, so it is better to report an issue.
- and it does not mean that the server can handle more connections later (and even if it could, by that moment clients may be gone or disconnected).
- usually if 4096 is not enough it shows some internal ClickHouse scaling issue, so it is better to report an issue.
- and it does not mean that the server can handle more connections later (and even if it could, by that moment clients may be gone or disconnected).
Examples:
@ -603,13 +603,13 @@ Logging settings.
Keys:
- `level` Logging level. Acceptable values: `trace`, `debug`, `information`, `warning`, `error`.
- `log` The log file. Contains all the entries according to `level`.
- `errorlog` Error log file.
- `size` Size of the file. Applies to `log` and `errorlog`. Once the file reaches `size`, ClickHouse archives and renames it, and creates a new log file in its place.
- `count` The number of archived log files that ClickHouse stores.
- `console` Send `log` and `errorlog` to the console instead of file. To enable, set to `1` or `true`.
- `stream_compress` Compress `log` and `errorlog` with `lz4` stream compression. To enable, set to `1` or `true`.
- `level` Logging level. Acceptable values: `trace`, `debug`, `information`, `warning`, `error`.
- `log` The log file. Contains all the entries according to `level`.
- `errorlog` Error log file.
- `size` Size of the file. Applies to `log` and `errorlog`. Once the file reaches `size`, ClickHouse archives and renames it, and creates a new log file in its place.
- `count` The number of archived log files that ClickHouse stores.
- `console` Send `log` and `errorlog` to the console instead of file. To enable, set to `1` or `true`.
- `stream_compress` Compress `log` and `errorlog` with `lz4` stream compression. To enable, set to `1` or `true`.
**Example**
@ -649,12 +649,12 @@ Writing to the syslog is also supported. Config example:
Keys for syslog:
- use_syslog — Required setting if you want to write to the syslog.
- address — The host\[:port\] of syslogd. If omitted, the local daemon is used.
- hostname — Optional. The name of the host that logs are sent from.
- facility — [The syslog facility keyword](https://en.wikipedia.org/wiki/Syslog#Facility) in uppercase letters with the “LOG_” prefix: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3`, and so on).
- use_syslog — Required setting if you want to write to the syslog.
- address — The host\[:port\] of syslogd. If omitted, the local daemon is used.
- hostname — Optional. The name of the host that logs are sent from.
- facility — [The syslog facility keyword](https://en.wikipedia.org/wiki/Syslog#Facility) in uppercase letters with the “LOG_” prefix: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3`, and so on).
Default value: `LOG_USER` if `address` is specified, `LOG_DAEMON` otherwise.
- format Message format. Possible values: `bsd` and `syslog.`
- format Message format. Possible values: `bsd` and `syslog.`
## send_crash_reports {#server_configuration_parameters-send_crash_reports}
@ -665,13 +665,13 @@ The server will need access to the public Internet via IPv4 (at the time of writ
Keys:
- `enabled` Boolean flag to enable the feature, `false` by default. Set to `true` to allow sending crash reports.
- `endpoint` You can override the Sentry endpoint URL for sending crash reports. It can be either a separate Sentry account or your self-hosted Sentry instance. Use the [Sentry DSN](https://docs.sentry.io/error-reporting/quickstart/?platform=native#configure-the-sdk) syntax.
- `anonymize` - Avoid attaching the server hostname to the crash report.
- `http_proxy` - Configure HTTP proxy for sending crash reports.
- `debug` - Sets the Sentry client into debug mode.
- `tmp_path` - Filesystem path for temporary crash report state.
- `environment` - An arbitrary name of an environment in which the ClickHouse server is running. It will be mentioned in each crash report. The default value is `test` or `prod` depending on the version of ClickHouse.
- `enabled` Boolean flag to enable the feature, `false` by default. Set to `true` to allow sending crash reports.
- `endpoint` You can override the Sentry endpoint URL for sending crash reports. It can be either a separate Sentry account or your self-hosted Sentry instance. Use the [Sentry DSN](https://docs.sentry.io/error-reporting/quickstart/?platform=native#configure-the-sdk) syntax.
- `anonymize` - Avoid attaching the server hostname to the crash report.
- `http_proxy` - Configure HTTP proxy for sending crash reports.
- `debug` - Sets the Sentry client into debug mode.
- `tmp_path` - Filesystem path for temporary crash report state.
- `environment` - An arbitrary name of an environment in which the ClickHouse server is running. It will be mentioned in each crash report. The default value is `test` or `prod` depending on the version of ClickHouse.
**Recommended way to use**
@ -713,8 +713,8 @@ Limits total RAM usage by the ClickHouse server.
Possible values:
- Positive integer.
- 0 — Auto.
- Positive integer.
- 0 — Auto.
Default value: `0`.
@ -724,8 +724,8 @@ The default `max_server_memory_usage` value is calculated as `memory_amount * ma
**See also**
- [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage)
- [max_server_memory_usage_to_ram_ratio](#max_server_memory_usage_to_ram_ratio)
- [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage)
- [max_server_memory_usage_to_ram_ratio](#max_server_memory_usage_to_ram_ratio)
## max_server_memory_usage_to_ram_ratio {#max_server_memory_usage_to_ram_ratio}
@ -733,8 +733,8 @@ Defines the fraction of total physical RAM amount, available to the ClickHouse s
Possible values:
- Positive double.
- 0 — The ClickHouse server can use all available RAM.
- Positive double.
- 0 — The ClickHouse server can use all available RAM.
Default value: `0.9`.
@ -750,21 +750,21 @@ On hosts with low RAM and swap, you possibly need setting `max_server_memory_usa
**See Also**
- [max_server_memory_usage](#max_server_memory_usage)
- [max_server_memory_usage](#max_server_memory_usage)
## concurrent_threads_soft_limit_num {#concurrent_threads_soft_limit_num}
The maximum number of query processing threads, excluding threads for retrieving data from remote servers, allowed to run all queries. This is not a hard limit. In case if the limit is reached the query will still get at least one thread to run. Query can upscale to desired number of threads during execution if more threads become available.
Possible values:
- Positive integer.
- 0 — No limit.
- Positive integer.
- 0 — No limit.
Default value: `0`.
**See Also**
- [Concurrency Control](/docs/en/development/architecture.md#concurrency-control)
- [Concurrency Control](/docs/en/development/architecture.md#concurrency-control)
## concurrent_threads_soft_limit_ratio_to_cores {#concurrent_threads_soft_limit_ratio_to_cores}
The maximum number of query processing threads as multiple of number of logical cores.
@ -772,8 +772,8 @@ More details: [concurrent_threads_soft_limit_num](#concurrent_threads_soft_limit
Possible values:
- Positive integer.
- 0 — No limit.
- Positive integer.
- 0 — No limit.
Default value: `0`.
@ -794,8 +794,8 @@ These settings can be modified at runtime and will take effect immediately. Quer
Possible values:
- Positive integer.
- 0 — No limit.
- Positive integer.
- 0 — No limit.
Default value: `100`.
@ -815,8 +815,8 @@ These settings can be modified at runtime and will take effect immediately. Quer
Possible values:
- Positive integer.
- 0 — No limit.
- Positive integer.
- 0 — No limit.
Default value: `0`.
@ -836,8 +836,8 @@ These settings can be modified at runtime and will take effect immediately. Quer
Possible values:
- Positive integer.
- 0 — No limit.
- Positive integer.
- 0 — No limit.
Default value: `0`.
@ -853,8 +853,8 @@ The maximum number of simultaneously processed queries related to MergeTree tabl
Possible values:
- Positive integer.
- 0 — No limit.
- Positive integer.
- 0 — No limit.
Default value: `0`.
@ -874,8 +874,8 @@ Modifying the setting for one query or user does not affect other queries.
Possible values:
- Positive integer.
- 0 — No limit.
- Positive integer.
- 0 — No limit.
Default value: `0`.
@ -887,7 +887,7 @@ Default value: `0`.
**See Also**
- [max_concurrent_queries](#max-concurrent-queries)
- [max_concurrent_queries](#max-concurrent-queries)
## max_connections {#max-connections}
@ -937,7 +937,7 @@ ClickHouse uses threads from the Global Thread pool to process queries. If there
Possible values:
- Positive integer.
- Positive integer.
Default value: `10000`.
@ -953,7 +953,7 @@ If the number of **idle** threads in the Global Thread pool is greater than `max
Possible values:
- Positive integer.
- Positive integer.
Default value: `1000`.
@ -969,8 +969,8 @@ The maximum number of jobs that can be scheduled on the Global Thread pool. Incr
Possible values:
- Positive integer.
- 0 — No limit.
- Positive integer.
- 0 — No limit.
Default value: `10000`.
@ -986,7 +986,7 @@ ClickHouse uses threads from the IO Thread pool to do some IO operations (e.g. t
Possible values:
- Positive integer.
- Positive integer.
Default value: `100`.
@ -996,7 +996,7 @@ If the number of **idle** threads in the IO Thread pool exceeds `max_io_thread_p
Possible values:
- Positive integer.
- Positive integer.
Default value: `0`.
@ -1006,8 +1006,8 @@ The maximum number of jobs that can be scheduled on the IO Thread pool.
Possible values:
- Positive integer.
- 0 — No limit.
- Positive integer.
- 0 — No limit.
Default value: `10000`.
@ -1017,7 +1017,7 @@ ClickHouse uses threads from the Backups IO Thread pool to do S3 backup IO opera
Possible values:
- Positive integer.
- Positive integer.
Default value: `1000`.
@ -1027,8 +1027,8 @@ If the number of **idle** threads in the Backups IO Thread pool exceeds `max_bac
Possible values:
- Positive integer.
- Zero.
- Positive integer.
- Zero.
Default value: `0`.
@ -1038,8 +1038,8 @@ The maximum number of jobs that can be scheduled on the Backups IO Thread pool.
Possible values:
- Positive integer.
- 0 — No limit.
- Positive integer.
- 0 — No limit.
Default value: `0`.
@ -1051,7 +1051,7 @@ Before changing it, please also take a look at related MergeTree settings, such
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 16.
@ -1069,7 +1069,7 @@ The same as for `background_pool_size` setting `background_merges_mutations_conc
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 2.
@ -1086,8 +1086,8 @@ Could be applied from the `default` profile for backward compatibility.
Possible values:
- "round_robin" — Every concurrent merge and mutation is executed in round-robin order to ensure starvation-free operation. Smaller merges are completed faster than bigger ones just because they have fewer blocks to merge.
- "shortest_task_first" — Always execute smaller merge or mutation. Merges and mutations are assigned priorities based on their resulting size. Merges with smaller sizes are strictly preferred over bigger ones. This policy ensures the fastest possible merge of small parts but can lead to indefinite starvation of big merges in partitions heavily overloaded by INSERTs.
- "round_robin" — Every concurrent merge and mutation is executed in round-robin order to ensure starvation-free operation. Smaller merges are completed faster than bigger ones just because they have fewer blocks to merge.
- "shortest_task_first" — Always execute smaller merge or mutation. Merges and mutations are assigned priorities based on their resulting size. Merges with smaller sizes are strictly preferred over bigger ones. This policy ensures the fastest possible merge of small parts but can lead to indefinite starvation of big merges in partitions heavily overloaded by INSERTs.
Default value: "round_robin".
@ -1103,7 +1103,7 @@ Sets the number of threads performing background moves for tables with MergeTree
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 8.
@ -1119,7 +1119,7 @@ Sets the number of threads performing background fetches for tables with Replica
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 8.
@ -1135,7 +1135,7 @@ Sets the number of threads performing background non-specialized operations like
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 8.
@ -1151,7 +1151,7 @@ Sets the number of threads performing background flush in [Buffer](../../engines
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 16.
@ -1161,7 +1161,7 @@ Sets the number of threads performing background tasks for [replicated](../../en
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 128.
@ -1233,26 +1233,26 @@ Support for SSL is provided by the `libpoco` library. The available configuratio
Keys for server/client settings:
- privateKeyFile The path to the file with the secret key of the PEM certificate. The file may contain a key and certificate at the same time.
- certificateFile The path to the client/server certificate file in PEM format. You can omit it if `privateKeyFile` contains the certificate.
- caConfig (default: none) The path to the file or directory that contains trusted CA certificates. If this points to a file, it must be in PEM format and can contain several CA certificates. If this points to a directory, it must contain one .pem file per CA certificate. The filenames are looked up by the CA subject name hash value. Details can be found in the man page of [SSL_CTX_load_verify_locations](https://www.openssl.org/docs/man3.0/man3/SSL_CTX_load_verify_locations.html).
- verificationMode (default: relaxed) The method for checking the nodes certificates. Details are in the description of the [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: `none`, `relaxed`, `strict`, `once`.
- verificationDepth (default: 9) The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value.
- loadDefaultCAFile (default: true) Wether built-in CA certificates for OpenSSL will be used. ClickHouse assumes that builtin CA certificates are in the file `/etc/ssl/cert.pem` (resp. the directory `/etc/ssl/certs`) or in file (resp. directory) specified by the environment variable `SSL_CERT_FILE` (resp. `SSL_CERT_DIR`).
- cipherList (default: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`) - Supported OpenSSL encryptions.
- cacheSessions (default: false) Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. Acceptable values: `true`, `false`.
- sessionIdContext (default: `${application.name}`) A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. This parameter is always recommended since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: `${application.name}`.
- sessionCacheSize (default: [1024\*20](https://github.com/ClickHouse/boringssl/blob/master/include/openssl/ssl.h#L1978)) The maximum number of sessions that the server caches. A value of 0 means unlimited sessions.
- sessionTimeout (default: [2h](https://github.com/ClickHouse/boringssl/blob/master/include/openssl/ssl.h#L1926)) Time for caching the session on the server.
- extendedVerification (default: false) If enabled, verify that the certificate CN or SAN matches the peer hostname.
- requireTLSv1 (default: false) Require a TLSv1 connection. Acceptable values: `true`, `false`.
- requireTLSv1_1 (default: false) Require a TLSv1.1 connection. Acceptable values: `true`, `false`.
- requireTLSv1_2 (default: false) Require a TLSv1.2 connection. Acceptable values: `true`, `false`.
- fips (default: false) Activates OpenSSL FIPS mode. Supported if the librarys OpenSSL version supports FIPS.
- privateKeyPassphraseHandler (default: `KeyConsoleHandler`) Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: `<privateKeyPassphraseHandler>`, `<name>KeyFileHandler</name>`, `<options><password>test</password></options>`, `</privateKeyPassphraseHandler>`.
- invalidCertificateHandler (default: `ConsoleCertificateHandler`) Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: `<invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>` .
- disableProtocols (default: "") Protocols that are not allowed to use.
- preferServerCiphers (default: false) Preferred server ciphers on the client.
- privateKeyFile The path to the file with the secret key of the PEM certificate. The file may contain a key and certificate at the same time.
- certificateFile The path to the client/server certificate file in PEM format. You can omit it if `privateKeyFile` contains the certificate.
- caConfig (default: none) The path to the file or directory that contains trusted CA certificates. If this points to a file, it must be in PEM format and can contain several CA certificates. If this points to a directory, it must contain one .pem file per CA certificate. The filenames are looked up by the CA subject name hash value. Details can be found in the man page of [SSL_CTX_load_verify_locations](https://www.openssl.org/docs/man3.0/man3/SSL_CTX_load_verify_locations.html).
- verificationMode (default: relaxed) The method for checking the nodes certificates. Details are in the description of the [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: `none`, `relaxed`, `strict`, `once`.
- verificationDepth (default: 9) The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value.
- loadDefaultCAFile (default: true) Wether built-in CA certificates for OpenSSL will be used. ClickHouse assumes that builtin CA certificates are in the file `/etc/ssl/cert.pem` (resp. the directory `/etc/ssl/certs`) or in file (resp. directory) specified by the environment variable `SSL_CERT_FILE` (resp. `SSL_CERT_DIR`).
- cipherList (default: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`) - Supported OpenSSL encryptions.
- cacheSessions (default: false) Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. Acceptable values: `true`, `false`.
- sessionIdContext (default: `${application.name}`) A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. This parameter is always recommended since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: `${application.name}`.
- sessionCacheSize (default: [1024\*20](https://github.com/ClickHouse/boringssl/blob/master/include/openssl/ssl.h#L1978)) The maximum number of sessions that the server caches. A value of 0 means unlimited sessions.
- sessionTimeout (default: [2h](https://github.com/ClickHouse/boringssl/blob/master/include/openssl/ssl.h#L1926)) Time for caching the session on the server.
- extendedVerification (default: false) If enabled, verify that the certificate CN or SAN matches the peer hostname.
- requireTLSv1 (default: false) Require a TLSv1 connection. Acceptable values: `true`, `false`.
- requireTLSv1_1 (default: false) Require a TLSv1.1 connection. Acceptable values: `true`, `false`.
- requireTLSv1_2 (default: false) Require a TLSv1.2 connection. Acceptable values: `true`, `false`.
- fips (default: false) Activates OpenSSL FIPS mode. Supported if the librarys OpenSSL version supports FIPS.
- privateKeyPassphraseHandler (default: `KeyConsoleHandler`) Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: `<privateKeyPassphraseHandler>`, `<name>KeyFileHandler</name>`, `<options><password>test</password></options>`, `</privateKeyPassphraseHandler>`.
- invalidCertificateHandler (default: `ConsoleCertificateHandler`) Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: `<invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>` .
- disableProtocols (default: "") Protocols that are not allowed to use.
- preferServerCiphers (default: false) Preferred server ciphers on the client.
**Example of settings:**
@ -1292,12 +1292,12 @@ Queries are logged in the [system.part_log](../../operations/system-tables/part_
Use the following parameters to configure logging:
- `database` Name of the database.
- `table` Name of the system table.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
- `database` Name of the database.
- `table` Name of the system table.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
**Example**
@ -1330,11 +1330,11 @@ Exposing metrics data for scraping from [Prometheus](https://prometheus.io).
Settings:
- `endpoint` HTTP endpoint for scraping metrics by prometheus server. Start from /.
- `port` Port for `endpoint`.
- `metrics` Flag that sets to expose metrics from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table.
- `events` Flag that sets to expose metrics from the [system.events](../../operations/system-tables/events.md#system_tables-events) table.
- `asynchronous_metrics` Flag that sets to expose current metrics values from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table.
- `endpoint` HTTP endpoint for scraping metrics by prometheus server. Start from /.
- `port` Port for `endpoint`.
- `metrics` Flag that sets to expose metrics from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table.
- `events` Flag that sets to expose metrics from the [system.events](../../operations/system-tables/events.md#system_tables-events) table.
- `asynchronous_metrics` Flag that sets to expose current metrics values from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table.
**Example**
@ -1356,12 +1356,12 @@ Queries are logged in the [system.query_log](../../operations/system-tables/quer
Use the following parameters to configure logging:
- `database` Name of the database.
- `table` Name of the system table the queries will be logged in.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
- `database` Name of the database.
- `table` Name of the system table the queries will be logged in.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
If the table does not exist, ClickHouse will create it. If the structure of the query log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically.
@ -1382,10 +1382,10 @@ If the table does not exist, ClickHouse will create it. If the structure of the
The following settings are available:
- `max_size`: The maximum cache size in bytes. 0 means the query cache is disabled. Default value: `1073741824` (1 GiB).
- `max_entries`: The maximum number of `SELECT` query results stored in the cache. Default value: `1024`.
- `max_entry_size`: The maximum size in bytes `SELECT` query results may have to be saved in the cache. Default value: `1048576` (1 MiB).
- `max_entry_rows`: The maximum number of rows `SELECT` query results may have to be saved in the cache. Default value: `30000000` (30 mil).
- `max_size`: The maximum cache size in bytes. 0 means the query cache is disabled. Default value: `1073741824` (1 GiB).
- `max_entries`: The maximum number of `SELECT` query results stored in the cache. Default value: `1024`.
- `max_entry_size`: The maximum size in bytes `SELECT` query results may have to be saved in the cache. Default value: `1048576` (1 MiB).
- `max_entry_rows`: The maximum number of rows `SELECT` query results may have to be saved in the cache. Default value: `30000000` (30 mil).
Changed settings take effect immediately.
@ -1397,7 +1397,7 @@ Data for the query cache is allocated in DRAM. If memory is scarce, make sure to
```xml
<query_cache>
<size>1073741824</size>
<max_size>1073741824</max_size>
<max_entries>1024</max_entries>
<max_entry_size>1048576</max_entry_size>
<max_entry_rows>30000000</max_entry_rows>
@ -1412,12 +1412,12 @@ Queries are logged in the [system.query_thread_log](../../operations/system-tabl
Use the following parameters to configure logging:
- `database` Name of the database.
- `table` Name of the system table the queries will be logged in.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
- `database` Name of the database.
- `table` Name of the system table the queries will be logged in.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
If the table does not exist, ClickHouse will create it. If the structure of the query thread log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically.
@ -1440,12 +1440,12 @@ Queries are logged in the [system.query_views_log](../../operations/system-table
Use the following parameters to configure logging:
- `database` Name of the database.
- `table` Name of the system table the queries will be logged in.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
- `database` Name of the database.
- `table` Name of the system table the queries will be logged in.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
If the table does not exist, ClickHouse will create it. If the structure of the query views log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically.
@ -1466,13 +1466,13 @@ Settings for the [text_log](../../operations/system-tables/text_log.md#system_ta
Parameters:
- `level` — Maximum Message Level (by default `Trace`) which will be stored in a table.
- `database` — Database name.
- `table` — Table name.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
- `level` — Maximum Message Level (by default `Trace`) which will be stored in a table.
- `database` — Database name.
- `table` — Table name.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
**Example**
```xml
@ -1495,12 +1495,12 @@ Settings for the [trace_log](../../operations/system-tables/trace_log.md#system_
Parameters:
- `database` — Database for storing a table.
- `table` — Table name.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
- `database` — Database for storing a table.
- `table` — Table name.
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` defined.
- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table.
- `storage_policy` Name of storage policy to use for the table (optional)
The default server configuration file `config.xml` contains the following settings section:
@ -1557,7 +1557,7 @@ For the value of the `incl` attribute, see the section “[Configuration files](
**See Also**
- [skip_unavailable_shards](../../operations/settings/settings.md#settings-skip_unavailable_shards)
- [skip_unavailable_shards](../../operations/settings/settings.md#settings-skip_unavailable_shards)
## timezone {#server_configuration_parameters-timezone}
@ -1731,11 +1731,11 @@ Default value: `0`.
**See also**
- [max_temporary_data_on_disk_size_for_user](../../operations/settings/query-complexity.md#settings_max_temporary_data_on_disk_size_for_user)
- [max_temporary_data_on_disk_size_for_query](../../operations/settings/query-complexity.md#settings_max_temporary_data_on_disk_size_for_query)
- [tmp_path](#tmp-path)
- [tmp_policy](#tmp-policy)
- [max_server_memory_usage](#max_server_memory_usage)
- [max_temporary_data_on_disk_size_for_user](../../operations/settings/query-complexity.md#settings_max_temporary_data_on_disk_size_for_user)
- [max_temporary_data_on_disk_size_for_query](../../operations/settings/query-complexity.md#settings_max_temporary_data_on_disk_size_for_query)
- [tmp_path](#tmp-path)
- [tmp_policy](#tmp-policy)
- [max_server_memory_usage](#max_server_memory_usage)
## uncompressed_cache_size {#server-settings-uncompressed_cache_size}
@ -1785,10 +1785,10 @@ The directory with user defined files. Used for SQL user defined functions [SQL
Path to the file that contains:
- User configurations.
- Access rights.
- Settings profiles.
- Quota settings.
- User configurations.
- Access rights.
- Settings profiles.
- Quota settings.
**Example**
@ -1804,7 +1804,7 @@ ClickHouse uses ZooKeeper for storing metadata of replicas when using replicated
This section contains the following parameters:
- `node` — ZooKeeper endpoint. You can set multiple endpoints.
- `node` — ZooKeeper endpoint. You can set multiple endpoints.
For example:
@ -1847,9 +1847,9 @@ This section contains the following parameters:
**See Also**
- [Replication](../../engines/table-engines/mergetree-family/replication.md)
- [ZooKeeper Programmers Guide](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html)
- [Optional secured communication between ClickHouse and Zookeeper](../ssl-zookeeper.md#secured-communication-with-zookeeper)
- [Replication](../../engines/table-engines/mergetree-family/replication.md)
- [ZooKeeper Programmers Guide](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html)
- [Optional secured communication between ClickHouse and Zookeeper](../ssl-zookeeper.md#secured-communication-with-zookeeper)
## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper}
@ -1857,18 +1857,18 @@ Storage method for data part headers in ZooKeeper.
This setting only applies to the `MergeTree` family. It can be specified:
- Globally in the [merge_tree](#server_configuration_parameters-merge_tree) section of the `config.xml` file.
- Globally in the [merge_tree](#server_configuration_parameters-merge_tree) section of the `config.xml` file.
ClickHouse uses the setting for all the tables on the server. You can change the setting at any time. Existing tables change their behaviour when the setting changes.
- For each table.
- For each table.
When creating a table, specify the corresponding [engine setting](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table). The behaviour of an existing table with this setting does not change, even if the global setting changes.
**Possible values**
- 0 — Functionality is turned off.
- 1 — Functionality is turned on.
- 0 — Functionality is turned off.
- 1 — Functionality is turned on.
If `use_minimalistic_part_header_in_zookeeper = 1`, then [replicated](../../engines/table-engines/mergetree-family/replication.md) tables store the headers of the data parts compactly using a single `znode`. If the table contains many columns, this storage method significantly reduces the volume of the data stored in Zookeeper.
@ -1896,7 +1896,7 @@ The update is performed asynchronously, in a separate system thread.
**See also**
- [background_schedule_pool_size](../../operations/settings/settings.md#background_schedule_pool_size)
- [background_schedule_pool_size](../../operations/settings/settings.md#background_schedule_pool_size)
## distributed_ddl {#server-settings-distributed_ddl}
@ -1954,9 +1954,9 @@ Default value: `/var/lib/clickhouse/access/`.
## user_directories {#user_directories}
Section of the configuration file that contains settings:
- Path to configuration file with predefined users.
- Path to folder where users created by SQL commands are stored.
- ZooKeeper node path where users created by SQL commands are stored and replicated (experimental).
- Path to configuration file with predefined users.
- Path to folder where users created by SQL commands are stored.
- ZooKeeper node path where users created by SQL commands are stored and replicated (experimental).
If this section is specified, the path from [users_config](../../operations/server-configuration-parameters/settings.md#users-config) and [access_control_path](../../operations/server-configuration-parameters/settings.md#access_control_path) won't be used.
@ -1991,8 +1991,8 @@ Users, roles, row policies, quotas, and profiles can be also stored in ZooKeeper
You can also define sections `memory` — means storing information only in memory, without writing to disk, and `ldap` — means storing information on an LDAP server.
To add an LDAP server as a remote user directory of users that are not defined locally, define a single `ldap` section with a following parameters:
- `server` — one of LDAP server names defined in `ldap_servers` config section. This parameter is mandatory and cannot be empty.
- `roles` — section with a list of locally defined roles that will be assigned to each user retrieved from the LDAP server. If no roles are specified, user will not be able to perform any actions after authentication. If any of the listed roles is not defined locally at the time of authentication, the authentication attempt will fail as if the provided password was incorrect.
- `server` — one of LDAP server names defined in `ldap_servers` config section. This parameter is mandatory and cannot be empty.
- `roles` — section with a list of locally defined roles that will be assigned to each user retrieved from the LDAP server. If no roles are specified, user will not be able to perform any actions after authentication. If any of the listed roles is not defined locally at the time of authentication, the authentication attempt will fail as if the provided password was incorrect.
**Example**
@ -2012,7 +2012,7 @@ Sets the memory size (in bytes) for a stack trace at every peak allocation step.
Possible values:
- Positive integer.
- Positive integer.
Default value: `4194304`.
@ -2022,8 +2022,8 @@ Allows to collect random allocations and deallocations and writes them in the [s
Possible values:
- Positive integer.
- 0 — Writing of random allocations and deallocations in the `system.trace_log` system table is disabled.
- Positive integer.
- 0 — Writing of random allocations and deallocations in the `system.trace_log` system table is disabled.
Default value: `0`.
@ -2033,7 +2033,7 @@ Sets the cache size (in bytes) for mapped files. This setting allows to avoid fr
Possible values:
- Positive integer.
- Positive integer.
Default value: `1000`.
@ -2043,7 +2043,7 @@ Sets the cache size (in bytes) for [compiled expressions](../../operations/cache
Possible values:
- Positive integer.
- Positive integer.
Default value: `134217728`.
@ -2053,6 +2053,6 @@ Sets the cache size (in elements) for [compiled expressions](../../operations/ca
Possible values:
- Positive integer.
- Positive integer.
Default value: `10000`.

View File

@ -22,9 +22,9 @@ The order of priority for defining a setting is:
3. Query settings
- When starting the ClickHouse console client in non-interactive mode, set the startup parameter `--setting=value`.
- When using the HTTP API, pass CGI parameters (`URL?setting_1=value&setting_2=value...`).
- Define settings in the [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select-query) clause of the SELECT query. The setting value is applied only to that query and is reset to the default or previous value after the query is executed.
- When starting the ClickHouse console client in non-interactive mode, set the startup parameter `--setting=value`.
- When using the HTTP API, pass CGI parameters (`URL?setting_1=value&setting_2=value...`).
- Define settings in the [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select-query) clause of the SELECT query. The setting value is applied only to that query and is reset to the default or previous value after the query is executed.
View the [Settings](./settings.md) page for a description of the ClickHouse settings.
@ -93,4 +93,4 @@ SELECT getSetting('custom_a');
**See Also**
- [Server Configuration Settings](../../operations/server-configuration-parameters/settings.md)
- [Server Configuration Settings](../../operations/server-configuration-parameters/settings.md)

View File

@ -13,7 +13,7 @@ If the number of broken parts in a single partition exceeds the `max_suspicious_
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 100.
@ -54,7 +54,7 @@ If the number of active parts in a single partition exceeds the `parts_to_throw_
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 300.
@ -69,7 +69,7 @@ If the number of active parts in a single partition exceeds the `parts_to_delay_
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 150.
@ -81,7 +81,7 @@ If the number of inactive parts in a single partition more than the `inactive_pa
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 0 (unlimited).
@ -91,7 +91,7 @@ If the number of inactive parts in a single partition in the table at least that
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 0 (unlimited).
@ -101,7 +101,7 @@ The value in seconds, which is used to calculate the `INSERT` delay, if the numb
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 1.
@ -127,7 +127,7 @@ If the total number of active parts in all partitions of a table exceeds the `ma
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 100000.
@ -146,8 +146,8 @@ The number of most recently inserted blocks for which ClickHouse Keeper stores h
Possible values:
- Any positive integer.
- 0 (disable deduplication)
- Any positive integer.
- 0 (disable deduplication)
Default value: 100.
@ -161,8 +161,8 @@ The number of the most recently inserted blocks in the non-replicated [MergeTree
Possible values:
- Any positive integer.
- 0 (disable deduplication).
- Any positive integer.
- 0 (disable deduplication).
Default value: 0.
@ -174,7 +174,7 @@ The number of seconds after which the hash sums of the inserted blocks are remov
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 604800 (1 week).
@ -188,8 +188,8 @@ The number of most recently async inserted blocks for which ClickHouse Keeper st
Possible values:
- Any positive integer.
- 0 (disable deduplication for async_inserts)
- Any positive integer.
- 0 (disable deduplication for async_inserts)
Default value: 10000.
@ -203,7 +203,7 @@ The number of seconds after which the hash sums of the async inserts are removed
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 604800 (1 week).
@ -229,7 +229,7 @@ The minimum interval (in milliseconds) to update the `use_async_block_ids_cache`
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 100.
@ -241,7 +241,7 @@ How many records may be in the ClickHouse Keeper log if there is inactive replic
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 1000
@ -251,7 +251,7 @@ Keep about this number of last records in ZooKeeper log, even if they are obsole
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 10
@ -261,7 +261,7 @@ If the time passed since a replication log (ClickHouse Keeper or ZooKeeper) entr
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 3600
@ -271,7 +271,7 @@ If the sum of the size of parts exceeds this threshold and the time since a repl
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 10,737,418,240
@ -281,7 +281,7 @@ When this setting has a value greater than zero, only a single replica starts th
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 0 (seconds)
@ -295,7 +295,7 @@ Zero-copy replication is disabled by default in ClickHouse version 22.8 and high
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 10800
@ -305,7 +305,7 @@ Recompression works slow in most cases, so we don't start merge with recompressi
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 7200
@ -315,7 +315,7 @@ If true, this replica never merges parts and always downloads merged parts from
Possible values:
- true, false
- true, false
Default value: false
@ -325,7 +325,7 @@ Max broken parts, if more - deny automatic deletion.
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 100
@ -336,7 +336,7 @@ Max size of all broken parts, if more - deny automatic deletion.
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 1,073,741,824
@ -346,7 +346,7 @@ Do not apply ALTER if number of files for modification(deletion, addition) is gr
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 75
@ -356,7 +356,7 @@ Do not apply ALTER, if the number of files for deletion is greater than this set
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 50
@ -366,7 +366,7 @@ If the ratio of wrong parts to total number of parts is less than this - allow t
Possible values:
- Float, 0.0 - 1.0
- Float, 0.0 - 1.0
Default value: 0.5
@ -376,7 +376,7 @@ Limit parallel fetches from endpoint (actually pool size).
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 15
@ -386,7 +386,7 @@ HTTP connection timeout for part fetch requests. Inherited from default profile
Possible values:
- Any positive integer.
- Any positive integer.
Default value: Inherited from default profile `http_connection_timeout` if not set explicitly.
@ -396,7 +396,7 @@ If true, replicated tables replicas on this node will try to acquire leadership.
Possible values:
- true, false
- true, false
Default value: true
@ -406,7 +406,7 @@ ZooKeeper session expiration check period, in seconds.
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 60
@ -416,7 +416,7 @@ Do not remove old local parts when repairing lost replica.
Possible values:
- true, false
- true, false
Default value: true
@ -426,8 +426,8 @@ HTTP connection timeout (in seconds) for part fetch requests. Inherited from def
Possible values:
- Any positive integer.
- 0 - Use value of `http_connection_timeout`.
- Any positive integer.
- 0 - Use value of `http_connection_timeout`.
Default value: 0.
@ -437,8 +437,8 @@ HTTP send timeout (in seconds) for part fetch requests. Inherited from default p
Possible values:
- Any positive integer.
- 0 - Use value of `http_send_timeout`.
- Any positive integer.
- 0 - Use value of `http_send_timeout`.
Default value: 0.
@ -448,8 +448,8 @@ HTTP receive timeout (in seconds) for fetch part requests. Inherited from defaul
Possible values:
- Any positive integer.
- 0 - Use value of `http_receive_timeout`.
- Any positive integer.
- 0 - Use value of `http_receive_timeout`.
Default value: 0.
@ -463,8 +463,8 @@ The setting isn't followed perfectly accurately.
Possible values:
- Positive integer.
- 0 — Unlimited.
- Positive integer.
- 0 — Unlimited.
Default value: `0`.
@ -482,8 +482,8 @@ The setting isn't followed perfectly accurately.
Possible values:
- Positive integer.
- 0 — Unlimited.
- Positive integer.
- 0 — Unlimited.
Default value: `0`.
@ -497,7 +497,7 @@ The time (in seconds) of storing inactive parts to protect against data loss dur
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 480.
@ -520,7 +520,7 @@ The maximum total parts size (in bytes) to be merged into one part, if there are
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 161061273600 (150 GB).
@ -534,7 +534,7 @@ The maximum total part size (in bytes) to be merged into one part, with the mini
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 1048576 (1 MB)
@ -547,7 +547,7 @@ The number of rows that are read from the merged parts into memory.
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 8192
@ -560,7 +560,7 @@ This is to allow small merges to process - not filling the pool with long runnin
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 8
@ -571,7 +571,7 @@ This is to leave free threads for regular merges and avoid "Too many parts".
Possible values:
- Any positive integer.
- Any positive integer.
Default value: 20
@ -585,7 +585,7 @@ The maximum number of threads that read parts when ClickHouse starts.
Possible values:
- Any positive integer.
- Any positive integer.
Default value: auto (number of CPU cores).
@ -599,7 +599,7 @@ The setting value specified when the table is created can be overridden via quer
Possible values:
- Any positive integer.
- Any positive integer.
Default value: -1 (unlimited).
@ -609,7 +609,7 @@ Merge parts if every part in the range is older than the value of `min_age_to_fo
Possible values:
- Positive integer.
- Positive integer.
Default value: 0 — Disabled.
@ -619,7 +619,7 @@ Whether `min_age_to_force_merge_seconds` should be applied only on the entire pa
Possible values:
- true, false
- true, false
Default value: false
@ -629,8 +629,8 @@ Enables to allow floating-point number as a partition key.
Possible values:
- 0 — Floating-point partition key not allowed.
- 1 — Floating-point partition key allowed.
- 0 — Floating-point partition key not allowed.
- 1 — Floating-point partition key allowed.
Default value: `0`.
@ -640,8 +640,8 @@ Enables the check at table creation, that the data type of a column for sampling
Possible values:
- true — The check is enabled.
- false — The check is disabled at table creation.
- true — The check is enabled.
- false — The check is disabled at table creation.
Default value: `true`.
@ -653,8 +653,8 @@ Sets minimal amount of bytes to enable balancing when distributing new big parts
Possible values:
- Positive integer.
- 0 — Balancing is disabled.
- Positive integer.
- 0 — Balancing is disabled.
Default value: `0`.
@ -670,8 +670,8 @@ The setting is applicable to `MergeTree` tables with enabled [data replication](
Possible values:
- 0 — Parts are removed.
- 1 — Parts are detached.
- 0 — Parts are removed.
- 1 — Parts are detached.
Default value: `0`.
@ -681,7 +681,7 @@ Sets the interval in seconds for ClickHouse to execute the cleanup of old tempor
Possible values:
- Any positive integer.
- Any positive integer.
Default value: `60` seconds.
@ -691,7 +691,7 @@ Sets the interval in seconds for ClickHouse to execute the cleanup of old parts,
Possible values:
- Any positive integer.
- Any positive integer.
Default value: `1` second.
@ -701,8 +701,8 @@ Max number of concurrently executed queries related to the MergeTree table. Quer
Possible values:
- Positive integer.
- 0 — No limit.
- Positive integer.
- 0 — No limit.
Default value: `0` (no limit).
@ -718,8 +718,8 @@ The minimal number of marks read by the query for applying the [max_concurrent_q
Possible values:
- Positive integer.
- 0 — Disabled (`max_concurrent_queries` limit applied to no queries).
- Positive integer.
- 0 — Disabled (`max_concurrent_queries` limit applied to no queries).
Default value: `0` (limit never applied).
@ -831,3 +831,13 @@ You can see which parts of `s` were stored using the sparse serialization:
│ s │ Sparse │
└────────┴────────────────────┘
```
## clean_deleted_rows
Enable/disable automatic deletion of rows flagged as `is_deleted` when perform `OPTIMIZE ... FINAL` on a table using the ReplacingMergeTree engine. When disabled, the `CLEANUP` keyword has to be added to the `OPTIMIZE ... FINAL` to have the same behaviour.
Possible values:
- `Always` or `Never`.
Default value: `Never`

View File

@ -21,8 +21,8 @@ Restricts permissions for read data, write data, and change settings queries.
When set to 1, allows:
- All types of read queries (like SELECT and equivalent queries).
- Queries that modify only session context (like USE).
- All types of read queries (like SELECT and equivalent queries).
- Queries that modify only session context (like USE).
When set to 2, allows the above plus:
- SET and CREATE TEMPORARY TABLE
@ -33,9 +33,9 @@ When set to 2, allows the above plus:
Possible values:
- 0 — Read, Write, and Change settings queries are allowed.
- 1 — Only Read data queries are allowed.
- 2 — Read data and Change settings queries are allowed.
- 0 — Read, Write, and Change settings queries are allowed.
- 1 — Only Read data queries are allowed.
- 2 — Read data and Change settings queries are allowed.
Default value: 0
@ -54,8 +54,8 @@ Allows or denies [DDL](https://en.wikipedia.org/wiki/Data_definition_language) q
Possible values:
- 0 — DDL queries are not allowed.
- 1 — DDL queries are allowed.
- 0 — DDL queries are not allowed.
- 1 — DDL queries are allowed.
Default value: 1

View File

@ -26,7 +26,7 @@ It can take one of two values: `throw` or `break`. Restrictions on aggregation (
The maximum amount of RAM to use for running a query on a single server.
In the default configuration file, the maximum is 10 GB.
The default setting is unlimited (set to `0`).
The setting does not consider the volume of available memory or the total volume of memory on the machine.
The restriction applies to a single query within a single server.
@ -101,8 +101,8 @@ Enables or disables execution of `GROUP BY` clauses in external memory. See [GRO
Possible values:
- Maximum volume of RAM (in bytes) that can be used by the single [GROUP BY](../../sql-reference/statements/select/group-by.md#select-group-by-clause) operation.
- 0 — `GROUP BY` in external memory disabled.
- Maximum volume of RAM (in bytes) that can be used by the single [GROUP BY](../../sql-reference/statements/select/group-by.md#select-group-by-clause) operation.
- 0 — `GROUP BY` in external memory disabled.
Default value: 0.
@ -257,8 +257,8 @@ ClickHouse can proceed with different actions when the limit is reached. Use the
Possible values:
- Positive integer.
- 0 — Unlimited number of rows.
- Positive integer.
- 0 — Unlimited number of rows.
Default value: 0.
@ -274,8 +274,8 @@ ClickHouse can proceed with different actions when the limit is reached. Use [jo
Possible values:
- Positive integer.
- 0 — Memory control is disabled.
- Positive integer.
- 0 — Memory control is disabled.
Default value: 0.
@ -283,27 +283,27 @@ Default value: 0.
Defines what action ClickHouse performs when any of the following join limits is reached:
- [max_bytes_in_join](#settings-max_bytes_in_join)
- [max_rows_in_join](#settings-max_rows_in_join)
- [max_bytes_in_join](#settings-max_bytes_in_join)
- [max_rows_in_join](#settings-max_rows_in_join)
Possible values:
- `THROW` — ClickHouse throws an exception and breaks operation.
- `BREAK` — ClickHouse breaks operation and does not throw an exception.
- `THROW` — ClickHouse throws an exception and breaks operation.
- `BREAK` — ClickHouse breaks operation and does not throw an exception.
Default value: `THROW`.
**See Also**
- [JOIN clause](../../sql-reference/statements/select/join.md#select-join)
- [Join table engine](../../engines/table-engines/special/join.md)
- [JOIN clause](../../sql-reference/statements/select/join.md#select-join)
- [Join table engine](../../engines/table-engines/special/join.md)
## max_partitions_per_insert_block {#max-partitions-per-insert-block}
Limits the maximum number of partitions in a single inserted block.
- Positive integer.
- 0 — Unlimited number of partitions.
- Positive integer.
- 0 — Unlimited number of partitions.
Default value: 100.

View File

@ -24,8 +24,8 @@ Supported formats:
Possible values:
- 0 — Disabled.
- 1 — Enabled.
- 0 — Disabled.
- 1 — Enabled.
Default value: 1.
@ -52,8 +52,8 @@ Supported formats:
Possible values:
- 0 — Disabled.
- 1 — Enabled.
- 0 — Disabled.
- 1 — Enabled.
Default value: 1.
@ -72,8 +72,8 @@ Supported formats:
Possible values:
- 0 — Disabled.
- 1 — Enabled.
- 0 — Disabled.
- 1 — Enabled.
Default value: 1.
@ -87,8 +87,8 @@ When this option is enabled, extended table metadata are sent from server to cli
Possible values:
- 0 — Disabled.
- 1 — Enabled.
- 0 — Disabled.
- 1 — Enabled.
Default value: 1.
@ -103,8 +103,8 @@ For complex default expressions `input_format_defaults_for_omitted_fields` must
Possible values:
- 0 — Inserting `NULL` into a not nullable column causes an exception.
- 1 — `NULL` fields are initialized with default column values.
- 0 — Inserting `NULL` into a not nullable column causes an exception.
- 1 — `NULL` fields are initialized with default column values.
Default value: `1`.
@ -179,11 +179,11 @@ The setting does not apply to [date and time functions](../../sql-reference/func
Possible values:
- `'best_effort'` — Enables extended parsing.
- `'best_effort'` — Enables extended parsing.
ClickHouse can parse the basic `YYYY-MM-DD HH:MM:SS` format and all [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) date and time formats. For example, `'2018-06-08T01:02:03.000Z'`.
- `'basic'` — Use basic parser.
- `'basic'` — Use basic parser.
ClickHouse can parse only the basic `YYYY-MM-DD HH:MM:SS` or `YYYY-MM-DD` format. For example, `2019-08-20 10:18:56` or `2019-08-20`.
@ -191,8 +191,8 @@ Default value: `'basic'`.
See also:
- [DateTime data type.](../../sql-reference/data-types/datetime.md)
- [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md)
- [DateTime data type.](../../sql-reference/data-types/datetime.md)
- [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md)
## date_time_output_format {#date_time_output_format}
@ -200,15 +200,15 @@ Allows choosing different output formats of the text representation of date and
Possible values:
- `simple` - Simple output format.
- `simple` - Simple output format.
ClickHouse output date and time `YYYY-MM-DD hh:mm:ss` format. For example, `2019-08-20 10:18:56`. The calculation is performed according to the data type's time zone (if present) or server time zone.
- `iso` - ISO output format.
- `iso` - ISO output format.
ClickHouse output date and time in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) `YYYY-MM-DDThh:mm:ssZ` format. For example, `2019-08-20T10:18:56Z`. Note that output is in UTC (`Z` means UTC).
- `unix_timestamp` - Unix timestamp output format.
- `unix_timestamp` - Unix timestamp output format.
ClickHouse output date and time in [Unix timestamp](https://en.wikipedia.org/wiki/Unix_time) format. For example `1566285536`.
@ -216,8 +216,8 @@ Default value: `simple`.
See also:
- [DateTime data type.](../../sql-reference/data-types/datetime.md)
- [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md)
- [DateTime data type.](../../sql-reference/data-types/datetime.md)
- [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md)
## input_format_ipv4_default_on_conversion_error {#input_format_ipv4_default_on_conversion_error}
@ -298,8 +298,8 @@ By default, when inserting data into a `Distributed` table with more than one sh
Possible values:
- 0 — Insertion is rejected if there are multiple shards and no distributed key is given.
- 1 — Insertion is done randomly among all available shards when no distributed key is given.
- 0 — Insertion is rejected if there are multiple shards and no distributed key is given.
- 1 — Insertion is done randomly among all available shards when no distributed key is given.
Default value: `0`.
@ -311,18 +311,18 @@ Enables or disables the insertion of JSON data with nested objects.
Supported formats:
- [JSONEachRow](../../interfaces/formats.md/#jsoneachrow)
- [JSONEachRow](../../interfaces/formats.md/#jsoneachrow)
Possible values:
- 0 — Disabled.
- 1 — Enabled.
- 0 — Disabled.
- 1 — Enabled.
Default value: 0.
See also:
- [Usage of Nested Structures](../../interfaces/formats.md/#jsoneachrow-nested) with the `JSONEachRow` format.
- [Usage of Nested Structures](../../interfaces/formats.md/#jsoneachrow-nested) with the `JSONEachRow` format.
## input_format_json_read_bools_as_numbers {#input_format_json_read_bools_as_numbers}
@ -373,8 +373,8 @@ Such integers are enclosed in quotes by default. This behavior is compatible wit
Possible values:
- 0 — Integers are output without quotes.
- 1 — Integers are enclosed in quotes.
- 0 — Integers are output without quotes.
- 1 — Integers are enclosed in quotes.
Default value: 1.
@ -390,8 +390,8 @@ Enables `+nan`, `-nan`, `+inf`, `-inf` outputs in [JSON](../../interfaces/format
Possible values:
- 0 — Disabled.
- 1 — Enabled.
- 0 — Disabled.
- 1 — Enabled.
Default value: 0.
@ -526,8 +526,8 @@ Enables the ability to output all rows as a JSON array in the [JSONEachRow](../.
Possible values:
- 1 — ClickHouse outputs all rows as an array, each row in the `JSONEachRow` format.
- 0 — ClickHouse outputs each row separately in the `JSONEachRow` format.
- 1 — ClickHouse outputs all rows as an array, each row in the `JSONEachRow` format.
- 0 — ClickHouse outputs each row separately in the `JSONEachRow` format.
Default value: `0`.
@ -594,8 +594,8 @@ When enabled, always treat enum values as enum ids for TSV input format. It's re
Possible values:
- 0 — Enum values are parsed as values or as enum IDs.
- 1 — Enum values are parsed only as enum IDs.
- 0 — Enum values are parsed as values or as enum IDs.
- 1 — Enum values are parsed only as enum IDs.
Default value: 0.
@ -743,8 +743,8 @@ When enabled, always treat enum values as enum ids for CSV input format. It's re
Possible values:
- 0 — Enum values are parsed as values or as enum IDs.
- 1 — Enum values are parsed only as enum IDs.
- 0 — Enum values are parsed as values or as enum IDs.
- 1 — Enum values are parsed only as enum IDs.
Default value: 0.
@ -873,11 +873,11 @@ Enables or disables the full SQL parser if the fast stream parser cant parse
Possible values:
- 0 — Disabled.
- 0 — Disabled.
In this case, you must provide formatted data. See the [Formats](../../interfaces/formats.md) section.
- 1 — Enabled.
- 1 — Enabled.
In this case, you can use an SQL expression as a value, but data insertion is much slower this way. If you insert only formatted data, then ClickHouse behaves as if the setting value is 0.
@ -923,8 +923,8 @@ Enables or disables template deduction for SQL expressions in [Values](../../int
Possible values:
- 0 — Disabled.
- 1 — Enabled.
- 0 — Disabled.
- 1 — Enabled.
Default value: 1.
@ -934,9 +934,9 @@ For the following query:
INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), (upper('Values')), ...
```
- If `input_format_values_interpret_expressions=1` and `format_values_deduce_templates_of_expressions=0`, expressions are interpreted separately for each row (this is very slow for large number of rows).
- If `input_format_values_interpret_expressions=0` and `format_values_deduce_templates_of_expressions=1`, expressions in the first, second and third rows are parsed using template `lower(String)` and interpreted together, expression in the forth row is parsed with another template (`upper(String)`).
- If `input_format_values_interpret_expressions=1` and `format_values_deduce_templates_of_expressions=1`, the same as in previous case, but also allows fallback to interpreting expressions separately if its not possible to deduce template.
- If `input_format_values_interpret_expressions=1` and `format_values_deduce_templates_of_expressions=0`, expressions are interpreted separately for each row (this is very slow for large number of rows).
- If `input_format_values_interpret_expressions=0` and `format_values_deduce_templates_of_expressions=1`, expressions in the first, second and third rows are parsed using template `lower(String)` and interpreted together, expression in the forth row is parsed with another template (`upper(String)`).
- If `input_format_values_interpret_expressions=1` and `format_values_deduce_templates_of_expressions=1`, the same as in previous case, but also allows fallback to interpreting expressions separately if its not possible to deduce template.
### input_format_values_accurate_types_of_literals {#input_format_values_accurate_types_of_literals}
@ -950,11 +950,11 @@ This setting is used only when `input_format_values_deduce_templates_of_expressi
Possible values:
- 0 — Disabled.
- 0 — Disabled.
In this case, ClickHouse may use a more general type for some literals (e.g., `Float64` or `Int64` instead of `UInt64` for `42`), but it may cause overflow and precision issues.
- 1 — Enabled.
- 1 — Enabled.
In this case, ClickHouse checks the actual type of literal and uses an expression template of the corresponding type. In some cases, it may significantly slow down expression evaluation in `Values`.
@ -968,8 +968,8 @@ Enables or disables the ability to insert the data into [Nested](../../sql-refer
Possible values:
- 0 — Data can not be inserted into `Nested` columns as an array of structs.
- 1 — Data can be inserted into `Nested` columns as an array of structs.
- 0 — Data can not be inserted into `Nested` columns as an array of structs.
- 1 — Data can be inserted into `Nested` columns as an array of structs.
Default value: `0`.
@ -997,8 +997,8 @@ Allows to convert the [LowCardinality](../../sql-reference/data-types/lowcardina
Possible values:
- 0 — The `LowCardinality` type is not converted to the `DICTIONARY` type.
- 1 — The `LowCardinality` type is converted to the `DICTIONARY` type.
- 0 — The `LowCardinality` type is not converted to the `DICTIONARY` type.
- 1 — The `LowCardinality` type is converted to the `DICTIONARY` type.
Default value: `0`.
@ -1028,8 +1028,8 @@ Enables or disables the ability to insert the data into [Nested](../../sql-refer
Possible values:
- 0 — Data can not be inserted into `Nested` columns as an array of structs.
- 1 — Data can be inserted into `Nested` columns as an array of structs.
- 0 — Data can not be inserted into `Nested` columns as an array of structs.
- 1 — Data can be inserted into `Nested` columns as an array of structs.
Default value: `0`.
@ -1077,8 +1077,8 @@ Enables or disables the ability to insert the data into [Nested](../../sql-refer
Possible values:
- 0 — Data can not be inserted into `Nested` columns as an array of structs.
- 1 — Data can be inserted into `Nested` columns as an array of structs.
- 0 — Data can not be inserted into `Nested` columns as an array of structs.
- 1 — Data can be inserted into `Nested` columns as an array of structs.
Default value: `0`.
@ -1192,8 +1192,8 @@ Enables using fields that are not specified in [Avro](../../interfaces/formats.m
Possible values:
- 0 — Disabled.
- 1 — Enabled.
- 0 — Disabled.
- 1 — Enabled.
Default value: 0.
@ -1211,9 +1211,9 @@ Type: string
Possible values:
- `null` — No compression
- `deflate` — Compress with Deflate (zlib)
- `snappy` — Compress with [Snappy](https://google.github.io/snappy/)
- `null` — No compression
- `deflate` — Compress with Deflate (zlib)
- `snappy` — Compress with [Snappy](https://google.github.io/snappy/)
Default value: `snappy` (if available) or `deflate`.
@ -1260,8 +1260,8 @@ Limits the width of value displayed in [Pretty](../../interfaces/formats.md/#pre
Possible values:
- Positive integer.
- 0 — The value is cut completely.
- Positive integer.
- 0 — The value is cut completely.
Default value: `10000` symbols.
@ -1336,8 +1336,8 @@ Adds row numbers to output in the [Pretty](../../interfaces/formats.md/#pretty)
Possible values:
- 0 — Output without row numbers.
- 1 — Output with row numbers.
- 0 — Output without row numbers.
- 1 — Output with row numbers.
Default value: `0`.
@ -1381,12 +1381,12 @@ Sets the field escaping rule for [CustomSeparated](../../interfaces/formats.md/#
Possible values:
- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md/#tabseparated).
- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md/#data-format-values).
- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md/#csv).
- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow).
- `'XML'` — Similarly to [XML](../../interfaces/formats.md/#xml).
- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md/#tabseparatedraw).
- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md/#tabseparated).
- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md/#data-format-values).
- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md/#csv).
- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow).
- `'XML'` — Similarly to [XML](../../interfaces/formats.md/#xml).
- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md/#tabseparatedraw).
Default value: `'Escaped'`.
@ -1434,12 +1434,12 @@ Field escaping rule.
Possible values:
- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md/#tabseparated).
- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md/#data-format-values).
- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md/#csv).
- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow).
- `'XML'` — Similarly to [XML](../../interfaces/formats.md/#xml).
- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md/#tabseparatedraw).
- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md/#tabseparated).
- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md/#data-format-values).
- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md/#csv).
- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow).
- `'XML'` — Similarly to [XML](../../interfaces/formats.md/#xml).
- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md/#tabseparatedraw).
Default value: `Raw`.
@ -1457,9 +1457,9 @@ Determines how to map ClickHouse `Enum` data type and [CapnProto](../../interfac
Possible values:
- `'by_values'` — Values in enums should be the same, names can be different.
- `'by_names'` — Names in enums should be the same, values can be different.
- `'by_name_case_insensitive'` — Names in enums should be the same case-insensitive, values can be different.
- `'by_values'` — Values in enums should be the same, names can be different.
- `'by_names'` — Names in enums should be the same, values can be different.
- `'by_name_case_insensitive'` — Names in enums should be the same case-insensitive, values can be different.
Default value: `'by_values'`.

View File

@ -47,13 +47,13 @@ Structure of the `users` section:
Password can be specified in plaintext or in SHA256 (hex format).
- To assign a password in plaintext (**not recommended**), place it in a `password` element.
- To assign a password in plaintext (**not recommended**), place it in a `password` element.
For example, `<password>qwerty</password>`. The password can be left blank.
<a id="password_sha256_hex"></a>
- To assign a password using its SHA256 hash, place it in a `password_sha256_hex` element.
- To assign a password using its SHA256 hash, place it in a `password_sha256_hex` element.
For example, `<password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>`.
@ -65,7 +65,7 @@ Password can be specified in plaintext or in SHA256 (hex format).
<a id="password_double_sha1_hex"></a>
- For compatibility with MySQL clients, password can be specified in double SHA1 hash. Place it in `password_double_sha1_hex` element.
- For compatibility with MySQL clients, password can be specified in double SHA1 hash. Place it in `password_double_sha1_hex` element.
For example, `<password_double_sha1_hex>08b4a0f1de6ad37da17359e592c8d74788a83eb0</password_double_sha1_hex>`.
@ -81,8 +81,8 @@ This setting enables or disables using of SQL-driven [access control and account
Possible values:
- 0 — Disabled.
- 1 — Enabled.
- 0 — Disabled.
- 1 — Enabled.
Default value: 0.
@ -92,17 +92,17 @@ List of networks from which the user can connect to the ClickHouse server.
Each element of the list can have one of the following forms:
- `<ip>` — IP address or network mask.
- `<ip>` — IP address or network mask.
Examples: `213.180.204.3`, `10.0.0.1/8`, `10.0.0.1/255.255.255.0`, `2a02:6b8::3`, `2a02:6b8::3/64`, `2a02:6b8::3/ffff:ffff:ffff:ffff::`.
- `<host>` — Hostname.
- `<host>` — Hostname.
Example: `example01.host.ru`.
To check access, a DNS query is performed, and all returned IP addresses are compared to the peer address.
- `<host_regexp>` — Regular expression for hostnames.
- `<host_regexp>` — Regular expression for hostnames.
Example, `^example\d\d-\d\d-\d\.host\.ru$`

File diff suppressed because it is too large Load Diff

View File

@ -45,11 +45,11 @@ Configuration markup:
Required parameters:
- `endpoint` — HDFS endpoint URL in `path` format. Endpoint URL should contain a root path to store data.
- `endpoint` — HDFS endpoint URL in `path` format. Endpoint URL should contain a root path to store data.
Optional parameters:
- `min_bytes_for_seek` — The minimal number of bytes to use seek operation instead of sequential read. Default value: `1 Mb`.
- `min_bytes_for_seek` — The minimal number of bytes to use seek operation instead of sequential read. Default value: `1 Mb`.
## Using Virtual File System for Data Encryption {#encrypted-virtual-file-system}
@ -78,16 +78,16 @@ When writing the same file to `disk2`, it will actually be written to the physic
Required parameters:
- `type``encrypted`. Otherwise the encrypted disk is not created.
- `disk` — Type of disk for data storage.
- `key` — The key for encryption and decryption. Type: [Uint64](/docs/en/sql-reference/data-types/int-uint.md). You can use `key_hex` parameter to encode the key in hexadecimal form.
- `type``encrypted`. Otherwise the encrypted disk is not created.
- `disk` — Type of disk for data storage.
- `key` — The key for encryption and decryption. Type: [Uint64](/docs/en/sql-reference/data-types/int-uint.md). You can use `key_hex` parameter to encode the key in hexadecimal form.
You can specify multiple keys using the `id` attribute (see example above).
Optional parameters:
- `path` — Path to the location on the disk where the data will be saved. If not specified, the data will be saved in the root directory.
- `current_key_id` — The key used for encryption. All the specified keys can be used for decryption, and you can always switch to another key while maintaining access to previously encrypted data.
- `algorithm` — [Algorithm](/docs/en/sql-reference/statements/create/table.md/#create-query-encryption-codecs) for encryption. Possible values: `AES_128_CTR`, `AES_192_CTR` or `AES_256_CTR`. Default value: `AES_128_CTR`. The key length depends on the algorithm: `AES_128_CTR` — 16 bytes, `AES_192_CTR` — 24 bytes, `AES_256_CTR` — 32 bytes.
- `path` — Path to the location on the disk where the data will be saved. If not specified, the data will be saved in the root directory.
- `current_key_id` — The key used for encryption. All the specified keys can be used for decryption, and you can always switch to another key while maintaining access to previously encrypted data.
- `algorithm` — [Algorithm](/docs/en/sql-reference/statements/create/table.md/#create-query-encryption-codecs) for encryption. Possible values: `AES_128_CTR`, `AES_192_CTR` or `AES_256_CTR`. Default value: `AES_128_CTR`. The key length depends on the algorithm: `AES_128_CTR` — 16 bytes, `AES_192_CTR` — 24 bytes, `AES_256_CTR` — 32 bytes.
Example of disk configuration:
@ -446,14 +446,14 @@ SETTINGS storage_policy='web';
Required parameters:
- `type``web`. Otherwise the disk is not created.
- `endpoint` — The endpoint URL in `path` format. Endpoint URL must contain a root path to store data, where they were uploaded.
- `type``web`. Otherwise the disk is not created.
- `endpoint` — The endpoint URL in `path` format. Endpoint URL must contain a root path to store data, where they were uploaded.
Optional parameters:
- `min_bytes_for_seek` — The minimal number of bytes to use seek operation instead of sequential read. Default value: `1` Mb.
- `remote_fs_read_backoff_threashold` — The maximum wait time when trying to read data for remote disk. Default value: `10000` seconds.
- `remote_fs_read_backoff_max_tries` — The maximum number of attempts to read with backoff. Default value: `5`.
- `min_bytes_for_seek` — The minimal number of bytes to use seek operation instead of sequential read. Default value: `1` Mb.
- `remote_fs_read_backoff_threashold` — The maximum wait time when trying to read data for remote disk. Default value: `10000` seconds.
- `remote_fs_read_backoff_max_tries` — The maximum number of attempts to read with backoff. Default value: `5`.
If a query fails with an exception `DB:Exception Unreachable URL`, then you can try to adjust the settings: [http_connection_timeout](/docs/en/operations/settings/settings.md/#http_connection_timeout), [http_receive_timeout](/docs/en/operations/settings/settings.md/#http_receive_timeout), [keep_alive_timeout](/docs/en/operations/server-configuration-parameters/settings.md/#keep-alive-timeout).

View File

@ -7,11 +7,11 @@ Contains the historical values for `system.asynchronous_metrics`, which are save
Columns:
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time.
- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Event time with microseconds resolution.
- `name` ([String](../../sql-reference/data-types/string.md)) — Metric name.
- `value` ([Float64](../../sql-reference/data-types/float.md)) — Metric value.
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time.
- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Event time with microseconds resolution.
- `name` ([String](../../sql-reference/data-types/string.md)) — Metric name.
- `value` ([Float64](../../sql-reference/data-types/float.md)) — Metric value.
**Example**

View File

@ -7,9 +7,9 @@ Contains metrics that are calculated periodically in the background. For example
Columns:
- `metric` ([String](../../sql-reference/data-types/string.md)) — Metric name.
- `value` ([Float64](../../sql-reference/data-types/float.md)) — Metric value.
- `description` ([String](../../sql-reference/data-types/string.md) - Metric description)
- `metric` ([String](../../sql-reference/data-types/string.md)) — Metric name.
- `value` ([Float64](../../sql-reference/data-types/float.md)) — Metric value.
- `description` ([String](../../sql-reference/data-types/string.md) - Metric description)
**Example**
@ -34,7 +34,7 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10
**See Also**
- [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring.
- [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics.
- [system.events](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that have occurred.
- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` and `system.events`.
- [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring.
- [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics.
- [system.events](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that have occurred.
- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` and `system.events`.

View File

@ -7,19 +7,19 @@ Contains information about clusters available in the config file and the servers
Columns:
- `cluster` ([String](../../sql-reference/data-types/string.md)) — The cluster name.
- `shard_num` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The shard number in the cluster, starting from 1.
- `shard_weight` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The relative weight of the shard when writing data.
- `replica_num` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The replica number in the shard, starting from 1.
- `host_name` ([String](../../sql-reference/data-types/string.md)) — The host name, as specified in the config.
- `host_address` ([String](../../sql-reference/data-types/string.md)) — The host IP address obtained from DNS.
- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — The port to use for connecting to the server.
- `is_local` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the host is local.
- `user` ([String](../../sql-reference/data-types/string.md)) — The name of the user for connecting to the server.
- `default_database` ([String](../../sql-reference/data-types/string.md)) — The default database name.
- `errors_count` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of times this host failed to reach replica.
- `slowdowns_count` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of slowdowns that led to changing replica when establishing a connection with hedged requests.
- `estimated_recovery_time` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Seconds remaining until the replica error count is zeroed and it is considered to be back to normal.
- `cluster` ([String](../../sql-reference/data-types/string.md)) — The cluster name.
- `shard_num` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The shard number in the cluster, starting from 1.
- `shard_weight` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The relative weight of the shard when writing data.
- `replica_num` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The replica number in the shard, starting from 1.
- `host_name` ([String](../../sql-reference/data-types/string.md)) — The host name, as specified in the config.
- `host_address` ([String](../../sql-reference/data-types/string.md)) — The host IP address obtained from DNS.
- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — The port to use for connecting to the server.
- `is_local` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the host is local.
- `user` ([String](../../sql-reference/data-types/string.md)) — The name of the user for connecting to the server.
- `default_database` ([String](../../sql-reference/data-types/string.md)) — The default database name.
- `errors_count` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of times this host failed to reach replica.
- `slowdowns_count` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of slowdowns that led to changing replica when establishing a connection with hedged requests.
- `estimated_recovery_time` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Seconds remaining until the replica error count is zeroed and it is considered to be back to normal.
**Example**
@ -67,6 +67,6 @@ estimated_recovery_time: 0
**See Also**
- [Table engine Distributed](../../engines/table-engines/special/distributed.md)
- [distributed_replica_error_cap setting](../../operations/settings/settings.md#settings-distributed_replica_error_cap)
- [distributed_replica_error_half_life setting](../../operations/settings/settings.md#settings-distributed_replica_error_half_life)
- [Table engine Distributed](../../engines/table-engines/special/distributed.md)
- [distributed_replica_error_cap setting](../../operations/settings/settings.md#settings-distributed_replica_error_cap)
- [distributed_replica_error_half_life setting](../../operations/settings/settings.md#settings-distributed_replica_error_half_life)

View File

@ -11,27 +11,27 @@ Columns from [temporary tables](../../sql-reference/statements/create/table.md#t
The `system.columns` table contains the following columns (the column type is shown in brackets):
- `database` ([String](../../sql-reference/data-types/string.md)) — Database name.
- `table` ([String](../../sql-reference/data-types/string.md)) — Table name.
- `name` ([String](../../sql-reference/data-types/string.md)) — Column name.
- `type` ([String](../../sql-reference/data-types/string.md)) — Column type.
- `position` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Ordinal position of a column in a table starting with 1.
- `default_kind` ([String](../../sql-reference/data-types/string.md)) — Expression type (`DEFAULT`, `MATERIALIZED`, `ALIAS`) for the default value, or an empty string if it is not defined.
- `default_expression` ([String](../../sql-reference/data-types/string.md)) — Expression for the default value, or an empty string if it is not defined.
- `data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of compressed data, in bytes.
- `data_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of decompressed data, in bytes.
- `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of marks, in bytes.
- `comment` ([String](../../sql-reference/data-types/string.md)) — Comment on the column, or an empty string if it is not defined.
- `is_in_partition_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the partition expression.
- `is_in_sorting_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sorting key expression.
- `is_in_primary_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the primary key expression.
- `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sampling key expression.
- `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name.
- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned.
- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned.
- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned.
- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned.
- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Decimal precision of `DateTime64` data type. For other data types, the `NULL` value is returned.
- `database` ([String](../../sql-reference/data-types/string.md)) — Database name.
- `table` ([String](../../sql-reference/data-types/string.md)) — Table name.
- `name` ([String](../../sql-reference/data-types/string.md)) — Column name.
- `type` ([String](../../sql-reference/data-types/string.md)) — Column type.
- `position` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Ordinal position of a column in a table starting with 1.
- `default_kind` ([String](../../sql-reference/data-types/string.md)) — Expression type (`DEFAULT`, `MATERIALIZED`, `ALIAS`) for the default value, or an empty string if it is not defined.
- `default_expression` ([String](../../sql-reference/data-types/string.md)) — Expression for the default value, or an empty string if it is not defined.
- `data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of compressed data, in bytes.
- `data_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of decompressed data, in bytes.
- `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of marks, in bytes.
- `comment` ([String](../../sql-reference/data-types/string.md)) — Comment on the column, or an empty string if it is not defined.
- `is_in_partition_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the partition expression.
- `is_in_sorting_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sorting key expression.
- `is_in_primary_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the primary key expression.
- `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sampling key expression.
- `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name.
- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned.
- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned.
- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned.
- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned.
- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Decimal precision of `DateTime64` data type. For other data types, the `NULL` value is returned.
**Example**

View File

@ -7,7 +7,7 @@ Contains information about contributors. The order is random at query execution
Columns:
- `name` (String) — Contributor (author) name from git log.
- `name` (String) — Contributor (author) name from git log.
**Example**

View File

@ -7,17 +7,17 @@ Contains information about stack traces for fatal errors. The table does not exi
Columns:
- `event_date` ([DateTime](../../sql-reference/data-types/datetime.md)) — Date of the event.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time of the event.
- `timestamp_ns` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Timestamp of the event with nanoseconds.
- `signal` ([Int32](../../sql-reference/data-types/int-uint.md)) — Signal number.
- `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Thread ID.
- `query_id` ([String](../../sql-reference/data-types/string.md)) — Query ID.
- `trace` ([Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Stack trace at the moment of crash. Each element is a virtual memory address inside ClickHouse server process.
- `trace_full` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Stack trace at the moment of crash. Each element contains a called method inside ClickHouse server process.
- `version` ([String](../../sql-reference/data-types/string.md)) — ClickHouse server version.
- `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse server revision.
- `build_id` ([String](../../sql-reference/data-types/string.md)) — BuildID that is generated by compiler.
- `event_date` ([DateTime](../../sql-reference/data-types/datetime.md)) — Date of the event.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time of the event.
- `timestamp_ns` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Timestamp of the event with nanoseconds.
- `signal` ([Int32](../../sql-reference/data-types/int-uint.md)) — Signal number.
- `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Thread ID.
- `query_id` ([String](../../sql-reference/data-types/string.md)) — Query ID.
- `trace` ([Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Stack trace at the moment of crash. Each element is a virtual memory address inside ClickHouse server process.
- `trace_full` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Stack trace at the moment of crash. Each element contains a called method inside ClickHouse server process.
- `version` ([String](../../sql-reference/data-types/string.md)) — ClickHouse server version.
- `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse server revision.
- `build_id` ([String](../../sql-reference/data-types/string.md)) — BuildID that is generated by compiler.
**Example**
@ -46,6 +46,6 @@ build_id:
```
**See also**
- [trace_log](../../operations/system-tables/trace_log.md) system table
- [trace_log](../../operations/system-tables/trace_log.md) system table
[Original article](https://clickhouse.com/docs/en/operations/system-tables/crash-log)

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