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

@ -33,6 +33,8 @@ 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).

View File

@ -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";

View File

@ -483,7 +483,7 @@ 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.

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.

View File

@ -38,6 +38,7 @@ The supported formats are:
| [JSONCompactStrings](#jsoncompactstrings) | ✗ | ✔ |
| [JSONCompactColumns](#jsoncompactcolumns) | ✔ | ✔ |
| [JSONEachRow](#jsoneachrow) | ✔ | ✔ |
| [PrettyJSONEachRow](#prettyjsoneachrow) | ✗ | ✔ |
| [JSONEachRowWithProgress](#jsoneachrowwithprogress) | ✗ | ✔ |
| [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ |
| [JSONStringsEachRowWithProgress](#jsonstringseachrowwithprogress) | ✗ | ✔ |
@ -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.

View File

@ -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>

View File

@ -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

@ -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.

View File

@ -5,7 +5,7 @@ slug: /en/operations/system-tables/mutations
The table contains information about [mutations](/docs/en/sql-reference/statements/alter/index.md#mutations) of [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) tables and their progress. Each mutation command is represented by a single row.
Columns:
## Columns:
- `database` ([String](/docs/en/sql-reference/data-types/string.md)) — The name of the database to which the mutation was applied.
@ -43,6 +43,19 @@ If there were problems with mutating some data parts, the following columns cont
- `latest_fail_reason` ([String](/docs/en/sql-reference/data-types/string.md)) — The exception message that caused the most recent part mutation failure.
## Monitoring Mutations
To track the progress on the system.mutations table, use a query like the following - this requires read permissions on the system.* tables:
``` sql
SELECT * FROM clusterAllReplicas('cluster_name', 'db', system.mutations)
WHERE is_done=0 AND table='tmp';
```
:::tip
replace `tmp` in `table='tmp'` with the name of the table that you are checking mutations on.
:::
**See Also**
- [Mutations](/docs/en/sql-reference/statements/alter/index.md#mutations)

View File

@ -1,12 +1,14 @@
---
slug: /en/sql-reference/functions/arithmetic-functions
sidebar_position: 34
sidebar_position: 5
sidebar_label: Arithmetic
---
# Arithmetic Functions
For all arithmetic functions, the result type is calculated as the smallest number type that the result fits in, if there is such a type. The minimum is taken simultaneously based on the number of bits, whether it is signed, and whether it floats. If there are not enough bits, the highest bit type is taken.
The result type of all arithmetic functions is the smallest type which can represent all possible results. Size promotion happens for integers up to 32 bit, e.g. `UInt8 + UInt16 = UInt32`. If one of the inters has 64 or more bits, the result is of the same type as the bigger of the input integers, e.g. `UInt16 + UInt128 = UInt128`. While this introduces a risk of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of 64 bit.
The result of addition or multiplication of two integers is unsigned unless one of the integers is signed.
Example:
@ -20,39 +22,78 @@ SELECT toTypeName(0), toTypeName(0 + 0), toTypeName(0 + 0 + 0), toTypeName(0 + 0
└───────────────┴────────────────────────┴─────────────────────────────────┴──────────────────────────────────────────┘
```
Arithmetic functions work for any pair of types from UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, or Float64.
Arithmetic functions work for any pair of `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`, `Float32`, or `Float64` values.
Overflow is produced the same way as in C++.
Overflows are produced the same way as in C++.
## plus(a, b), a + b operator
## plus
Calculates the sum of the numbers.
You can also add integer numbers with a date or date and time. In the case of a date, adding an integer means adding the corresponding number of days. For a date with time, it means adding the corresponding number of seconds.
Calculates the sum of two values `a` and `b`.
## minus(a, b), a - b operator
**Syntax**
Calculates the difference. The result is always signed.
```sql
plus(a, b)
```
You can also calculate integer numbers from a date or date with time. The idea is the same see above for plus.
It is possible to add an integer and a date or date with time. The former operation increments the number of days in the date, the latter operation increments the number of seconds in the date with time.
## multiply(a, b), a \* b operator
Alias: `a + b` (operator)
Calculates the product of the numbers.
## minus
## divide(a, b), a / b operator
Calculates the difference of two values `a` and `b`. The result is always signed.
Calculates the quotient of the numbers. The result type is always a floating-point type.
It is not integer division. For integer division, use the intDiv function.
When dividing by zero you get inf, -inf, or nan.
Similar to `plus`, it is possible to subtract an integer from a date or date with time.
## intDiv(a, b)
**Syntax**
Calculates the quotient of the numbers. Divides into integers, rounding down (by the absolute value).
```sql
minus(a, b)
```
Returns an integer of the type of the dividend (the first parameter).
Alias: `a - b` (operator)
## multiply
Calculates the product of two values `a` and `b`.
**Syntax**
```sql
multiply(a, b)
```
Alias: `a \* b` (operator)
## divide
Calculates the quotient of two values `a` and `b`. The result is always a floating-point value. If you need integer division, you can use the `intDiv` function.
Division by 0 returns `inf`, `-inf`, or `nan`.
**Syntax**
```sql
divide(a, b)
```
Alias: `a / b` (operator)
## intDiv
Performs an integer division of two values `a` by `b`, i.e. computes the quotient rounded down to the next smallest integer.
The result has the same type as the dividend (the first parameter).
An exception is thrown when dividing by zero, when the quotient does not fit in the range of the dividend, or when dividing a minimal negative number by minus one.
**Syntax**
```sql
intDiv(a, b)
```
**Example**
Query:
@ -62,6 +103,7 @@ SELECT
intDiv(toFloat64(1), 0.001) AS res,
toTypeName(res)
```
```response
┌──res─┬─toTypeName(intDiv(toFloat64(1), 0.001))─┐
│ 1000 │ Int64 │
@ -73,30 +115,65 @@ SELECT
intDiv(1, 0.001) AS res,
toTypeName(res)
```
```response
Received exception from server (version 23.2.1):
Code: 153. DB::Exception: Received from localhost:9000. DB::Exception: Cannot perform integer division, because it will produce infinite or too large number: While processing intDiv(1, 0.001) AS res, toTypeName(res). (ILLEGAL_DIVISION)
```
## intDivOrZero(a, b)
## intDivOrZero
Differs from intDiv in that it returns zero when dividing by zero or when dividing a minimal negative number by minus one.
Same as `intDiv` but returns zero when dividing by zero or when dividing a minimal negative number by minus one.
## modulo(a, b), a % b operator
**Syntax**
```sql
intDivOrZero(a, b)
```
## modulo
Calculates the remainder of the division of two values `a` by `b`.
Calculates the remainder when dividing `a` by `b`.
The result type is an integer if both inputs are integers. If one of the inputs is a floating-point number, the result is a floating-point number.
The remainder is computed like in C++. Truncated division is used for negative numbers.
An exception is thrown when dividing by zero or when dividing a minimal negative number by minus one.
## moduloOrZero(a, b)
**Syntax**
Differs from [modulo](#modulo) in that it returns zero when the divisor is zero.
```sql
modulo(a, b)
```
## positiveModulo(a, b), positive_modulo(a, b), pmod(a, b)
Calculates the remainder when dividing `a` by `b`. Similar to the function `modulo` except that `positive_modulo` always returns a non-negative number.
Alias: `a % b` (operator)
Notice that `positive_modulo` is 4-5 times slower than `modulo`. You should not use `positive_modulo` unless you want to get a positive result and don't care about performance too much.
## moduloOrZero
Like [modulo](#modulo) but returns zero when the divisor is zero.
**Syntax**
```sql
moduloOrZero(a, b)
```
## positiveModulo(a, b)
Like [modulo](#modulo) but always returns a non-negative number.
This function is 4-5 times slower than `modulo`.
**Syntax**
```sql
positiveModulo(a, b)
```
Alias:
- `positive_modulo(a, b)`
- `pmod(a, b)`
**Example**
@ -108,51 +185,67 @@ SELECT positiveModulo(-1, 10)
Result:
```text
```result
┌─positiveModulo(-1, 10)─┐
│ 9 │
└────────────────────────┘
```
## negate(a), -a operator
## negate
Calculates a number with the reverse sign. The result is always signed.
## abs(a)
Calculates the absolute value of the number (a). That is, if a \< 0, it returns -a. For unsigned types it does not do anything. For signed integer types, it returns an unsigned number.
## gcd(a, b)
Returns the greatest common divisor of the numbers.
An exception is thrown when dividing by zero or when dividing a minimal negative number by minus one.
## lcm(a, b)
Returns the least common multiple of the numbers.
An exception is thrown when dividing by zero or when dividing a minimal negative number by minus one.
## max2
Compares two values and returns the maximum. The returned value is converted to [Float64](../../sql-reference/data-types/float.md).
Negates a value `a`. The result is always signed.
**Syntax**
```sql
max2(value1, value2)
negate(a)
```
**Arguments**
Alias: `-a`
- `value1` — First value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md).
- `value2` — Second value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md).
## abs
**Returned value**
Calculates the absolute value of `a`. Has no effect if `a` is of an unsigned type. If `a` is of a signed type, it returns an unsigned number.
- The maximum of two values.
**Syntax**
Type: [Float](../../sql-reference/data-types/float.md).
```sql
abs(a)
```
## gcd
Returns the greatest common divisor of two values `a` and `b`.
An exception is thrown when dividing by zero or when dividing a minimal negative number by minus one.
**Syntax**
```sql
gcd(a, b)
```
## lcm(a, b)
Returns the least common multiple of two values `a` and `b`.
An exception is thrown when dividing by zero or when dividing a minimal negative number by minus one.
**Syntax**
```sql
lcm(a, b)
```
## max2
Returns the bigger of two values `a` and `b`. The returned value is of type [Float64](../../sql-reference/data-types/float.md).
**Syntax**
```sql
max2(a, b)
```
**Example**
@ -164,7 +257,7 @@ SELECT max2(-1, 2);
Result:
```text
```result
┌─max2(-1, 2)─┐
│ 2 │
└─────────────┘
@ -172,25 +265,14 @@ Result:
## min2
Compares two values and returns the minimum. The returned value is converted to [Float64](../../sql-reference/data-types/float.md).
Returns the smaller of two values `a` and `b`. The returned value is of type [Float64](../../sql-reference/data-types/float.md).
**Syntax**
```sql
min2(value1, value2)
min2(a, b)
```
**Arguments**
- `value1` — First value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md).
- `value2` — Second value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md).
**Returned value**
- The minimum of two values.
Type: [Float](../../sql-reference/data-types/float.md).
**Example**
Query:
@ -201,21 +283,19 @@ SELECT min2(-1, 2);
Result:
```text
```result
┌─min2(-1, 2)─┐
│ -1 │
└─────────────┘
```
## multiplyDecimal(a, b[, result_scale])
## multiplyDecimal
Performs multiplication on two decimals. Result value will be of type [Decimal256](../../sql-reference/data-types/decimal.md).
Result scale can be explicitly specified by `result_scale` argument (const Integer in range `[0, 76]`). If not specified, the result scale is the max scale of given arguments.
Multiplies two decimals `a` and `b`. The result value will be of type [Decimal256](../../sql-reference/data-types/decimal.md).
:::note
These functions work significantly slower than usual `multiply`.
In case you don't really need controlled precision and/or need fast computation, consider using [multiply](#multiply)
:::
The scale of the result can be explicitly specified by `result_scale`. If `result_scale` is not specified, it is assumed to be the maximum scale of the input values.
This function work significantly slower than usual `multiply`. In case no control over the result precision is needed and/or fast computation is desired, consider using `multiply`.
**Syntax**
@ -237,19 +317,22 @@ Type: [Decimal256](../../sql-reference/data-types/decimal.md).
**Example**
```text
```result
┌─multiplyDecimal(toDecimal256(-12, 0), toDecimal32(-2.1, 1), 1)─┐
│ 25.2 │
└────────────────────────────────────────────────────────────────┘
```
**Difference from regular multiplication:**
**Differences compared to regular multiplication:**
```sql
SELECT toDecimal64(-12.647, 3) * toDecimal32(2.1239, 4);
SELECT toDecimal64(-12.647, 3) as a, toDecimal32(2.1239, 4) as b, multiplyDecimal(a, b);
```
```text
Result:
```result
┌─multiply(toDecimal64(-12.647, 3), toDecimal32(2.1239, 4))─┐
│ -26.8609633 │
└───────────────────────────────────────────────────────────┘
@ -270,7 +353,9 @@ SELECT
a * b;
```
```text
Result:
```result
┌─────────────a─┬─────────────b─┬─multiplyDecimal(toDecimal64(-12.647987876, 9), toDecimal64(123.967645643, 9))─┐
│ -12.647987876 │ 123.967645643 │ -1567.941279108 │
└───────────────┴───────────────┴───────────────────────────────────────────────────────────────────────────────┘
@ -279,15 +364,14 @@ Received exception from server (version 22.11.1):
Code: 407. DB::Exception: Received from localhost:9000. DB::Exception: Decimal math overflow: While processing toDecimal64(-12.647987876, 9) AS a, toDecimal64(123.967645643, 9) AS b, a * b. (DECIMAL_OVERFLOW)
```
## divideDecimal(a, b[, result_scale])
## divideDecimal
Performs division on two decimals. Result value will be of type [Decimal256](../../sql-reference/data-types/decimal.md).
Result scale can be explicitly specified by `result_scale` argument (const Integer in range `[0, 76]`). If not specified, the result scale is the max scale of given arguments.
:::note
These function work significantly slower than usual `divide`.
In case you don't really need controlled precision and/or need fast computation, consider using [divide](#divide).
:::
Divides two decimals `a` and `b`. The result value will be of type [Decimal256](../../sql-reference/data-types/decimal.md).
The scale of the result can be explicitly specified by `result_scale`. If `result_scale` is not specified, it is assumed to be the maximum scale of the input values.
This function work significantly slower than usual `divide`. In case no control over the result precision is needed and/or fast computation is desired, consider using `divide`.
**Syntax**
@ -309,19 +393,22 @@ Type: [Decimal256](../../sql-reference/data-types/decimal.md).
**Example**
```text
```result
┌─divideDecimal(toDecimal256(-12, 0), toDecimal32(2.1, 1), 10)─┐
│ -5.7142857142 │
└──────────────────────────────────────────────────────────────┘
```
**Difference from regular division:**
**Differences compared to regular division:**
```sql
SELECT toDecimal64(-12, 1) / toDecimal32(2.1, 1);
SELECT toDecimal64(-12, 1) as a, toDecimal32(2.1, 1) as b, divideDecimal(a, b, 1), divideDecimal(a, b, 5);
```
```text
Result:
```result
┌─divide(toDecimal64(-12, 1), toDecimal32(2.1, 1))─┐
│ -5.7 │
└──────────────────────────────────────────────────┘
@ -336,7 +423,9 @@ SELECT toDecimal64(-12, 0) / toDecimal32(2.1, 1);
SELECT toDecimal64(-12, 0) as a, toDecimal32(2.1, 1) as b, divideDecimal(a, b, 1), divideDecimal(a, b, 5);
```
```text
Result:
```result
DB::Exception: Decimal result's scale is less than argument's one: While processing toDecimal64(-12, 0) / toDecimal32(2.1, 1). (ARGUMENT_OUT_OF_BOUND)
┌───a─┬───b─┬─divideDecimal(toDecimal64(-12, 0), toDecimal32(2.1, 1), 1)─┬─divideDecimal(toDecimal64(-12, 0), toDecimal32(2.1, 1), 5)─┐

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/array-functions
sidebar_position: 35
sidebar_position: 10
sidebar_label: Arrays
---
@ -118,12 +118,12 @@ Accepts zero arguments and returns an empty array of the appropriate type.
Accepts an empty array and returns a one-element array that is equal to the default value.
## range(end), range(\[start, \] end \[, step\])
Returns an array of numbers from `start` to `end - 1` by `step`. The supported types are [UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64](../data-types/int-uint.md).
**Syntax**
``` sql
range([start, ] end [, step])
```
@ -146,10 +146,13 @@ range([start, ] end [, step])
**Examples**
Query:
``` sql
SELECT range(5), range(1, 5), range(1, 5, 2), range(-1, 5, 2);
```
Result:
```txt
┌─range(5)────┬─range(1, 5)─┬─range(1, 5, 2)─┬─range(-1, 5, 2)─┐
│ [0,1,2,3,4] │ [1,2,3,4] │ [1,3] │ [-1,1,3] │
@ -301,6 +304,7 @@ In other words, the functions will check whether all the elements of `array2` ar
the `hasAll` function. In addition, it will check that the elements are observed in the same order in both `array1` and `array2`.
For Example:
- `hasSubstr([1,2,3,4], [2,3])` returns 1. However, `hasSubstr([1,2,3,4], [3,2])` will return `0`.
- `hasSubstr([1,2,3,4], [1,2,3])` returns 1. However, `hasSubstr([1,2,3,4], [1,2,4])` will return `0`.
@ -857,7 +861,7 @@ A special function. See the section [“ArrayJoin function”](../../sql-referen
## arrayDifference
Calculates the difference between adjacent array elements. Returns an array where the first element will be 0, the second is the difference between `a[1] - a[0]`, etc. The type of elements in the resulting array is determined by the type inference rules for subtraction (e.g. `UInt8` - `UInt8` = `Int16`).
Calculates an array of differences between adjacent array elements. The first element of the result array will be 0, the second `a[1] - a[0]`, the third `a[2] - a[1]`, etc. The type of elements in the result array is determined by the type inference rules for subtraction (e.g. `UInt8` - `UInt8` = `Int16`).
**Syntax**
@ -871,7 +875,7 @@ arrayDifference(array)
**Returned values**
Returns an array of differences between adjacent elements.
Returns an array of differences between adjacent array elements.
Type: [UInt\*](https://clickhouse.com/docs/en/data_types/int_uint/#uint-ranges), [Int\*](https://clickhouse.com/docs/en/data_types/int_uint/#int-ranges), [Float\*](https://clickhouse.com/docs/en/data_types/float/).
@ -1213,7 +1217,7 @@ Result:
## arrayAUC
Calculate AUC (Area Under the Curve, which is a concept in machine learning, see more details: https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve).
Calculate AUC (Area Under the Curve, which is a concept in machine learning, see more details: <https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve>).
**Syntax**
@ -1639,7 +1643,23 @@ Result:
## arrayCumSum(\[func,\] arr1, …)
Returns an array of partial sums of elements in the source array (a running sum). If the `func` function is specified, then the values of the array elements are converted by `func(arr1[i], …, arrN[i])` before summing.
Returns an array of the partial (running) sums of the elements in the source array `arr1`. If `func` is specified, then the sum is computed from applying `func` to `arr1`, `arr2`, ..., `arrN`, i.e. `func(arr1[i], …, arrN[i])`.
**Syntax**
``` sql
arrayCumSum(arr)
```
**Arguments**
- `arr` — [Array](../../sql-reference/data-types/array.md) of numeric values.
**Returned value**
- Returns an array of the partial sums of the elements in the source array.
Type: [UInt\*](https://clickhouse.com/docs/en/data_types/int_uint/#uint-ranges), [Int\*](https://clickhouse.com/docs/en/data_types/int_uint/#int-ranges), [Float\*](https://clickhouse.com/docs/en/data_types/float/).
Example:
@ -1655,9 +1675,25 @@ SELECT arrayCumSum([1, 1, 1, 1]) AS res
Note that the `arrayCumSum` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument.
## arrayCumSumNonNegative(arr)
## arrayCumSumNonNegative(\[func,\] arr1, …)
Same as `arrayCumSum`, returns an array of partial sums of elements in the source array (a running sum). Different `arrayCumSum`, when then returned value contains a value less than zero, the value is replace with zero and the subsequent calculation is performed with zero parameters. For example:
Same as `arrayCumSum`, returns an array of the partial (running) sums of the elements in the source array. If `func` is specified, then the sum is computed from applying `func` to `arr1`, `arr2`, ..., `arrN`, i.e. `func(arr1[i], …, arrN[i])`. Unlike `arrayCumSum`, if the current running sum is smaller than `0`, it is replaced by `0`.
**Syntax**
``` sql
arrayCumSumNonNegative(arr)
```
**Arguments**
- `arr` — [Array](../../sql-reference/data-types/array.md) of numeric values.
**Returned value**
- Returns an array of non-negative partial sums of elements in the source array.
Type: [UInt\*](https://clickhouse.com/docs/en/data_types/int_uint/#uint-ranges), [Int\*](https://clickhouse.com/docs/en/data_types/int_uint/#int-ranges), [Float\*](https://clickhouse.com/docs/en/data_types/float/).
``` sql
SELECT arrayCumSumNonNegative([1, 1, -4, 1]) AS res
@ -1668,6 +1704,7 @@ SELECT arrayCumSumNonNegative([1, 1, -4, 1]) AS res
│ [1,2,0,1] │
└───────────┘
```
Note that the `arraySumNonNegative` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument.
## arrayProduct

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/array-join
sidebar_position: 61
sidebar_position: 15
sidebar_label: arrayJoin
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/bit-functions
sidebar_position: 48
sidebar_position: 20
sidebar_label: Bit
---

View File

@ -1,22 +1,18 @@
---
slug: /en/sql-reference/functions/bitmap-functions
sidebar_position: 49
sidebar_position: 25
sidebar_label: Bitmap
---
# Bitmap Functions
Bitmap functions work for two bitmaps Object value calculation, it is to return new bitmap or cardinality while using formula calculation, such as and, or, xor, and not, etc.
There are 2 kinds of construction methods for Bitmap Object. One is to be constructed by aggregation function groupBitmap with -State, the other is to be constructed by Array Object. It is also to convert Bitmap Object to Array Object.
RoaringBitmap is wrapped into a data structure while actual storage of Bitmap objects. When the cardinality is less than or equal to 32, it uses Set objet. When the cardinality is greater than 32, it uses RoaringBitmap object. That is why storage of low cardinality set is faster.
For more information on RoaringBitmap, see: [CRoaring](https://github.com/RoaringBitmap/CRoaring).
Bitmaps can be constructed in two ways. The first way is constructed by aggregation function groupBitmap with `-State`, the other way is to constructed a bitmap from an Array object.
## bitmapBuild
Build a bitmap from unsigned integer array.
Builds a bitmap from an unsigned integer array.
**Syntax**
``` sql
bitmapBuild(array)
@ -40,7 +36,9 @@ SELECT bitmapBuild([1, 2, 3, 4, 5]) AS res, toTypeName(res);
## bitmapToArray
Convert bitmap to integer array.
Converts bitmap to an integer array.
**Syntax**
``` sql
bitmapToArray(bitmap)
@ -56,6 +54,8 @@ bitmapToArray(bitmap)
SELECT bitmapToArray(bitmapBuild([1, 2, 3, 4, 5])) AS res;
```
Result:
``` text
┌─res─────────┐
│ [1,2,3,4,5] │
@ -64,7 +64,9 @@ SELECT bitmapToArray(bitmapBuild([1, 2, 3, 4, 5])) AS res;
## bitmapSubsetInRange
Return subset in specified range (not include the range_end).
Returns the subset of a bitmap with bits within a value interval.
**Syntax**
``` sql
bitmapSubsetInRange(bitmap, range_start, range_end)
@ -73,8 +75,8 @@ bitmapSubsetInRange(bitmap, range_start, range_end)
**Arguments**
- `bitmap` [Bitmap object](#bitmap_functions-bitmapbuild).
- `range_start` Range start point. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
- `range_end` Range end point (excluded). Type: [UInt32](../../sql-reference/data-types/int-uint.md).
- `range_start` Start of the range (inclusive). Type: [UInt32](../../sql-reference/data-types/int-uint.md).
- `range_end` End of the range (exclusive). Type: [UInt32](../../sql-reference/data-types/int-uint.md).
**Example**
@ -82,6 +84,8 @@ bitmapSubsetInRange(bitmap, range_start, range_end)
SELECT bitmapToArray(bitmapSubsetInRange(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(30), toUInt32(200))) AS res;
```
Result:
``` text
┌─res───────────────┐
│ [30,31,32,33,100] │
@ -90,7 +94,7 @@ SELECT bitmapToArray(bitmapSubsetInRange(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,
## bitmapSubsetLimit
Creates a subset of bitmap with n elements taken between `range_start` and `cardinality_limit`.
Returns a subset of a bitmap with smallest bit value `range_start` and at most `cardinality_limit` elements.
**Syntax**
@ -101,19 +105,11 @@ bitmapSubsetLimit(bitmap, range_start, cardinality_limit)
**Arguments**
- `bitmap` [Bitmap object](#bitmap_functions-bitmapbuild).
- `range_start` The subset starting point. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
- `cardinality_limit` The subset cardinality upper limit. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
**Returned value**
The subset.
Type: [Bitmap object](#bitmap_functions-bitmapbuild).
- `range_start` Start of the range (inclusive). Type: [UInt32](../../sql-reference/data-types/int-uint.md).
- `cardinality_limit` Maximum cardinality of the subset. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT bitmapToArray(bitmapSubsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(30), toUInt32(200))) AS res;
```
@ -128,7 +124,7 @@ Result:
## subBitmap
Returns the bitmap elements, starting from the `offset` position. The number of returned elements is limited by the `cardinality_limit` parameter. Analog of the [substring](string-functions.md#substring)) string function, but for bitmap.
Returns a subset of the bitmap, starting from position `offset`. The maximum cardinality of the returned bitmap is `cardinality_limit`.
**Syntax**
@ -142,16 +138,8 @@ subBitmap(bitmap, offset, cardinality_limit)
- `offset` The position of the first element of the subset. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
- `cardinality_limit` The maximum number of elements in the subset. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
**Returned value**
The subset.
Type: [Bitmap object](#bitmap_functions-bitmapbuild).
**Example**
Query:
``` sql
SELECT bitmapToArray(subBitmap(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(10), toUInt32(10))) AS res;
```
@ -169,18 +157,18 @@ Result:
Checks whether the bitmap contains an element.
``` sql
bitmapContains(haystack, needle)
bitmapContains(bitmap, needle)
```
**Arguments**
- `haystack` [Bitmap object](#bitmap_functions-bitmapbuild), where the function searches.
- `needle` Value that the function searches. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
- `bitmap` [Bitmap object](#bitmap_functions-bitmapbuild).
- `needle` Searched bit value. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
**Returned values**
- 0 — If `haystack` does not contain `needle`.
- 1 — If `haystack` contains `needle`.
- 0 — If `bitmap` does not contain `needle`.
- 1 — If `bitmap` contains `needle`.
Type: `UInt8`.
@ -190,6 +178,8 @@ Type: `UInt8`.
SELECT bitmapContains(bitmapBuild([1,5,7,9]), toUInt32(9)) AS res;
```
Result:
``` text
┌─res─┐
│ 1 │
@ -198,21 +188,24 @@ SELECT bitmapContains(bitmapBuild([1,5,7,9]), toUInt32(9)) AS res;
## bitmapHasAny
Checks whether two bitmaps have intersection by some elements.
Checks whether two bitmaps intersect.
If `bitmap2` contains exactly one element, consider using [bitmapContains](#bitmap_functions-bitmapcontains) instead as it works more efficiently.
**Syntax**
``` sql
bitmapHasAny(bitmap1, bitmap2)
```
If you are sure that `bitmap2` contains strictly one element, consider using the [bitmapContains](#bitmap_functions-bitmapcontains) function. It works more efficiently.
**Arguments**
- `bitmap*` Bitmap object.
- `bitmap1` Bitmap object 1.
- `bitmap2` Bitmap object 2.
**Return values**
- `1`, if `bitmap1` and `bitmap2` have one similar element at least.
- `1`, if `bitmap1` and `bitmap2` have at least one shared element.
- `0`, otherwise.
**Example**
@ -221,6 +214,8 @@ If you are sure that `bitmap2` contains strictly one element, consider using the
SELECT bitmapHasAny(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res;
```
Result:
``` text
┌─res─┐
│ 1 │
@ -229,16 +224,21 @@ SELECT bitmapHasAny(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res;
## bitmapHasAll
Analogous to `hasAll(array, array)` returns 1 if the first bitmap contains all the elements of the second one, 0 otherwise.
If the second argument is an empty bitmap then returns 1.
Returns 1 if the first bitmap contains all elements of the second bitmap, otherwise 0.
If the second bitmap is empty, returns 1.
Also see `hasAll(array, array)`.
**Syntax**
``` sql
bitmapHasAll(bitmap,bitmap)
bitmapHasAll(bitmap1, bitmap2)
```
**Arguments**
- `bitmap` Bitmap object.
- `bitmap1` Bitmap object 1.
- `bitmap2` Bitmap object 2.
**Example**
@ -246,6 +246,8 @@ bitmapHasAll(bitmap,bitmap)
SELECT bitmapHasAll(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res;
```
Result:
``` text
┌─res─┐
│ 0 │
@ -254,7 +256,9 @@ SELECT bitmapHasAll(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res;
## bitmapCardinality
Retrun bitmap cardinality of type UInt64.
Rerturn the cardinality of a bitmap.
**Syntax**
``` sql
bitmapCardinality(bitmap)
@ -270,6 +274,8 @@ bitmapCardinality(bitmap)
SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res;
```
Result:
``` text
┌─res─┐
│ 5 │
@ -278,9 +284,13 @@ SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res;
## bitmapMin
Retrun the smallest value of type UInt64 in the set, UINT32_MAX if the set is empty.
Computes the smallest bit set in a bitmap, or UINT32_MAX if the bitmap is empty.
bitmapMin(bitmap)
**Syntax**
```sql
bitmapMin(bitmap)
```
**Arguments**
@ -292,6 +302,8 @@ Retrun the smallest value of type UInt64 in the set, UINT32_MAX if the set is em
SELECT bitmapMin(bitmapBuild([1, 2, 3, 4, 5])) AS res;
```
Result:
``` text
┌─res─┐
│ 1 │
@ -300,9 +312,13 @@ SELECT bitmapMin(bitmapBuild([1, 2, 3, 4, 5])) AS res;
## bitmapMax
Retrun the greatest value of type UInt64 in the set, 0 if the set is empty.
Computes the greatest bit set in a bitmap, or 0 if the bitmap is empty.
bitmapMax(bitmap)
**Syntax**
```sql
bitmapMax(bitmap)
```
**Arguments**
@ -314,6 +330,8 @@ Retrun the greatest value of type UInt64 in the set, 0 if the set is empty.
SELECT bitmapMax(bitmapBuild([1, 2, 3, 4, 5])) AS res;
```
Result:
``` text
┌─res─┐
│ 5 │
@ -322,15 +340,21 @@ SELECT bitmapMax(bitmapBuild([1, 2, 3, 4, 5])) AS res;
## bitmapTransform
Transform an array of values in a bitmap to another array of values, the result is a new bitmap.
Replaces at most N bits in a bitmap. The old and new value of the i-th replaced bit is given by `from_array[i]` and `to_array[i]`.
bitmapTransform(bitmap, from_array, to_array)
The result depends on the array ordering if `from_array` and `to_array`.
**Syntax**
``` sql
bitmapTransform(bitmap, from_array, to_array)
```
**Arguments**
- `bitmap` Bitmap object.
- `from_array` UInt32 array. For idx in range \[0, from_array.size()), if bitmap contains from_array\[idx\], then replace it with to_array\[idx\]. Note that the result depends on array ordering if there are common elements between from_array and to_array.
- `to_array` UInt32 array, its size shall be the same to from_array.
- `from_array` UInt32 array. For idx in range \[0, from_array.size()), if bitmap contains from_array\[idx\], then replace it with to_array\[idx\].
- `to_array` UInt32 array with the same size as `from_array`.
**Example**
@ -338,6 +362,8 @@ Transform an array of values in a bitmap to another array of values, the result
SELECT bitmapToArray(bitmapTransform(bitmapBuild([1, 2, 3, 4, 5, 6, 7, 8, 9, 10]), cast([5,999,2] as Array(UInt32)), cast([2,888,20] as Array(UInt32)))) AS res;
```
Result:
``` text
┌─res───────────────────┐
│ [1,3,4,6,7,8,9,10,20] │
@ -346,7 +372,9 @@ SELECT bitmapToArray(bitmapTransform(bitmapBuild([1, 2, 3, 4, 5, 6, 7, 8, 9, 10]
## bitmapAnd
Two bitmap and calculation, the result is a new bitmap.
Computes the logical conjunction of two two bitmaps.
**Syntax**
``` sql
bitmapAnd(bitmap,bitmap)
@ -362,6 +390,8 @@ bitmapAnd(bitmap,bitmap)
SELECT bitmapToArray(bitmapAnd(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS res;
```
Result:
``` text
┌─res─┐
│ [3] │
@ -370,7 +400,9 @@ SELECT bitmapToArray(bitmapAnd(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS re
## bitmapOr
Two bitmap or calculation, the result is a new bitmap.
Computes the logical disjunction of two bitmaps.
**Syntax**
``` sql
bitmapOr(bitmap,bitmap)
@ -386,6 +418,8 @@ bitmapOr(bitmap,bitmap)
SELECT bitmapToArray(bitmapOr(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS res;
```
Result:
``` text
┌─res─────────┐
│ [1,2,3,4,5] │
@ -394,7 +428,9 @@ SELECT bitmapToArray(bitmapOr(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS res
## bitmapXor
Two bitmap xor calculation, the result is a new bitmap.
Xor-s two bitmaps.
**Syntax**
``` sql
bitmapXor(bitmap,bitmap)
@ -410,6 +446,8 @@ bitmapXor(bitmap,bitmap)
SELECT bitmapToArray(bitmapXor(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS res;
```
Result:
``` text
┌─res───────┐
│ [1,2,4,5] │
@ -418,7 +456,9 @@ SELECT bitmapToArray(bitmapXor(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS re
## bitmapAndnot
Two bitmap andnot calculation, the result is a new bitmap.
Computes the logical conjunction of two bitmaps and negates the result.
**Syntax**
``` sql
bitmapAndnot(bitmap,bitmap)
@ -434,6 +474,8 @@ bitmapAndnot(bitmap,bitmap)
SELECT bitmapToArray(bitmapAndnot(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS res;
```
Result:
``` text
┌─res───┐
│ [1,2] │
@ -442,7 +484,9 @@ SELECT bitmapToArray(bitmapAndnot(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS
## bitmapAndCardinality
Two bitmap and calculation, return cardinality of type UInt64.
Returns the cardinality of the logical conjunction of two bitmaps.
**Syntax**
``` sql
bitmapAndCardinality(bitmap,bitmap)
@ -458,6 +502,8 @@ bitmapAndCardinality(bitmap,bitmap)
SELECT bitmapAndCardinality(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res;
```
Result:
``` text
┌─res─┐
│ 1 │
@ -466,7 +512,7 @@ SELECT bitmapAndCardinality(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res;
## bitmapOrCardinality
Two bitmap or calculation, return cardinality of type UInt64.
Returns the cardinality of the logical disjunction of two bitmaps.
``` sql
bitmapOrCardinality(bitmap,bitmap)
@ -482,6 +528,8 @@ bitmapOrCardinality(bitmap,bitmap)
SELECT bitmapOrCardinality(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res;
```
Result:
``` text
┌─res─┐
│ 5 │
@ -490,7 +538,7 @@ SELECT bitmapOrCardinality(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res;
## bitmapXorCardinality
Two bitmap xor calculation, return cardinality of type UInt64.
Returns the cardinality of the XOR of two bitmaps.
``` sql
bitmapXorCardinality(bitmap,bitmap)
@ -506,6 +554,8 @@ bitmapXorCardinality(bitmap,bitmap)
SELECT bitmapXorCardinality(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res;
```
Result:
``` text
┌─res─┐
│ 4 │
@ -514,7 +564,7 @@ SELECT bitmapXorCardinality(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res;
## bitmapAndnotCardinality
Two bitmap andnot calculation, return cardinality of type UInt64.
Returns the cardinality of the AND-NOT operation of two bitmaps.
``` sql
bitmapAndnotCardinality(bitmap,bitmap)
@ -530,6 +580,8 @@ bitmapAndnotCardinality(bitmap,bitmap)
SELECT bitmapAndnotCardinality(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res;
```
Result:
``` text
┌─res─┐
│ 2 │

View File

@ -1,34 +1,89 @@
---
slug: /en/sql-reference/functions/comparison-functions
sidebar_position: 36
sidebar_position: 35
sidebar_label: Comparison
---
# Comparison Functions
Comparison functions always return 0 or 1 (Uint8).
Below comparison functions return 0 or 1 as Uint8.
The following types can be compared:
- numbers
- strings and fixed strings
- dates
- dates with times
within each group, but not between different groups.
Only values within the same group can be compared (e.g. UInt16 and UInt64) but not accross groups (e.g. UInt16 and DateTime).
For example, you cant compare a date with a string. You have to use a function to convert the string to a date, or vice versa.
Strings are compared byte-by-byte. Note that this may lead to unexpected results if one of the strings contains UTF-8 encoded multi-byte characters.
Strings are compared by bytes. A shorter string is smaller than all strings that start with it and that contain at least one more character.
A string S1 which has another string S2 as prefix is considered longer than S2.
### equals, a `=` b and a `==` b operator
## equals
### notEquals, a `!=` b and a `<>` b operator
**Syntax**
### less, `<` operator
```sql
equals(a, b)
```
### greater, `>` operator
Alias:
- `a = b` (operator)
- `a == b` (operator)
### lessOrEquals, `<=` operator
## notEquals
### greaterOrEquals, `>=` operator
**Syntax**
```sql
notEquals(a, b)
```
Alias:
- `a != b` (operator)
- `a <> b` (operator)
## less
**Syntax**
```sql
less(a, b)
```
Alias:
- `a < b` (operator)
## greater
**Syntax**
```sql
greater(a, b)
```
Alias:
- `a > b` (operator)
## lessOrEquals
**Syntax**
```sql
lessOrEquals(a, b)
```
Alias:
- `a <= b` (operator)
## greaterOrEquals, `>=` operator
**Syntax**
```sql
greaterOrEquals(a, b)
```
Alias:
- `a >= b` (operator)

View File

@ -1,39 +1,40 @@
---
slug: /en/sql-reference/functions/conditional-functions
sidebar_position: 43
sidebar_label: 'Conditional '
sidebar_position: 40
sidebar_label: Conditional
---
# Conditional Functions
## if
Controls conditional branching. Unlike most systems, ClickHouse always evaluate both expressions `then` and `else`.
Performs conditional branching.
If the condition `cond` evaluates to a non-zero value, the function returns the result of the expression `then`. If `cond` evaluates to zero or `NULL`, then the result of the `else` expression is returned.
Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, the `then` expression is evaluated only on rows where `cond` is `true` and the `else` expression where `cond` is `false`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT if(number = 0, 0, intDiv(42, number)) FROM numbers(10)`.
`then` and `else` must be of a similar type.
**Syntax**
``` sql
if(cond, then, else)
```
If the condition `cond` evaluates to a non-zero value, returns the result of the expression `then`, and the result of the expression `else`, if present, is skipped. If the `cond` is zero or `NULL`, then the result of the `then` expression is skipped and the result of the `else` expression, if present, is returned.
You can use the [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) setting to calculate the `if` function according to a short scheme. If this setting is enabled, `then` expression is evaluated only on rows where `cond` is true, `else` expression where `cond` is false. For example, an exception about division by zero is not thrown when executing the query `SELECT if(number = 0, 0, intDiv(42, number)) FROM numbers(10)`, because `intDiv(42, number)` will be evaluated only for numbers that doesn't satisfy condition `number = 0`.
Alias: `cond ? then : else` (ternary operator)
**Arguments**
- `cond` The condition for evaluation that can be zero or not. The type is UInt8, Nullable(UInt8) or NULL.
- `then` The expression to return if condition is met.
- `else` The expression to return if condition is not met.
- `cond` The evaluated condition. UInt8, Nullable(UInt8) or NULL.
- `then` The expression returned if `condition` is true.
- `else` The expression returned if `condition` is `false` or NULL.
**Returned values**
The function executes `then` and `else` expressions and returns its result, depending on whether the condition `cond` ended up being zero or not.
The result of either the `then` and `else` expressions, depending on condition `cond`.
**Example**
Query:
``` sql
SELECT if(1, plus(2, 2), plus(2, 6));
```
@ -46,30 +47,34 @@ Result:
└────────────┘
```
Query:
## multiIf
Allows to write the [CASE](../../sql-reference/operators/index.md#operator_case) operator more compactly in the query.
**Syntax**
``` sql
SELECT if(0, plus(2, 2), plus(2, 6));
multiIf(cond_1, then_1, cond_2, then_2, ..., else)
```
Result:
Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, the `then_i` expression is evaluated only on rows where `((NOT cond_1) AND (NOT cond_2) AND ... AND (NOT cond_{i-1}) AND cond_i)` is `true`, `cond_i` will be evaluated only on rows where `((NOT cond_1) AND (NOT cond_2) AND ... AND (NOT cond_{i-1}))` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT multiIf(number = 2, intDiv(1, number), number = 5) FROM numbers(10)`.
**Arguments**
The function accepts `2N+1` parameters:
- `cond_N` — The N-th evaluated condition which controls if `then_N` is returned.
- `then_N` — The result of the function when `cond_N` is true.
- `else` — The result of the function if none of conditions is true.
**Returned values**
The result of either any of the `then_N` or `else` expressions, depending on the conditions `cond_N`.
**Example**
Assuming this table:
``` text
┌─plus(2, 6)─┐
│ 8 │
└────────────┘
```
- `then` and `else` must have the lowest common type.
**Example:**
Take this `LEFT_RIGHT` table:
``` sql
SELECT *
FROM LEFT_RIGHT
┌─left─┬─right─┐
│ ᴺᵁᴸᴸ │ 4 │
│ 1 │ 3 │
@ -79,69 +84,6 @@ FROM LEFT_RIGHT
└──────┴───────┘
```
The following query compares `left` and `right` values:
``` sql
SELECT
left,
right,
if(left < right, 'left is smaller than right', 'right is greater or equal than left') AS is_smaller
FROM LEFT_RIGHT
WHERE isNotNull(left) AND isNotNull(right)
┌─left─┬─right─┬─is_smaller──────────────────────────┐
│ 1 │ 3 │ left is smaller than right │
│ 2 │ 2 │ right is greater or equal than left │
│ 3 │ 1 │ right is greater or equal than left │
└──────┴───────┴─────────────────────────────────────┘
```
Note: `NULL` values are not used in this example, check [NULL values in conditionals](#null-values-in-conditionals) section.
## Ternary Operator
It works same as `if` function.
Syntax: `cond ? then : else`
Returns `then` if the `cond` evaluates to be true (greater than zero), otherwise returns `else`.
- `cond` must be of type of `UInt8`, and `then` and `else` must have the lowest common type.
- `then` and `else` can be `NULL`
**See also**
- [ifNotFinite](../../sql-reference/functions/other-functions.md#ifnotfinite).
## multiIf
Allows you to write the [CASE](../../sql-reference/operators/index.md#operator_case) operator more compactly in the query.
**Syntax**
``` sql
multiIf(cond_1, then_1, cond_2, then_2, ..., else)
```
You can use the [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) setting to calculate the `multiIf` function according to a short scheme. If this setting is enabled, `then_i` expression is evaluated only on rows where `((NOT cond_1) AND (NOT cond_2) AND ... AND (NOT cond_{i-1}) AND cond_i)` is true, `cond_i` will be evaluated only on rows where `((NOT cond_1) AND (NOT cond_2) AND ... AND (NOT cond_{i-1}))` is true. For example, an exception about division by zero is not thrown when executing the query `SELECT multiIf(number = 2, intDiv(1, number), number = 5) FROM numbers(10)`.
**Arguments**
- `cond_N` — The condition for the function to return `then_N`.
- `then_N` — The result of the function when executed.
- `else` — The result of the function if none of the conditions is met.
The function accepts `2N+1` parameters.
**Returned values**
The function returns one of the values `then_N` or `else`, depending on the conditions `cond_N`.
**Example**
Again using `LEFT_RIGHT` table.
``` sql
SELECT
left,

View File

@ -1,14 +1,14 @@
---
slug: /en/sql-reference/functions/date-time-functions
sidebar_position: 39
sidebar_position: 45
sidebar_label: Dates and Times
---
# Functions for Working with Dates and Times
Support for time zones.
Most functions in this section accept an optional time zone argument, e.g. `Europe/Amsterdam`. In this case, the time zone is the specified one instead of the local (default) one.
All functions for working with the date and time that have a logical use for the time zone can accept a second optional time zone argument. Example: Asia/Yekaterinburg. In this case, they use the specified time zone instead of the local (default) one.
**Example**
``` sql
SELECT
@ -27,11 +27,11 @@ SELECT
## timeZone
Returns the timezone of the server.
If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value.
If the function is executed in the context of a distributed table, it generates a normal column with values relevant to each shard, otherwise it produces a constant value.
**Syntax**
``` sql
```sql
timeZone()
```
@ -45,7 +45,7 @@ Type: [String](../../sql-reference/data-types/string.md).
## toTimeZone
Converts time or date and time to the specified time zone. The time zone is an attribute of the `Date` and `DateTime` data types. The internal value (number of seconds) of the table field or of the resultset's column does not change, the column's type changes and its string representation changes accordingly.
Converts a date or date with time to the specified time zone. Does not change the internal value (number of unix seconds) of the data, only the value's time zone attribute and the value's string representation changes.
**Syntax**
@ -68,8 +68,6 @@ Type: [DateTime](../../sql-reference/data-types/datetime.md).
**Example**
Query:
```sql
SELECT toDateTime('2019-01-01 00:00:00', 'UTC') AS time_utc,
toTypeName(time_utc) AS type_utc,
@ -99,8 +97,6 @@ type_samoa: DateTime('US/Samoa')
int32samoa: 1546300800
```
`toTimeZone(time_utc, 'Asia/Yekaterinburg')` changes the `DateTime('UTC')` type to `DateTime('Asia/Yekaterinburg')`. The value (Unixtimestamp) 1546300800 stays the same, but the string representation (the result of the toString() function) changes from `time_utc: 2019-01-01 00:00:00` to `time_yekat: 2019-01-01 05:00:00`.
## timeZoneOf
Returns the timezone name of [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md) data types.
@ -125,7 +121,6 @@ Type: [String](../../sql-reference/data-types/string.md).
**Example**
Query:
``` sql
SELECT timezoneOf(now());
```
@ -139,8 +134,9 @@ Result:
## timeZoneOffset
Returns a timezone offset in seconds from [UTC](https://en.wikipedia.org/wiki/Coordinated_Universal_Time). The function takes into account [daylight saving time](https://en.wikipedia.org/wiki/Daylight_saving_time) and historical timezone changes at the specified date and time.
[IANA timezone database](https://www.iana.org/time-zones) is used to calculate the offset.
Returns the timezone offset in seconds from [UTC](https://en.wikipedia.org/wiki/Coordinated_Universal_Time).
The function [daylight saving time](https://en.wikipedia.org/wiki/Daylight_saving_time) and historical timezone changes at the specified date and time into account.
The [IANA timezone database](https://www.iana.org/time-zones) is used to calculate the offset.
**Syntax**
@ -162,8 +158,6 @@ Type: [Int32](../../sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT toDateTime('2021-04-21 10:20:30', 'America/New_York') AS Time, toTypeName(Time) AS Type,
timeZoneOffset(Time) AS Offset_in_seconds, (Offset_in_seconds / 3600) AS Offset_in_hours;
@ -179,37 +173,37 @@ Result:
## toYear
Converts a date or date with time to a UInt16 number containing the year number (AD).
Converts a date or date with time to the year number (AD) as UInt16 value.
Alias: `YEAR`.
## toQuarter
Converts a date or date with time to a UInt8 number containing the quarter number.
Converts a date or date with time to the quarter number as UInt8 value.
Alias: `QUARTER`.
## toMonth
Converts a date or date with time to a UInt8 number containing the month number (1-12).
Converts a date or date with time to the month number (1-12) as UInt8 value.
Alias: `MONTH`.
## toDayOfYear
Converts a date or date with time to a UInt16 number containing the number of the day of the year (1-366).
Converts a date or date with time to the number of the day of the year (1-366) as UInt16 value.
Alias: `DAYOFYEAR`.
## toDayOfMonth
Converts a date or date with time to a UInt8 number containing the number of the day of the month (1-31).
Converts a date or date with time to the number of the day in the month (1-31) as UInt8 value.
Aliases: `DAYOFMONTH`, `DAY`.
## toDayOfWeek
Converts a date or date with time to a UInt8 number containing the number of the day of the week.
Converts a date or date with time to the number of the day in the week as UInt8 value.
The two-argument form of `toDayOfWeek()` enables you to specify whether the week starts on Monday or Sunday, and whether the return value should be in the range from 0 to 6 or 1 to 7. If the mode argument is ommited, the default mode is 0. The time zone of the date can be specified as the third argument.
@ -230,27 +224,28 @@ toDayOfWeek(t[, mode[, timezone]])
## toHour
Converts a date with time to a UInt8 number containing the number of the hour in 24-hour time (0-23).
This function assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always true even in Moscow the clocks were twice changed at a different time).
Converts a date with time the number of the hour in 24-hour time (0-23) as UInt8 value.
Assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always true even in Moscow the clocks were twice changed at a different time).
Alias: `HOUR`.
## toMinute
Converts a date with time to a UInt8 number containing the number of the minute of the hour (0-59).
Converts a date with time to the number of the minute of the hour (0-59) as UInt8 value.
Alias: `MINUTE`.
## toSecond
Converts a date with time to a UInt8 number containing the number of the second in the minute (0-59).
Leap seconds are not accounted for.
Converts a date with time to the second in the minute (0-59) as UInt8 value. Leap seconds are not considered.
Alias: `SECOND`.
## toUnixTimestamp
For DateTime argument: converts value to the number with type UInt32 -- Unix Timestamp (https://en.wikipedia.org/wiki/Unix_time).
For DateTime arguments: converts the value to the number with type UInt32 -- Unix Timestamp (https://en.wikipedia.org/wiki/Unix_time).
For String argument: converts the input string to the datetime according to the timezone (optional second argument, server timezone is used by default) and returns the corresponding unix timestamp.
**Syntax**
@ -268,8 +263,6 @@ Type: `UInt32`.
**Example**
Query:
``` sql
SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp
```
@ -596,8 +589,6 @@ Type: [Int](../../sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT age('hour', toDateTime('2018-01-01 22:30:00'), toDateTime('2018-01-02 23:00:00'));
```
@ -610,8 +601,6 @@ Result:
└───────────────────────────────────────────────────────────────────────────────────┘
```
Query:
``` sql
SELECT
toDate('2022-01-01') AS e,
@ -675,8 +664,6 @@ Type: [Int](../../sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'));
```
@ -689,8 +676,6 @@ Result:
└────────────────────────────────────────────────────────────────────────────────────────┘
```
Query:
``` sql
SELECT
toDate('2022-01-01') AS e,
@ -814,8 +799,6 @@ Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-ref
**Example**
Query:
```sql
SELECT date_add(YEAR, 3, toDate('2018-01-01'));
```
@ -866,8 +849,6 @@ Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-ref
**Example**
Query:
``` sql
SELECT date_sub(YEAR, 3, toDate('2018-01-01'));
```
@ -916,8 +897,6 @@ Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-ref
**Example**
Query:
```sql
select timestamp_add(toDate('2018-01-01'), INTERVAL 3 MONTH);
```
@ -967,8 +946,6 @@ Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-ref
**Example**
Query:
```sql
select timestamp_sub(MONTH, 5, toDateTime('2018-12-18 01:02:03'));
```
@ -1301,8 +1278,6 @@ Note 2: In ClickHouse versions earlier than v23.4, `%M` prints the minute (00-59
**Example**
Query:
``` sql
SELECT formatDateTime(toDate('2010-01-04'), '%g')
```
@ -1315,8 +1290,6 @@ Result:
└────────────────────────────────────────────┘
```
Query:
``` sql
SELECT formatDateTime(toDateTime64('2010-01-04 12:34:56.123456', 7), '%f')
```
@ -1373,8 +1346,6 @@ Using replacement fields, you can define a pattern for the resulting string.
**Example**
Query:
``` sql
SELECT formatDateTimeInJodaSyntax(toDateTime('2010-01-04 12:34:56'), 'yyyy-MM-dd HH:mm:ss')
```
@ -1412,8 +1383,6 @@ Type: [String](../../sql-reference/data-types/string.md#string)
**Example**
Query:
```sql
WITH toDateTime('2021-04-14 11:22:33') AS date_value
SELECT
@ -1452,8 +1421,6 @@ Type: [String](../../sql-reference/data-types/string.md#string)
**Example**
Query:
```sql
WITH toDateTime('2021-04-14 11:22:33') AS date_value
SELECT monthName(date_value);
@ -1477,8 +1444,6 @@ Alias: `FROM_UNIXTIME`.
**Example:**
Query:
```sql
SELECT fromUnixTimestamp(423543535);
```
@ -1515,7 +1480,6 @@ Similar to fromUnixTimestamp, except that it formats time in Joda style instead
**Example:**
Query:
``` sql
SELECT fromUnixTimestampInJodaSyntax(1669804872, 'yyyy-MM-dd HH:mm:ss', 'UTC');
```
@ -1549,8 +1513,6 @@ Type: [Int32](../../sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT toModifiedJulianDay('2020-01-01');
```
@ -1585,8 +1547,6 @@ Type: [Nullable(Int32)](../../sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT toModifiedJulianDayOrNull('2020-01-01');
```
@ -1621,8 +1581,6 @@ Type: [String](../../sql-reference/data-types/string.md)
**Example**
Query:
``` sql
SELECT fromModifiedJulianDay(58849);
```
@ -1657,8 +1615,6 @@ Type: [Nullable(String)](../../sql-reference/data-types/string.md)
**Example**
Query:
``` sql
SELECT fromModifiedJulianDayOrNull(58849);
```

View File

@ -1,7 +1,10 @@
---
slug: /en/sql-reference/functions/distance-functions
sidebar_position: 55
sidebar_label: Distance
---
# Distance functions
# Distance Functions
## L1Norm

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/encoding-functions
sidebar_position: 52
sidebar_position: 65
sidebar_label: Encoding
---

View File

@ -1,8 +1,7 @@
---
slug: /en/sql-reference/functions/encryption-functions
sidebar_position: 67
sidebar_position: 70
sidebar_label: Encryption
title: "Encryption functions"
---
These functions implement encryption and decryption of data with AES (Advanced Encryption Standard) algorithm.

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/ext-dict-functions
sidebar_position: 58
sidebar_position: 50
sidebar_label: Dictionaries
---

View File

@ -1,13 +1,14 @@
---
slug: /en/sql-reference/functions/files
sidebar_position: 43
sidebar_position: 75
sidebar_label: Files
title: "Functions for Working with Files"
---
## file
Reads file as a String. The file content is not parsed, so any information is read as one string and placed into the specified column.
Reads file as string and loads the data into the specified column. The actual file content is not interpreted.
Also see table function [file](../table-functions/file.md).
**Syntax**
@ -17,8 +18,8 @@ file(path[, default])
**Arguments**
- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following wildcards: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` strings.
- `default` — The value that will be returned in the case when a file does not exist or cannot be accessed. Data types supported: [String](../../sql-reference/data-types/string.md) and [NULL](../../sql-reference/syntax.md#null-literal).
- `path` — The path of the file relative to [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports the following wildcards: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` are numbers and `'abc', 'def'` are strings.
- `default` — The value that will be returned in the case the file does not exist or cannot be accessed. Supported data types: [String](../../sql-reference/data-types/string.md) and [NULL](../../sql-reference/syntax.md#null-literal).
**Example**
@ -29,8 +30,3 @@ Query:
``` sql
INSERT INTO table SELECT file('a.txt'), file('b.txt');
```
**See Also**
- [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path)
- [file](../table-functions/file.md)

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/functions-for-nulls
sidebar_position: 63
sidebar_position: 135
sidebar_label: Nullable
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/hash-functions
sidebar_position: 50
sidebar_position: 85
sidebar_label: Hash
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/in-functions
sidebar_position: 60
sidebar_position: 90
sidebar_label: IN Operator
---

View File

@ -1,10 +1,10 @@
---
slug: /en/sql-reference/functions/
sidebar_position: 32
sidebar_label: Functions
sidebar_position: 1
sidebar_label: Overview
---
# Functions
# Regular Functions
There are at least\* two types of functions - regular functions (they are just called “functions”) and aggregate functions. These are completely different concepts. Regular functions work as if they are applied to each row separately (for each row, the result of the function does not depend on the other rows). Aggregate functions accumulate a set of values from various rows (i.e. they depend on the entire set of rows).

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/introspection
sidebar_position: 65
sidebar_position: 100
sidebar_label: Introspection
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/ip-address-functions
sidebar_position: 55
sidebar_position: 95
sidebar_label: IP Addresses
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/json-functions
sidebar_position: 56
sidebar_position: 105
sidebar_label: JSON
---

View File

@ -1,18 +1,18 @@
---
slug: /en/sql-reference/functions/logical-functions
sidebar_position: 37
sidebar_position: 110
sidebar_label: Logical
---
# Logical Functions
Performs logical operations on arguments of any numeric types, but returns a [UInt8](../../sql-reference/data-types/int-uint.md) number equal to 0, 1 or `NULL` in some cases.
Below functions perform logical operations on arguments of arbitrary numeric types. They return either 0 or 1 as [UInt8](../../sql-reference/data-types/int-uint.md) or in some cases `NULL`.
Zero as an argument is considered `false`, while any non-zero value is considered `true`.
Zero as an argument is considered `false`, non-zero values are considered `true`.
## and
Calculates the result of the logical conjunction between two or more values. Corresponds to [Logical AND Operator](../../sql-reference/operators/index.md#logical-and-operator).
Calculates the logical conjunction between two or more values.
**Syntax**
@ -20,7 +20,9 @@ Calculates the result of the logical conjunction between two or more values. Cor
and(val1, val2...)
```
You can use the [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) setting to calculate the `and` function according to a short scheme. If this setting is enabled, `vali` is evaluated only on rows where `(val1 AND val2 AND ... AND val{i-1})` is true. For example, an exception about division by zero is not thrown when executing the query `SELECT and(number = 2, intDiv(1, number)) FROM numbers(10)`.
Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `(val_1 AND val_2 AND ... AND val_{i-1})` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT and(number = 2, intDiv(1, number)) FROM numbers(5)`.
Alias: The [AND Operator](../../sql-reference/operators/index.md#logical-and-operator).
**Arguments**
@ -28,16 +30,14 @@ You can use the [short_circuit_function_evaluation](../../operations/settings/se
**Returned value**
- `0`, if there is at least one zero value argument.
- `NULL`, if there are no zero values arguments and there is at least one `NULL` argument.
- `0`, if there at least one argument evaluates to `false`,
- `NULL`, if no argumetn evaluates to `false` and at least one argument is `NULL`,
- `1`, otherwise.
Type: [UInt8](../../sql-reference/data-types/int-uint.md) or [Nullable](../../sql-reference/data-types/nullable.md)([UInt8](../../sql-reference/data-types/int-uint.md)).
**Example**
Query:
``` sql
SELECT and(0, 1, -2);
```
@ -66,7 +66,7 @@ Result:
## or
Calculates the result of the logical disjunction between two or more values. Corresponds to [Logical OR Operator](../../sql-reference/operators/index.md#logical-or-operator).
Calculates the logical disjunction between two or more values.
**Syntax**
@ -74,7 +74,9 @@ Calculates the result of the logical disjunction between two or more values. Cor
or(val1, val2...)
```
You can use the [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) setting to calculate the `or` function according to a short scheme. If this setting is enabled, `vali` is evaluated only on rows where `((NOT val1) AND (NOT val2) AND ... AND (NOT val{i-1}))` is true. For example, an exception about division by zero is not thrown when executing the query `SELECT or(number = 0, intDiv(1, number) != 0) FROM numbers(10)`.
Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `((NOT val_1) AND (NOT val_2) AND ... AND (NOT val_{i-1}))` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT or(number = 0, intDiv(1, number) != 0) FROM numbers(5)`.
Alias: The [OR Operator](../../sql-reference/operators/index.md#logical-or-operator).
**Arguments**
@ -82,16 +84,14 @@ You can use the [short_circuit_function_evaluation](../../operations/settings/se
**Returned value**
- `1`, if there is at least one non-zero value.
- `0`, if there are only zero values.
- `NULL`, if there are only zero values and `NULL`.
- `1`, if at least one argument evalutes to `true`,
- `0`, if all arguments evaluate to `false`,
- `NULL`, if all arguments evaluate to `false` and at least one argument is `NULL`.
Type: [UInt8](../../sql-reference/data-types/int-uint.md) or [Nullable](../../sql-reference/data-types/nullable.md)([UInt8](../../sql-reference/data-types/int-uint.md)).
**Example**
Query:
``` sql
SELECT or(1, 0, 0, 2, NULL);
```
@ -120,7 +120,7 @@ Result:
## not
Calculates the result of the logical negation of the value. Corresponds to [Logical Negation Operator](../../sql-reference/operators/index.md#logical-negation-operator).
Calculates logical negation of a value.
**Syntax**
@ -128,22 +128,22 @@ Calculates the result of the logical negation of the value. Corresponds to [Logi
not(val);
```
Alias: The [Negation Operator](../../sql-reference/operators/index.md#logical-negation-operator).
**Arguments**
- `val` — The value. [Int](../../sql-reference/data-types/int-uint.md), [UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Nullable](../../sql-reference/data-types/nullable.md).
**Returned value**
- `1`, if the `val` is `0`.
- `0`, if the `val` is a non-zero value.
- `NULL`, if the `val` is a `NULL` value.
- `1`, if `val` evaluates to `false`,
- `0`, if `val` evaluates to `true`,
- `NULL`, if `val` is `NULL`.
Type: [UInt8](../../sql-reference/data-types/int-uint.md) or [Nullable](../../sql-reference/data-types/nullable.md)([UInt8](../../sql-reference/data-types/int-uint.md)).
**Example**
Query:
``` sql
SELECT NOT(1);
```
@ -158,7 +158,7 @@ Result:
## xor
Calculates the result of the logical exclusive disjunction between two or more values. For more than two values the function works as if it calculates `XOR` of the first two values and then uses the result with the next value to calculate `XOR` and so on.
Calculates the logical exclusive disjunction between two or more values. For more than two values the function first xor-s the first two values, then xor-s the result with the third value etc.
**Syntax**
@ -172,16 +172,14 @@ xor(val1, val2...)
**Returned value**
- `1`, for two values: if one of the values is zero and other is not.
- `0`, for two values: if both values are zero or non-zero at the same time.
- `NULL`, if there is at least one `NULL` value.
- `1`, for two values: if one of the values evaluates to `false` and other does not,
- `0`, for two values: if both values evalute to `false` or to both `true`,
- `NULL`, if at least one of the inputs is `NULL`
Type: [UInt8](../../sql-reference/data-types/int-uint.md) or [Nullable](../../sql-reference/data-types/nullable.md)([UInt8](../../sql-reference/data-types/int-uint.md)).
**Example**
Query:
``` sql
SELECT xor(0, 1, 1);
```

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/machine-learning-functions
sidebar_position: 64
sidebar_position: 115
sidebar_label: Machine Learning
---

View File

@ -1,120 +1,260 @@
---
slug: /en/sql-reference/functions/math-functions
sidebar_position: 44
sidebar_position: 125
sidebar_label: Mathematical
---
# Mathematical Functions
All the functions return a Float64 number. The accuracy of the result is close to the maximum precision possible, but the result might not coincide with the machine representable number nearest to the corresponding real number.
All the functions return a Float64 number. Results are generally as close to the actual result as possible, but in some cases less precise than the machine-representable number.
## e()
## e
Returns a Float64 number that is close to the number e.
Returns e.
## pi()
**Syntax**
Returns a Float64 number that is close to the number π.
```sql
e()
```
## exp(x)
## pi
Accepts a numeric argument and returns a Float64 number close to the exponent of the argument.
Returns π.
## log(x), ln(x)
**Syntax**
Accepts a numeric argument and returns a Float64 number close to the natural logarithm of the argument.
```sql
pi()
```
## exp2(x)
## exp
Accepts a numeric argument and returns a Float64 number close to 2 to the power of x.
Returns e to the power of the given argument.
## log2(x)
**Syntax**
Accepts a numeric argument and returns a Float64 number close to the binary logarithm of the argument.
```sql
exp(x)
```
## exp10(x)
## log
Accepts a numeric argument and returns a Float64 number close to 10 to the power of x.
Returns the natural logarithm of the argument.
## log10(x)
**Syntax**
Accepts a numeric argument and returns a Float64 number close to the decimal logarithm of the argument.
```sql
log(x)
```
## sqrt(x)
Alias: `ln(x)`
Accepts a numeric argument and returns a Float64 number close to the square root of the argument.
## exp2
## cbrt(x)
Returns 2 to the power of the given argumetn
Accepts a numeric argument and returns a Float64 number close to the cubic root of the argument.
**Syntax**
## erf(x)
```sql
exp2(x)
```
If x is non-negative, then `erf(x / σ√2)` is the probability that a random variable having a normal distribution with standard deviation σ takes the value that is separated from the expected value by more than x.
## intExp2
Example (three sigma rule):
Like `exp` but returns a UInt64.
**Syntax**
```sql
intExp2(x)
```
## log2
Returns the binary logarithm of the argument.
**Syntax**
```sql
log2(x)
```
## exp10
Returns 10 to the power of the given argumetn
**Syntax**
```sql
exp10(x)
```
## intExp10
Like `exp10` but returns a UInt64.
**Syntax**
```sql
intExp10(x)
```
## log10
Returns the decimal logarithm of the argument.
**Syntax**
```sql
log10(x)
```
## sqrt
Returns the square root of the argument.
```sql
sqrt(x)
```
## cbrt
Returns the cubic root of the argument.
```sql
cbrt(x)
```
## erf
If `x` is non-negative, then `erf(x / σ√2)` is the probability that a random variable having a normal distribution with standard deviation `σ` takes the value that is separated from the expected value by more than `x`.
**Syntax**
```sql
erf(x)
```
**Example**
(three sigma rule)
``` sql
SELECT erf(3 / sqrt(2));
```
``` text
```result
┌─erf(divide(3, sqrt(2)))─┐
│ 0.9973002039367398 │
└─────────────────────────┘
```
## erfc(x)
## erfc
Accepts a numeric argument and returns a Float64 number close to 1 - erf(x), but without loss of precision for large x values.
Returns a number close to `1 - erf(x)` without loss of precision for large x values.
## lgamma(x)
**Syntax**
The logarithm of the gamma function.
```sql
erfc(x)
```
## tgamma(x)
## lgamma
Gamma function.
Returns the logarithm of the gamma function.
## sin(x)
**Syntax**
The sine.
```sql
lgamma(x)
```
## cos(x)
## tgamma
The cosine.
Returns the gamma function.
## tan(x)
**Syntax**
The tangent.
```sql
gamma(x)
```
## asin(x)
## sin
The arc sine.
Returns the sine of the argument
## acos(x)
**Syntax**
The arc cosine.
```sql
sin(x)
```
## atan(x)
## cos
The arc tangent.
Returns the cosine of the argument.
## pow(x, y), power(x, y)
**Syntax**
Takes two numeric arguments x and y. Returns a Float64 number close to x to the power of y.
```sql
cos(x)
```
## intExp2
## tan
Accepts a numeric argument and returns a UInt64 number close to 2 to the power of x.
Returns the tangent of the argument.
## intExp10
**Syntax**
Accepts a numeric argument and returns a UInt64 number close to 10 to the power of x.
```sql
tan(x)
```
## cosh(x)
## asin
[Hyperbolic cosine](https://in.mathworks.com/help/matlab/ref/cosh.html).
Returns the arc sine of the argument.
**Syntax**
```sql
asin(x)
```
## acos
Returns the arc cosine of the argument.
**Syntax**
```sql
acos(x)
```
## atan
Returns the arc tangent of the argument.
**Syntax**
```sql
atan(x)
```
## pow
Returns `x` to the power of `y`.
**Syntax**
```sql
pow(x, y)
```
Alias: `power(x, y)`
## cosh
Returns the [hyperbolic cosine](https://in.mathworks.com/help/matlab/ref/cosh.html) of the argument.
**Syntax**
@ -134,23 +274,21 @@ Type: [Float64](../../sql-reference/data-types/float.md#float32-float64).
**Example**
Query:
``` sql
SELECT cosh(0);
```
Result:
``` text
```result
┌─cosh(0)──┐
│ 1 │
└──────────┘
```
## acosh(x)
## acosh
[Inverse hyperbolic cosine](https://www.mathworks.com/help/matlab/ref/acosh.html).
Returns the [inverse hyperbolic cosine](https://www.mathworks.com/help/matlab/ref/acosh.html).
**Syntax**
@ -170,27 +308,21 @@ Type: [Float64](../../sql-reference/data-types/float.md#float32-float64).
**Example**
Query:
``` sql
SELECT acosh(1);
```
Result:
``` text
```result
┌─acosh(1)─┐
│ 0 │
└──────────┘
```
**See Also**
## sinh
- [cosh(x)](../../sql-reference/functions/math-functions.md#coshx)
## sinh(x)
[Hyperbolic sine](https://www.mathworks.com/help/matlab/ref/sinh.html).
Returns the [hyperbolic sine](https://www.mathworks.com/help/matlab/ref/sinh.html).
**Syntax**
@ -210,23 +342,21 @@ Type: [Float64](../../sql-reference/data-types/float.md#float32-float64).
**Example**
Query:
``` sql
SELECT sinh(0);
```
Result:
``` text
```result
┌─sinh(0)──┐
│ 0 │
└──────────┘
```
## asinh(x)
## asinh
[Inverse hyperbolic sine](https://www.mathworks.com/help/matlab/ref/asinh.html).
Returns the [inverse hyperbolic sine](https://www.mathworks.com/help/matlab/ref/asinh.html).
**Syntax**
@ -246,27 +376,21 @@ Type: [Float64](../../sql-reference/data-types/float.md#float32-float64).
**Example**
Query:
``` sql
SELECT asinh(0);
```
Result:
``` text
```result
┌─asinh(0)─┐
│ 0 │
└──────────┘
```
**See Also**
## atanh
- [sinh(x)](../../sql-reference/functions/math-functions.md#sinhx)
## atanh(x)
[Inverse hyperbolic tangent](https://www.mathworks.com/help/matlab/ref/atanh.html).
Returns the [inverse hyperbolic tangent](https://www.mathworks.com/help/matlab/ref/atanh.html).
**Syntax**
@ -286,23 +410,21 @@ Type: [Float64](../../sql-reference/data-types/float.md#float32-float64).
**Example**
Query:
``` sql
SELECT atanh(0);
```
Result:
``` text
```result
┌─atanh(0)─┐
│ 0 │
└──────────┘
```
## atan2(y, x)
## atan2
The [function](https://en.wikipedia.org/wiki/Atan2) calculates the angle in the Euclidean plane, given in radians, between the positive x axis and the ray to the point `(x, y) ≠ (0, 0)`.
Returns the [atan2](https://en.wikipedia.org/wiki/Atan2) as the angle in the Euclidean plane, given in radians, between the positive x axis and the ray to the point `(x, y) ≠ (0, 0)`.
**Syntax**
@ -323,23 +445,21 @@ Type: [Float64](../../sql-reference/data-types/float.md#float32-float64).
**Example**
Query:
``` sql
SELECT atan2(1, 1);
```
Result:
``` text
```result
┌────────atan2(1, 1)─┐
│ 0.7853981633974483 │
└────────────────────┘
```
## hypot(x, y)
## hypot
Calculates the length of the hypotenuse of a right-angle triangle. The [function](https://en.wikipedia.org/wiki/Hypot) avoids problems that occur when squaring very large or very small numbers.
Returns the length of the hypotenuse of a right-angle triangle. [Hypot](https://en.wikipedia.org/wiki/Hypot) avoids problems that occur when squaring very large or very small numbers.
**Syntax**
@ -360,23 +480,21 @@ Type: [Float64](../../sql-reference/data-types/float.md#float32-float64).
**Example**
Query:
``` sql
SELECT hypot(1, 1);
```
Result:
``` text
```result
┌────────hypot(1, 1)─┐
│ 1.4142135623730951 │
└────────────────────┘
```
## log1p(x)
## log1p
Calculates `log(1+x)`. The [function](https://en.wikipedia.org/wiki/Natural_logarithm#lnp1) `log1p(x)` is more accurate than `log(1+x)` for small values of x.
Calculates `log(1+x)`. The [calculation](https://en.wikipedia.org/wiki/Natural_logarithm#lnp1) `log1p(x)` is more accurate than `log(1+x)` for small values of x.
**Syntax**
@ -396,25 +514,19 @@ Type: [Float64](../../sql-reference/data-types/float.md#float32-float64).
**Example**
Query:
``` sql
SELECT log1p(0);
```
Result:
``` text
```result
┌─log1p(0)─┐
│ 0 │
└──────────┘
```
**See Also**
- [log(x)](../../sql-reference/functions/math-functions.md#logx-lnx)
## sign(x)
## sign
Returns the sign of a real number.
@ -444,7 +556,7 @@ SELECT sign(0);
Result:
``` text
```result
┌─sign(0)─┐
│ 0 │
└─────────┘
@ -458,7 +570,7 @@ SELECT sign(1);
Result:
``` text
```result
┌─sign(1)─┐
│ 1 │
└─────────┘
@ -472,15 +584,15 @@ SELECT sign(-1);
Result:
``` text
```result
┌─sign(-1)─┐
│ -1 │
└──────────┘
```
## degrees(x)
## degrees
Converts the input value in radians to degrees.
Converts radians to degrees.
**Syntax**
@ -500,23 +612,21 @@ Type: [Float64](../../sql-reference/data-types/float.md#float32-float64).
**Example**
Query:
``` sql
SELECT degrees(3.141592653589793);
```
Result:
``` text
```result
┌─degrees(3.141592653589793)─┐
│ 180 │
└────────────────────────────┘
```
## radians(x)
## radians
Converts the input value in degrees to radians.
Converts degrees to radians.
**Syntax**
@ -536,24 +646,21 @@ Type: [Float64](../../sql-reference/data-types/float.md#float32-float64).
**Example**
Query:
``` sql
SELECT radians(180);
```
Result:
``` text
```result
┌──────radians(180)─┐
│ 3.141592653589793 │
└───────────────────┘
```
## factorial
## factorial(n)
Computes the factorial of an integer value. It works with any native integer type including UInt(8|16|32|64) and Int(8|16|32|64). The return type is UInt64.
Computes the factorial of an integer value. Works with any native integer type including UInt(8|16|32|64) and Int(8|16|32|64). The return type is UInt64.
The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20, a value of 21 or greater will cause exception throw.
@ -566,21 +673,19 @@ factorial(n)
**Example**
Query:
``` sql
SELECT factorial(10);
```
Result:
``` text
```result
┌─factorial(10)─┐
│ 3628800 │
└───────────────┘
```
## width_bucket(operand, low, high, count)
## width_bucket
Returns the number of the bucket in which `operand` falls in a histogram having `count` equal-width buckets spanning the range `low` to `high`. Returns `0` if `operand < low`, and returns `count+1` if `operand >= high`.
@ -591,20 +696,17 @@ Returns the number of the bucket in which `operand` falls in a histogram having
```sql
widthBucket(operand, low, high, count)
```
There is also a case insensitive alias called `WIDTH_BUCKET` to provide compatibility with other databases.
Alias: `WIDTH_BUCKET`
**Example**
Query:
``` sql
SELECT widthBucket(10.15, -8.6, 23, 18);
```
Result:
``` text
```result
┌─widthBucket(10.15, -8.6, 23, 18)─┐
│ 11 │
└──────────────────────────────────┘

View File

@ -1,8 +1,7 @@
---
slug: /en/sql-reference/functions/nlp-functions
sidebar_position: 67
sidebar_label: NLP
title: "[experimental] Natural Language Processing functions"
sidebar_position: 130
sidebar_label: NLP (experimental)
---
:::note

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/other-functions
sidebar_position: 67
sidebar_position: 140
sidebar_label: Other
---

View File

@ -1,58 +1,44 @@
---
slug: /en/sql-reference/functions/random-functions
sidebar_position: 51
sidebar_label: Pseudo-Random Numbers
sidebar_position: 145
sidebar_label: Random Numbers
---
# Functions for Generating Pseudo-Random Numbers
# Functions for Generating Random Numbers
All the functions accept zero arguments or one argument. If an argument is passed, it can be any type, and its value is not used for anything. The only purpose of this argument is to prevent common subexpression elimination, so that two different instances of the same function return different columns with different random numbers.
All functions in this section accept zero or one arguments. The only use of the argument (if provided) is to prevent prevent [common subexpression
elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) such that two different execution of the same random
function in a query return different random values.
Related content
- Blog: [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse)
:::note
Non-cryptographic generators of pseudo-random numbers are used.
The random numbers are generated by non-cryptographic algorithms.
:::
## rand, rand32
Returns a pseudo-random UInt32 number, evenly distributed among all UInt32-type numbers.
Returns a random UInt32 number, evenly distributed accross the range of all possible UInt32 numbers.
Uses a linear congruential generator.
## rand64
Returns a pseudo-random UInt64 number, evenly distributed among all UInt64-type numbers.
Returns a random UInt64 number, evenly distributed accross the range of all possible UInt64 numbers.
Uses a linear congruential generator.
## randCanonical
The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1).
Non-deterministic. Return type is Float64.
Returns a Float64 value, evenly distributed in [0, 1).
## randConstant
Produces a constant column with a random value.
**Syntax**
``` sql
randConstant([x])
```
**Arguments**
- `x` — [Expression](../../sql-reference/syntax.md#syntax-expressions) resulting in any of the [supported data types](../../sql-reference/data-types/index.md#data_types). The resulting value is discarded, but the expression itself if used for bypassing [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in one query. Optional parameter.
**Returned value**
- Pseudo-random number.
Type: [UInt32](../../sql-reference/data-types/int-uint.md).
Like `rand` but produces a constant column with a random value.
**Example**
Query:
``` sql
SELECT rand(), rand(1), rand(number), randConstant(), randConstant(1), randConstant(number)
FROM numbers(3)
@ -60,7 +46,7 @@ FROM numbers(3)
Result:
``` text
``` result
┌─────rand()─┬────rand(1)─┬─rand(number)─┬─randConstant()─┬─randConstant(1)─┬─randConstant(number)─┐
│ 3047369878 │ 4132449925 │ 4044508545 │ 2740811946 │ 4229401477 │ 1924032898 │
│ 2938880146 │ 1267722397 │ 4154983056 │ 2740811946 │ 4229401477 │ 1924032898 │
@ -68,17 +54,11 @@ Result:
└────────────┴────────────┴──────────────┴────────────────┴─────────────────┴──────────────────────┘
```
# Functions for Generating Random Numbers based on Distributions
:::note
These functions are available starting from 22.10.
:::
# Functions for Generating Random Numbers based on a Distribution
## randUniform
Return random number based on [continuous uniform distribution](https://en.wikipedia.org/wiki/Continuous_uniform_distribution) in a specified range from `min` to `max`.
Returns a Float64 drawn uniformly from the interval between `min` and `max` ([continuous uniform distribution](https://en.wikipedia.org/wiki/Continuous_uniform_distribution)).
**Syntax**
@ -93,21 +73,19 @@ randUniform(min, max)
**Returned value**
- Pseudo-random number.
- Random number.
Type: [Float64](/docs/en/sql-reference/data-types/float.md).
**Example**
Query:
``` sql
SELECT randUniform(5.5, 10) FROM numbers(5)
```
Result:
``` text
``` result
┌─randUniform(5.5, 10)─┐
│ 8.094978491443102 │
│ 7.3181248914450885 │
@ -117,40 +95,36 @@ Result:
└──────────────────────┘
```
## randNormal
Return random number based on [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution).
Returns a Float64 drawn from a [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution).
**Syntax**
``` sql
randNormal(meam, variance)
randNormal(mean, variance)
```
**Arguments**
- `meam` - `Float64` mean value of distribution,
- `mean` - `Float64` - mean value of distribution,
- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance).
**Returned value**
- Pseudo-random number.
- Random number.
Type: [Float64](/docs/en/sql-reference/data-types/float.md).
**Example**
Query:
``` sql
SELECT randNormal(10, 2) FROM numbers(5)
```
Result:
``` text
``` result
┌──randNormal(10, 2)─┐
│ 13.389228911709653 │
│ 8.622949707401295 │
@ -160,40 +134,36 @@ Result:
└────────────────────┘
```
## randLogNormal
Return random number based on [log-normal distribution](https://en.wikipedia.org/wiki/Log-normal_distribution).
Returns a Float64 drawn from a [log-normal distribution](https://en.wikipedia.org/wiki/Log-normal_distribution).
**Syntax**
``` sql
randLogNormal(meam, variance)
randLogNormal(mean, variance)
```
**Arguments**
- `meam` - `Float64` mean value of distribution,
- `mean` - `Float64` - mean value of distribution,
- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance).
**Returned value**
- Pseudo-random number.
- Random number.
Type: [Float64](/docs/en/sql-reference/data-types/float.md).
**Example**
Query:
``` sql
SELECT randLogNormal(100, 5) FROM numbers(5)
```
Result:
``` text
``` result
┌─randLogNormal(100, 5)─┐
│ 1.295699673937363e48 │
│ 9.719869109186684e39 │
@ -203,11 +173,9 @@ Result:
└───────────────────────┘
```
## randBinomial
Return random number based on [binomial distribution](https://en.wikipedia.org/wiki/Binomial_distribution).
Returns a UInt64 drawn from a [binomial distribution](https://en.wikipedia.org/wiki/Binomial_distribution).
**Syntax**
@ -217,26 +185,24 @@ randBinomial(experiments, probability)
**Arguments**
- `experiments` - `UInt64` number of experiments,
- `experiments` - `UInt64` - number of experiments,
- `probability` - `Float64` - probability of success in each experiment (values in `0...1` range only).
**Returned value**
- Pseudo-random number.
- Random number.
Type: [UInt64](/docs/en/sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT randBinomial(100, .75) FROM numbers(5)
```
Result:
``` text
``` result
┌─randBinomial(100, 0.75)─┐
│ 74 │
│ 78 │
@ -246,11 +212,9 @@ Result:
└─────────────────────────┘
```
## randNegativeBinomial
Return random number based on [negative binomial distribution](https://en.wikipedia.org/wiki/Negative_binomial_distribution).
Returns a UInt64 drawn from a [negative binomial distribution](https://en.wikipedia.org/wiki/Negative_binomial_distribution).
**Syntax**
@ -260,26 +224,24 @@ randNegativeBinomial(experiments, probability)
**Arguments**
- `experiments` - `UInt64` number of experiments,
- `experiments` - `UInt64` - number of experiments,
- `probability` - `Float64` - probability of failure in each experiment (values in `0...1` range only).
**Returned value**
- Pseudo-random number.
- Random number.
Type: [UInt64](/docs/en/sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT randNegativeBinomial(100, .75) FROM numbers(5)
```
Result:
``` text
``` result
┌─randNegativeBinomial(100, 0.75)─┐
│ 33 │
│ 32 │
@ -289,11 +251,9 @@ Result:
└─────────────────────────────────┘
```
## randPoisson
Return random number based on [Poisson distribution](https://en.wikipedia.org/wiki/Poisson_distribution).
Returns a UInt64 drawn from a [Poisson distribution](https://en.wikipedia.org/wiki/Poisson_distribution).
**Syntax**
@ -303,25 +263,23 @@ randPoisson(n)
**Arguments**
- `n` - `UInt64` mean number of occurrences.
- `n` - `UInt64` - mean number of occurrences.
**Returned value**
- Pseudo-random number.
- Random number.
Type: [UInt64](/docs/en/sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT randPoisson(10) FROM numbers(5)
```
Result:
``` text
``` result
┌─randPoisson(10)─┐
│ 8 │
│ 8 │
@ -331,11 +289,9 @@ Result:
└─────────────────┘
```
## randBernoulli
Return random number based on [Bernoulli distribution](https://en.wikipedia.org/wiki/Bernoulli_distribution).
Returns a UInt64 drawn from a [Bernoulli distribution](https://en.wikipedia.org/wiki/Bernoulli_distribution).
**Syntax**
@ -349,21 +305,19 @@ randBernoulli(probability)
**Returned value**
- Pseudo-random number.
- Random number.
Type: [UInt64](/docs/en/sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT randBernoulli(.75) FROM numbers(5)
```
Result:
``` text
``` result
┌─randBernoulli(0.75)─┐
│ 1 │
│ 1 │
@ -373,11 +327,9 @@ Result:
└─────────────────────┘
```
## randExponential
Return random number based on [exponential distribution](https://en.wikipedia.org/wiki/Exponential_distribution).
Returns a Float64 drawn from a [exponential distribution](https://en.wikipedia.org/wiki/Exponential_distribution).
**Syntax**
@ -387,25 +339,23 @@ randExponential(lambda)
**Arguments**
- `lambda` - `Float64` lambda value.
- `lambda` - `Float64` - lambda value.
**Returned value**
- Pseudo-random number.
- Random number.
Type: [Float64](/docs/en/sql-reference/data-types/float.md).
**Example**
Query:
``` sql
SELECT randExponential(1/10) FROM numbers(5)
```
Result:
``` text
``` result
┌─randExponential(divide(1, 10))─┐
│ 44.71628934340778 │
│ 4.211013337903262 │
@ -415,11 +365,9 @@ Result:
└────────────────────────────────┘
```
## randChiSquared
Return random number based on [Chi-square distribution](https://en.wikipedia.org/wiki/Chi-squared_distribution) - a distribution of a sum of the squares of k independent standard normal random variables.
Returns a Float64 drawn from a [Chi-square distribution](https://en.wikipedia.org/wiki/Chi-squared_distribution) - a distribution of a sum of the squares of k independent standard normal random variables.
**Syntax**
@ -429,25 +377,23 @@ randChiSquared(degree_of_freedom)
**Arguments**
- `degree_of_freedom` - `Float64` degree of freedom.
- `degree_of_freedom` - `Float64` - degree of freedom.
**Returned value**
- Pseudo-random number.
- Random number.
Type: [Float64](/docs/en/sql-reference/data-types/float.md).
**Example**
Query:
``` sql
SELECT randChiSquared(10) FROM numbers(5)
```
Result:
``` text
``` result
┌─randChiSquared(10)─┐
│ 10.015463656521543 │
│ 9.621799919882768 │
@ -457,11 +403,9 @@ Result:
└────────────────────┘
```
## randStudentT
Return random number based on [Student's t-distribution](https://en.wikipedia.org/wiki/Student%27s_t-distribution).
Returns a Float64 drawn from a [Student's t-distribution](https://en.wikipedia.org/wiki/Student%27s_t-distribution).
**Syntax**
@ -471,25 +415,23 @@ randStudentT(degree_of_freedom)
**Arguments**
- `degree_of_freedom` - `Float64` degree of freedom.
- `degree_of_freedom` - `Float64` - degree of freedom.
**Returned value**
- Pseudo-random number.
- Random number.
Type: [Float64](/docs/en/sql-reference/data-types/float.md).
**Example**
Query:
``` sql
SELECT randStudentT(10) FROM numbers(5)
```
Result:
``` text
``` result
┌─────randStudentT(10)─┐
│ 1.2217309938538725 │
│ 1.7941971681200541 │
@ -499,11 +441,9 @@ Result:
└──────────────────────┘
```
## randFisherF
Return random number based on [F-distribution](https://en.wikipedia.org/wiki/F-distribution).
Returns a Float64 drawn from a [F-distribution](https://en.wikipedia.org/wiki/F-distribution).
**Syntax**
@ -513,26 +453,24 @@ randFisherF(d1, d2)
**Arguments**
- `d1` - `Float64` d1 degree of freedom in `X = (S1 / d1) / (S2 / d2)`,
- `d2` - `Float64` d2 degree of freedom in `X = (S1 / d1) / (S2 / d2)`,
- `d1` - `Float64` - d1 degree of freedom in `X = (S1 / d1) / (S2 / d2)`,
- `d2` - `Float64` - d2 degree of freedom in `X = (S1 / d1) / (S2 / d2)`,
**Returned value**
- Pseudo-random number.
- Random number.
Type: [Float64](/docs/en/sql-reference/data-types/float.md).
**Example**
Query:
``` sql
SELECT randFisherF(10, 3) FROM numbers(5)
```
Result:
``` text
``` result
┌──randFisherF(10, 3)─┐
│ 7.286287504216609 │
│ 0.26590779413050386 │
@ -542,35 +480,61 @@ Result:
└─────────────────────┘
```
# Random Functions for Working with Strings
# Functions for Generating Random Strings
## randomString
Returns a random String of specified `length`. Not all characters may be printable.
**Syntax**
```sql
randomString(length)
```
## randomFixedString
Like `randomString` but returns a FixedString.
## randomPrintableASCII
Returns a random String of specified `length`. All characters are printable.
**Syntax**
```sql
randomPrintableASCII(length)
```
## randomStringUTF8
Returns a random String containing `length` many UTF8 codepoints. Not all characters may be printable
**Syntax**
```sql
randomStringUTF8(length)
```
## fuzzBits
**Syntax**
``` sql
fuzzBits([s], [prob])
```
Inverts the bits of String or FixedString `s`, each with probability `prob`.
Inverts bits of `s`, each with probability `prob`.
**Syntax**
``` sql
fuzzBits(s, prob)
```
**Arguments**
- `s` - `String` or `FixedString`
- `prob` - constant `Float32/64`
**Returned value**
Fuzzed string with same as s type.
Fuzzed string with same type as `s`.
**Example**
@ -581,13 +545,10 @@ FROM numbers(3)
Result:
``` text
``` result
┌─fuzzBits(materialize('abacaba'), 0.1)─┐
│ abaaaja │
│ a*cjab+ │
│ aeca2A │
└───────────────────────────────────────┘
```
## Related content
- Blog: [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse)

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/rounding-functions
sidebar_position: 45
sidebar_position: 155
sidebar_label: Rounding
---

View File

@ -1,14 +1,14 @@
---
slug: /en/sql-reference/functions/splitting-merging-functions
sidebar_position: 47
sidebar_label: Splitting and Merging Strings and Arrays
sidebar_position: 165
sidebar_label: Splitting Strings
---
# Functions for Splitting and Merging Strings and Arrays
# Functions for Splitting Strings
## splitByChar(separator, s[, max_substrings])
## splitByChar
Splits a string into substrings separated by a specified character. It uses a constant string `separator` which consists of exactly one character.
Splits a string into substrings separated by a specified character. Uses a constant string `separator` which consists of exactly one character.
Returns an array of selected substrings. Empty substrings may be selected if the separator occurs at the beginning or end of the string, or if there are multiple consecutive separators.
**Syntax**
@ -39,13 +39,15 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere
SELECT splitByChar(',', '1,2,3,abcde');
```
Result:
``` text
┌─splitByChar(',', '1,2,3,abcde')─┐
│ ['1','2','3','abcde'] │
└─────────────────────────────────┘
```
## splitByString(separator, s[, max_substrings])
## splitByString
Splits a string into substrings separated by a string. It uses a constant string `separator` of multiple characters as the separator. If the string `separator` is empty, it will split the string `s` into an array of single characters.
@ -78,6 +80,8 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere
SELECT splitByString(', ', '1, 2 3, 4,5, abcde');
```
Result:
``` text
┌─splitByString(', ', '1, 2 3, 4,5, abcde')─┐
│ ['1','2 3','4,5','abcde'] │
@ -88,13 +92,15 @@ SELECT splitByString(', ', '1, 2 3, 4,5, abcde');
SELECT splitByString('', 'abcde');
```
Result:
``` text
┌─splitByString('', 'abcde')─┐
│ ['a','b','c','d','e'] │
└────────────────────────────┘
```
## splitByRegexp(regexp, s[, max_substrings])
## splitByRegexp
Splits a string into substrings separated by a regular expression. It uses a regular expression string `regexp` as the separator. If the `regexp` is empty, it will split the string `s` into an array of single characters. If no match is found for this regular expression, the string `s` won't be split.
@ -123,8 +129,6 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere
**Example**
Query:
``` sql
SELECT splitByRegexp('\\d+', 'a12bc23de345f');
```
@ -137,8 +141,6 @@ Result:
└────────────────────────────────────────┘
```
Query:
``` sql
SELECT splitByRegexp('', 'abcde');
```
@ -151,7 +153,7 @@ Result:
└────────────────────────────┘
```
## splitByWhitespace(s[, max_substrings])
## splitByWhitespace
Splits a string into substrings separated by whitespace characters.
Returns an array of selected substrings.
@ -180,13 +182,15 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere
SELECT splitByWhitespace(' 1! a, b. ');
```
Result:
``` text
┌─splitByWhitespace(' 1! a, b. ')─┐
│ ['1!','a,','b.'] │
└─────────────────────────────────────┘
```
## splitByNonAlpha(s[, max_substrings])
## splitByNonAlpha
Splits a string into substrings separated by whitespace and punctuation characters.
Returns an array of selected substrings.
@ -221,23 +225,32 @@ SELECT splitByNonAlpha(' 1! a, b. ');
└───────────────────────────────────┘
```
## arrayStringConcat(arr\[, separator\])
## arrayStringConcat
Concatenates string representations of values listed in the array with the separator. `separator` is an optional parameter: a constant string, set to an empty string by default.
Returns the string.
**Syntax**
```sql
arrayStringConcat(arr\[, separator\])
```
**Example**
``` sql
SELECT arrayStringConcat(['12/05/2021', '12:50:00'], ' ') AS DateString;
```
Result:
```text
┌─DateString──────────┐
│ 12/05/2021 12:50:00 │
└─────────────────────┘
```
## alphaTokens(s[, max_substrings]), splitByAlpha(s[, max_substrings])
## alphaTokens
Selects substrings of consecutive bytes from the ranges a-z and A-Z.Returns an array of substrings.
@ -245,9 +258,10 @@ Selects substrings of consecutive bytes from the ranges a-z and A-Z.Returns an a
``` sql
alphaTokens(s[, max_substrings]))
splitByAlpha(s[, max_substrings])
```
Alias: `splitByAlpha`
**Arguments**
- `s` — The string to split. [String](../../sql-reference/data-types/string.md).
@ -271,7 +285,7 @@ SELECT alphaTokens('abca1abc');
└─────────────────────────┘
```
## extractAllGroups(text, regexp)
## extractAllGroups
Extracts all groups from non-overlapping substrings matched by a regular expression.
@ -296,8 +310,6 @@ Type: [Array](../data-types/array.md).
**Example**
Query:
``` sql
SELECT extractAllGroups('abc=123, 8="hkl"', '("[^"]+"|\\w+)=("[^"]+"|\\w+)');
```
@ -312,7 +324,7 @@ Result:
## ngrams
Splits the UTF-8 string into n-grams of `ngramsize` symbols.
Splits a UTF-8 string into n-grams of `ngramsize` symbols.
**Syntax**
@ -333,8 +345,6 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere
**Example**
Query:
``` sql
SELECT ngrams('ClickHouse', 3);
```
@ -363,8 +373,6 @@ Type: [Array](../data-types/array.md).
**Example**
Query:
``` sql
SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens;
```

File diff suppressed because it is too large Load Diff

View File

@ -1,35 +1,54 @@
---
slug: /en/sql-reference/functions/string-replace-functions
sidebar_position: 42
sidebar_position: 150
sidebar_label: Replacing in Strings
---
# Functions for Searching and Replacing in Strings
# Functions for Replacing in Strings
:::note
Functions for [searching](../../sql-reference/functions/string-search-functions.md) and [other manipulations with strings](../../sql-reference/functions/string-functions.md) are described separately.
:::
[General strings functions](string-functions.md) and [functions for searchin in strings](string-search-functions.md) are described separately.
## replaceOne(haystack, pattern, replacement)
## replaceOne
Replaces the first occurrence of the substring pattern (if it exists) in haystack by the replacement string.
Replaces the first occurrence of the substring `pattern` in `haystack` by the `replacement` string.
## replaceAll(haystack, pattern, replacement), replace(haystack, pattern, replacement)
**Syntax**
Replaces all occurrences of the substring pattern in haystack by the replacement string.
```sql
replaceOne(haystack, pattern, replacement)
```
## replaceAll
Replaces all occurrences of the substring `pattern` in `haystack` by the `replacement` string.
**Syntax**
```sql
replaceAll(haystack, pattern, replacement)
```
Alias: `replace`.
## replaceRegexpOne(haystack, pattern, replacement)
## replaceRegexpOne
Replaces the first occurrence of the substring matching the regular expression pattern in haystack by the replacement string.
pattern must be a [re2 regular expression](https://github.com/google/re2/wiki/Syntax).
replacement must be a plain string or a string containing substitutions `\0-\9`.
Replaces the first occurrence of the substring matching the regular expression `pattern` (in [re2 syntax](https://github.com/google/re2/wiki/Syntax)) in `haystack` by the `replacement` string.
`replacement` can containing substitutions `\0-\9`.
Substitutions `\1-\9` correspond to the 1st to 9th capturing group (submatch), substitution `\0` corresponds to the entire match.
To use a verbatim `\` character in the pattern or replacement string, escape it using `\`.
Also keep in mind that string literals require an extra escaping.
Example 1. Converting ISO dates to American format:
To use a verbatim `\` character in the `pattern` or `replacement` strings, escape it using `\`.
Also keep in mind that string literals require extra escaping.
**Syntax**
```sql
replaceRegexpOne(haystack, pattern, replacement)
```
**Example**
Converting ISO dates to American format:
``` sql
SELECT DISTINCT
@ -40,6 +59,8 @@ LIMIT 7
FORMAT TabSeparated
```
Result:
``` text
2014-03-17 03/17/2014
2014-03-18 03/18/2014
@ -50,83 +71,91 @@ FORMAT TabSeparated
2014-03-23 03/23/2014
```
Example 2. Copying a string ten times:
Copying a string ten times:
``` sql
SELECT replaceRegexpOne('Hello, World!', '.*', '\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0') AS res
```
Result:
``` text
┌─res────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
│ Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World! │
└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
```
## replaceRegexpAll(haystack, pattern, replacement)
## replaceRegexpAll
Like replaceRegexpOne, but replaces all occurrences of the pattern. Example:
Like `replaceRegexpOne` but replaces all occurrences of the pattern.
Alias: `REGEXP_REPLACE`.
**Example**
``` sql
SELECT replaceRegexpAll('Hello, World!', '.', '\\0\\0') AS res
```
Result:
``` text
┌─res────────────────────────┐
│ HHeelllloo,, WWoorrlldd!! │
└────────────────────────────┘
```
As an exception, if a regular expression worked on an empty substring, the replacement is not made more than once.
Example:
As an exception, if a regular expression worked on an empty substring, the replacement is not made more than once, e.g.:
``` sql
SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res
```
Result:
``` text
┌─res─────────────────┐
│ here: Hello, World! │
└─────────────────────┘
```
Alias: `REGEXP_REPLACE`.
## regexpQuoteMeta
## regexpQuoteMeta(s)
Adds a backslash before these characters with special meaning in regular expressions: `\0`, `\\`, `|`, `(`, `)`, `^`, `$`, `.`, `[`, `]`, `?`, `*`, `+`, `{`, `:`, `-`.
The function adds a backslash before some predefined characters in the string.
Predefined characters: `\0`, `\\`, `|`, `(`, `)`, `^`, `$`, `.`, `[`, `]`, `?`, `*`, `+`, `{`, `:`, `-`.
This implementation slightly differs from re2::RE2::QuoteMeta. It escapes zero byte as `\0` instead of `\x00` and it escapes only required characters.
For more information, see the link: [RE2](https://github.com/google/re2/blob/master/re2/re2.cc#L473)
For more information, see [RE2](https://github.com/google/re2/blob/master/re2/re2.cc#L473)
**Syntax**
## translate(s, from, to)
```sql
regexpQuoteMeta(s)
```
The function replaces characters in the string s in accordance with one-to-one character mapping defined by from and to strings. from and to must be constant ASCII strings of the same size. Non-ASCII characters in the original string are not modified.
## translate
Example:
Replaces characters in the string `s` using a one-to-one character mapping defined by `from` and `to` strings. `from` and `to` must be constant ASCII strings of the same size. Non-ASCII characters in the original string are not modified.
**Syntax**
```sql
translate(s, from, to)
```
**Example**
``` sql
SELECT translate('Hello, World!', 'delor', 'DELOR') AS res
```
Result:
``` text
┌─res───────────┐
│ HELLO, WORLD! │
└───────────────┘
```
## translateUTF8(string, from, to)
## translateUTF8
Similar to previous function, but works with UTF-8 arguments. from and to must be valid constant UTF-8 strings of the same size.
Example:
``` sql
SELECT translateUTF8('Hélló, Wórld¡', 'óé¡', 'oe!') AS res
```
``` text
┌─res───────────┐
│ Hello, World! │
└───────────────┘
```
Like [translate](#translate) but assumes `s`, `from` and `to` are UTF-8 encoded strings.

View File

@ -1,24 +1,26 @@
---
slug: /en/sql-reference/functions/string-search-functions
sidebar_position: 41
sidebar_position: 160
sidebar_label: Searching in Strings
---
# Functions for Searching in Strings
The search is case-sensitive by default in all these functions. There are separate variants for case insensitive search.
All functions in this section search by default case-sensitively. Case-insensitive search is usually provided by separate function variants.
Note that case-insensitive search follows the lowercase-uppercase rules of the English language. E.g. Uppercased `i` in English language is
`I` whereas in Turkish language it is `İ` - results for languages other than English may be unexpected.
:::note
Functions for [replacing](../../sql-reference/functions/string-replace-functions.md) and [other manipulations with strings](../../sql-reference/functions/string-functions.md) are described separately.
:::
Functions in this section also assume that the searched string and the search string are single-byte encoded text. If this assumption is
violated, no exception is thrown and results are undefined. Search with UTF-8 encoded strings is usually provided by separate function
variants. Likewise, if a UTF-8 function variant is used and the input strings are not UTF-8 encoded text, no exception is thrown and the
results are undefined. Note that no automatic Unicode normalization is performed, you can use the
[normalizeUTF8*()](https://clickhouse.com/docs/en/sql-reference/functions/string-functions/) functions for that.
## position(haystack, needle), locate(haystack, needle)
[General strings functions](string-functions.md) and [functions for replacing in strings](string-replace-functions.md) are described separately.
Searches for the substring `needle` in the string `haystack`.
## position
Returns the position (in bytes) of the found substring in the string, starting from 1.
For a case-insensitive search, use the function [positionCaseInsensitive](#positioncaseinsensitive).
Returns the position (in bytes, starting at 1) of a substring `needle` in a string `haystack`.
**Syntax**
@ -26,35 +28,33 @@ For a case-insensitive search, use the function [positionCaseInsensitive](#posit
position(haystack, needle[, start_pos])
```
``` sql
position(needle IN haystack)
```
Alias: `locate(haystack, needle[, start_pos])`.
:::note
Syntax of `position(needle IN haystack)` provides SQL-compatibility, the function works the same way as to `position(haystack, needle)`.
:::
Alias:
- `position(needle IN haystack)`
- `locate(haystack, needle[, start_pos])`.
**Arguments**
- `haystack` — String, in which substring will to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `start_pos` Position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md). Optional.
- `start_pos` Position (1-based) in `haystack` at which the search starts. [UInt](../../sql-reference/data-types/int-uint.md). Optional.
**Returned values**
- Starting position in bytes (counting from 1), if substring was found.
- Starting position in bytes and counting from 1, if the substring was found.
- 0, if the substring was not found.
If substring `needle` is empty, these rules apply:
- if no `start_pos` was specified: return `1`
- if `start_pos = 0`: return `1`
- if `start_pos >= 1` and `start_pos <= length(haystack) + 1`: return `start_pos`
- otherwise: return `0`
The same rules also apply to functions `positionCaseInsensitive`, `positionUTF8` and `positionCaseInsensitiveUTF8`
Type: `Integer`.
**Examples**
The phrase “Hello, world!” contains a set of bytes representing a single-byte encoded text. The function returns some expected result:
Query:
``` sql
SELECT position('Hello, world!', '!');
```
@ -67,6 +67,8 @@ Result:
└────────────────────────────────┘
```
Example with `start_pos` argument:
``` sql
SELECT
position('Hello, world!', 'o', 1),
@ -79,29 +81,21 @@ SELECT
└───────────────────────────────────┴───────────────────────────────────┘
```
The same phrase in Russian contains characters which cant be represented using a single byte. The function returns some unexpected result (use [positionUTF8](#positionutf8) function for multi-byte encoded text):
Example for `needle IN haystack` syntax:
Query:
``` sql
SELECT position('Привет, мир!', '!');
```sql
SELECT 6 = position('/' IN s) FROM (SELECT 'Hello/World' AS s);
```
Result:
``` text
┌─position('Привет, мир!', '!')─┐
21 │
└───────────────────────────────
```text
┌─equals(6, position(s, '/'))─┐
│ 1 │
└─────────────────────────────┘
```
If argument `needle` is empty the following rules apply:
- if no `start_pos` was specified: return `1`
- if `start_pos = 0`: return `1`
- if `start_pos >= 1` and `start_pos <= length(haystack) + 1`: return `start_pos`
- otherwise: return `0`
The same rules also apply to functions `positionCaseInsensitive`, `positionUTF8` and `positionCaseInsensitiveUTF8`
Examples with empty `needle` substring:
``` sql
SELECT
@ -120,223 +114,59 @@ SELECT
└─────────────────────┴────────────────────────┴────────────────────────┴────────────────────────┴────────────────────────┴────────────────────────┴────────────────────────┘
```
**Examples for POSITION(needle IN haystack) syntax**
Query:
```sql
SELECT 3 = position('c' IN 'abc');
```
Result:
```text
┌─equals(3, position('abc', 'c'))─┐
│ 1 │
└─────────────────────────────────┘
```
Query:
```sql
SELECT 6 = position('/' IN s) FROM (SELECT 'Hello/World' AS s);
```
Result:
```text
┌─equals(6, position(s, '/'))─┐
│ 1 │
└─────────────────────────────┘
```
## positionCaseInsensitive
The same as [position](#position) returns the position (in bytes) of the found substring in the string, starting from 1. Use the function for a case-insensitive search.
Works under the assumption that the string contains a set of bytes representing a single-byte encoded text. If this assumption is not met and a character cant be represented using a single byte, the function does not throw an exception and returns some unexpected result. If character can be represented using two bytes, it will use two bytes and so on.
**Syntax**
``` sql
positionCaseInsensitive(haystack, needle[, start_pos])
```
**Arguments**
- `haystack` — String, in which substring will to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `start_pos` — Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md).
**Returned values**
- Starting position in bytes (counting from 1), if substring was found.
- 0, if the substring was not found.
Type: `Integer`.
**Example**
Query:
``` sql
SELECT positionCaseInsensitive('Hello, world!', 'hello');
```
Result:
``` text
┌─positionCaseInsensitive('Hello, world!', 'hello')─┐
│ 1 │
└───────────────────────────────────────────────────┘
```
Like [position](#position) but searches case-insensitively.
## positionUTF8
Returns the position (in Unicode points) of the found substring in the string, starting from 1.
Works under the assumption that the string contains a set of bytes representing a UTF-8 encoded text. If this assumption is not met, the function does not throw an exception and returns some unexpected result. If character can be represented using two Unicode points, it will use two and so on.
For a case-insensitive search, use the function [positionCaseInsensitiveUTF8](#positioncaseinsensitiveutf8).
**Syntax**
``` sql
positionUTF8(haystack, needle[, start_pos])
```
**Arguments**
- `haystack` — String, in which substring will to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `start_pos` — Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md)
**Returned values**
- Starting position in Unicode points (counting from 1), if substring was found.
- 0, if the substring was not found.
Type: `Integer`.
Like [position](#position) but assumes `haystack` and `needle` are UTF-8 encoded strings.
**Examples**
The phrase “Hello, world!” in Russian contains a set of Unicode points representing a single-point encoded text. The function returns some expected result:
Query:
Function `positionUTF8` correctly counts character `ö` (represented by two points) as a single Unicode codepoint:
``` sql
SELECT positionUTF8('Привет, мир!', '!');
SELECT positionUTF8('Motörhead', 'r');
```
Result:
``` text
┌─positionUTF8('Привет, мир!', '!')─┐
│ 12 │
└───────────────────────────────────┘
```
The phrase “Salut, étudiante!”, where character `é` can be represented using a one point (`U+00E9`) or two points (`U+0065U+0301`) the function can be returned some unexpected result:
Query for the letter `é`, which is represented one Unicode point `U+00E9`:
``` sql
SELECT positionUTF8('Salut, étudiante!', '!');
```
Result:
``` text
┌─positionUTF8('Salut, étudiante!', '!')─┐
│ 17 │
└────────────────────────────────────────┘
```
Query for the letter `é`, which is represented two Unicode points `U+0065U+0301`:
``` sql
SELECT positionUTF8('Salut, étudiante!', '!');
```
Result:
``` text
┌─positionUTF8('Salut, étudiante!', '!')─┐
│ 18 │
└────────────────────────────────────────┘
┌─position('Motörhead', 'r')─┐
│ 5 │
└────────────────────────────┘
```
## positionCaseInsensitiveUTF8
The same as [positionUTF8](#positionutf8), but is case-insensitive. Returns the position (in Unicode points) of the found substring in the string, starting from 1.
Works under the assumption that the string contains a set of bytes representing a UTF-8 encoded text. If this assumption is not met, the function does not throw an exception and returns some unexpected result. If character can be represented using two Unicode points, it will use two and so on.
**Syntax**
``` sql
positionCaseInsensitiveUTF8(haystack, needle[, start_pos])
```
**Arguments**
- `haystack` — String, in which substring will to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `start_pos` — Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md)
**Returned value**
- Starting position in Unicode points (counting from 1), if substring was found.
- 0, if the substring was not found.
Type: `Integer`.
**Example**
Query:
``` sql
SELECT positionCaseInsensitiveUTF8('Привет, мир!', 'Мир');
```
Result:
``` text
┌─positionCaseInsensitiveUTF8('Привет, мир!', 'Мир')─┐
│ 9 │
└────────────────────────────────────────────────────┘
```
Like [positionUTF8](#positionutf8) but searches case-insensitively.
## multiSearchAllPositions
The same as [position](../../sql-reference/functions/string-search-functions.md#position) but returns `Array` of positions (in bytes) of the found corresponding substrings in the string. Positions are indexed starting from 1.
Like [position](#position) but returns an array of positions (in bytes, starting at 1) for multiple `needle` substrings in a `haystack` string.
The search is performed on sequences of bytes without respect to string encoding and collation.
- For case-insensitive ASCII search, use the function `multiSearchAllPositionsCaseInsensitive`.
- For search in UTF-8, use the function [multiSearchAllPositionsUTF8](#multiSearchAllPositionsUTF8).
- For case-insensitive UTF-8 search, use the function multiSearchAllPositionsCaseInsensitiveUTF8.
:::note
All `multiSearch*()` functions only support up to 2<sup>8</sup> needles.
:::
**Syntax**
``` sql
multiSearchAllPositions(haystack, [needle1, needle2, ..., needlen])
multiSearchAllPositions(haystack, [needle1, needle2, ..., needleN])
```
**Arguments**
- `haystack` — String, in which substring will to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substrings to be searched. Array
**Returned values**
- Array of starting positions in bytes (counting from 1), if the corresponding substring was found and 0 if not found.
- Array of the starting position in bytes and counting from 1 (if the substring was found) or 0 (if the substring was not found)
**Example**
Query:
``` sql
SELECT multiSearchAllPositions('Hello, World!', ['hello', '!', 'world']);
```
@ -351,103 +181,172 @@ Result:
## multiSearchAllPositionsUTF8
See `multiSearchAllPositions`.
Like [multiSearchAllPositions](#multiSearchAllPositions) but assumes `haystack` and the `needle`-s are UTF-8 encoded strings.
## multiSearchFirstPosition(haystack, \[needle<sub>1</sub>, needle<sub>2</sub>, …, needle<sub>n</sub>\])
## multiSearchFirstPosition
The same as `position` but returns the leftmost offset of the string `haystack` that is matched to some of the needles.
Like `position` but returns the leftmost offset in a `haystack` string which matches any of multiple `needle` strings.
For a case-insensitive search or/and in UTF-8 format use functions `multiSearchFirstPositionCaseInsensitive, multiSearchFirstPositionUTF8, multiSearchFirstPositionCaseInsensitiveUTF8`.
Functions `multiSearchFirstPositionCaseInsensitive`, `multiSearchFirstPositionUTF8` and `multiSearchFirstPositionCaseInsensitiveUTF8` provide case-insensitive and/or UTF-8 variants of this function.
## multiSearchFirstIndex(haystack, \[needle<sub>1</sub>, needle<sub>2</sub>, …, needle<sub>n</sub>\])
**Syntax**
```sql
multiSearchFirstPosition(haystack, \[needle<sub>1</sub>, needle<sub>2</sub>, …, needle<sub>n</sub>\])
```
## multiSearchFirstIndex
Returns the index `i` (starting from 1) of the leftmost found needle<sub>i</sub> in the string `haystack` and 0 otherwise.
For a case-insensitive search or/and in UTF-8 format use functions `multiSearchFirstIndexCaseInsensitive, multiSearchFirstIndexUTF8, multiSearchFirstIndexCaseInsensitiveUTF8`.
Functions `multiSearchFirstIndexCaseInsensitive`, `multiSearchFirstIndexUTF8` and `multiSearchFirstIndexCaseInsensitiveUTF8` provide case-insensitive and/or UTF-8 variants of this function.
## multiSearchAny(haystack, \[needle<sub>1</sub>, needle<sub>2</sub>, …, needle<sub>n</sub>\])
**Syntax**
```sql
multiSearchFirstIndex(haystack, \[needle<sub>1</sub>, needle<sub>2</sub>, …, needle<sub>n</sub>\])
```
## multiSearchAny
Returns 1, if at least one string needle<sub>i</sub> matches the string `haystack` and 0 otherwise.
For a case-insensitive search or/and in UTF-8 format use functions `multiSearchAnyCaseInsensitive, multiSearchAnyUTF8, multiSearchAnyCaseInsensitiveUTF8`.
Functions `multiSearchAnyCaseInsensitive`, `multiSearchAnyUTF8` and `multiSearchAnyCaseInsensitiveUTF8` provide case-insensitive and/or UTF-8 variants of this function.
:::note
In all `multiSearch*` functions the number of needles should be less than 2<sup>8</sup> because of implementation specification.
:::
**Syntax**
## match(haystack, pattern), haystack REGEXP pattern operator
```sql
multiSearchAny(haystack, \[needle<sub>1</sub>, needle<sub>2</sub>, …, needle<sub>n</sub>\])
```
Checks whether string `haystack` matches the regular expression `pattern`. The pattern is an [re2 regular expression](https://github.com/google/re2/wiki/Syntax) which has a more limited syntax than Perl regular expressions.
## match
Returns 1 in case of a match, and 0 otherwise.
Returns whether string `haystack` matches the regular expression `pattern` in [re2 regular syntax](https://github.com/google/re2/wiki/Syntax).
Matching is based on UTF-8, e.g. `.` matches the Unicode code point `¥` which is represented in UTF-8 using two bytes. The regular expression must not contain null bytes.
If the haystack or the pattern are not valid UTF-8, then the behavior is undefined.
No automatic Unicode normalization is performed, you can use the [normalizeUTF8*()](https://clickhouse.com/docs/en/sql-reference/functions/string-functions/) functions for that.
Matching is based on UTF-8, e.g. `.` matches the Unicode code point `¥` which is represented in UTF-8 using two bytes. The regular
expression must not contain null bytes. If the haystack or the pattern are not valid UTF-8, then the behavior is undefined.
Unlike re2's default behavior, `.` matches line breaks. To disable this, prepend the pattern with `(?-s)`.
For patterns to search for substrings in a string, it is better to use functions [like](#like) or [position](#position) since they work much faster.
If you only want to search substrings in a string, you can use functions [like](#like) or [position](#position) instead - they work much faster than this function.
## multiMatchAny(haystack, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
**Syntax**
The same as `match`, but returns 0 if none of the regular expressions are matched and 1 if any of the patterns matches. For patterns to search substrings in a string, it is better to use `multiSearchAny` since it works much faster.
```sql
match(haystack, pattern)
```
Alias: `haystack REGEXP pattern operator`
## multiMatchAny
Like `match` but returns 1 if at least one of the patterns match and 0 otherwise.
:::note
Functions `multiMatchAny`, `multiMatchAnyIndex`, `multiMatchAllIndices` and their fuzzy equivalents (`multiFuzzyMatchAny`,
`multiFuzzyMatchAnyIndex`, `multiFuzzyMatchAllIndices`) use the (Vectorscan)[https://github.com/VectorCamp/vectorscan] library. As such,
they are only enabled if ClickHouse is compiled with support for vectorscan.
Functions in the `multi[Fuzzy]Match*()` family use the the (Vectorscan)[https://github.com/VectorCamp/vectorscan] library. As such, they are only enabled if ClickHouse is compiled with support for vectorscan.
To turn off all functions that use hyperscan, use setting `SET allow_hyperscan = 0;`.
Due to restrictions of vectorscan, the length of the `haystack` string must be less than 2<sup>32</sup> bytes.
Hyperscan is generally vulnerable to regular expression denial of service (ReDoS) attacks (e.g. see
(here)[https://www.usenix.org/conference/usenixsecurity22/presentation/turonova], (here)[https://doi.org/10.1007/s10664-021-10033-1] and
(here)[ https://doi.org/10.1145/3236024.3236027]. Users are adviced to check the provided patterns carefully.
(here)[https://doi.org/10.1145/3236024.3236027]. Users are adviced to check the provided patterns carefully.
:::
## multiMatchAnyIndex(haystack, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
If you only want to search multiple substrings in a string, you can use function [multiSearchAny](#multisearchany) instead - it works much faster than this function.
The same as `multiMatchAny`, but returns any index that matches the haystack.
**Syntax**
## multiMatchAllIndices(haystack, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
```sql
multiMatchAny(haystack, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
```
The same as `multiMatchAny`, but returns the array of all indices that match the haystack in any order.
## multiMatchAnyIndex
## multiFuzzyMatchAny(haystack, distance, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
Like `multiMatchAny` but returns any index that matches the haystack.
The same as `multiMatchAny`, but returns 1 if any pattern matches the haystack within a constant [edit distance](https://en.wikipedia.org/wiki/Edit_distance). This function relies on the experimental feature of [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) library, and can be slow for some corner cases. The performance depends on the edit distance value and patterns used, but it's always more expensive compared to a non-fuzzy variants.
**Syntax**
## multiFuzzyMatchAnyIndex(haystack, distance, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
```sql
multiMatchAnyIndex(haystack, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
```
The same as `multiFuzzyMatchAny`, but returns any index that matches the haystack within a constant edit distance.
## multiMatchAllIndices
## multiFuzzyMatchAllIndices(haystack, distance, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
Like `multiMatchAny` but returns the array of all indices that match the haystack in any order.
The same as `multiFuzzyMatchAny`, but returns the array of all indices in any order that match the haystack within a constant edit distance.
**Syntax**
```sql
multiMatchAllIndices(haystack, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
```
## multiFuzzyMatchAny
Like `multiMatchAny` but returns 1 if any pattern matches the haystack within a constant [edit distance](https://en.wikipedia.org/wiki/Edit_distance). This function relies on the experimental feature of [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) library, and can be slow for some corner cases. The performance depends on the edit distance value and patterns used, but it's always more expensive compared to a non-fuzzy variants.
:::note
`multiFuzzyMatch*` functions do not support UTF-8 regular expressions, and such expressions are treated as bytes because of hyperscan restriction.
`multiFuzzyMatch*()` function family do not support UTF-8 regular expressions (it threats them as a sequence of bytes) due to restrictions of hyperscan.
:::
:::note
To turn off all functions that use hyperscan, use setting `SET allow_hyperscan = 0;`.
:::
**Syntax**
## extract(haystack, pattern)
```sql
multiFuzzyMatchAny(haystack, distance, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
```
Extracts a fragment of a string using a regular expression. If haystack does not match the pattern regex, an empty string is returned. If the regex does not contain subpatterns, it takes the fragment that matches the entire regex. Otherwise, it takes the fragment that matches the first subpattern.
## multiFuzzyMatchAnyIndex
## extractAll(haystack, pattern)
Like `multiFuzzyMatchAny` but returns any index that matches the haystack within a constant edit distance.
Extracts all the fragments of a string using a regular expression. If haystack does not match the pattern regex, an empty string is returned. Returns an array of strings consisting of all matches to the regex. In general, the behavior is the same as the extract function (it takes the first subpattern, or the entire expression if there isnt a subpattern).
**Syntax**
```sql
multiFuzzyMatchAnyIndex(haystack, distance, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
```
## multiFuzzyMatchAllIndices
Like `multiFuzzyMatchAny` but returns the array of all indices in any order that match the haystack within a constant edit distance.
**Syntax**
```sql
multiFuzzyMatchAllIndices(haystack, distance, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
```
## extract
Extracts a fragment of a string using a regular expression. If `haystack` does not match the `pattern` regex, an empty string is returned.
For regex without subpatterns, the function uses the fragment that matches the entire regex. Otherwise, it uses the fragment that matches the first subpattern.
**Syntax**
```sql
extract(haystack, pattern)
```
## extractAll
Extracts all fragments of a string using a regular expression. If `haystack` does not match the `pattern` regex, an empty string is returned.
Returns an array of strings consisting of all matches of the regex.
The behavior with respect to subpatterns is the same as in function `extract`.
**Syntax**
```sql
extractAll(haystack, pattern)
```
## extractAllGroupsHorizontal
Matches all groups of the `haystack` string using the `pattern` regular expression. Returns an array of arrays, where the first array includes all fragments matching the first group, the second array - matching the second group, etc.
:::note
`extractAllGroupsHorizontal` function is slower than [extractAllGroupsVertical](#extractallgroups-vertical).
:::
This function is slower than [extractAllGroupsVertical](#extractallgroups-vertical).
**Syntax**
@ -468,8 +367,6 @@ If `haystack` does not match the `pattern` regex, an array of empty arrays is re
**Example**
Query:
``` sql
SELECT extractAllGroupsHorizontal('abc=111, def=222, ghi=333', '("[^"]+"|\\w+)=("[^"]+"|\\w+)');
```
@ -482,10 +379,6 @@ Result:
└──────────────────────────────────────────────────────────────────────────────────────────┘
```
**See Also**
- [extractAllGroupsVertical](#extractallgroups-vertical)
## extractAllGroupsVertical
Matches all groups of the `haystack` string using the `pattern` regular expression. Returns an array of arrays, where each array includes matching fragments from every group. Fragments are grouped in order of appearance in the `haystack`.
@ -509,8 +402,6 @@ If `haystack` does not match the `pattern` regex, an empty array is returned.
**Example**
Query:
``` sql
SELECT extractAllGroupsVertical('abc=111, def=222, ghi=333', '("[^"]+"|\\w+)=("[^"]+"|\\w+)');
```
@ -523,119 +414,88 @@ Result:
└────────────────────────────────────────────────────────────────────────────────────────┘
```
**See Also**
## like
- [extractAllGroupsHorizontal](#extractallgroups-horizontal)
Returns whether string `haystack` matches the LIKE expression `pattern`.
## like(haystack, pattern), haystack LIKE pattern operator
Checks whether a string matches a LIKE expression.
A LIKE expression contains a mix of normal characters and the following metasymbols:
A LIKE expression can contain normal characters and the following metasymbols:
- `%` indicates an arbitrary number of arbitrary characters (including zero characters).
- `_` indicates a single arbitrary character.
- `\` is for escaping literals `%`, `_` and `\`.
Matching is based on UTF-8, e.g. `_` matches the Unicode code point `¥` which is represented in UTF-8 using two bytes.
If the haystack or the pattern are not valid UTF-8, then the behavior is undefined.
If the haystack or the LIKE expression are not valid UTF-8, the behavior is undefined.
No automatic Unicode normalization is performed, you can use the [normalizeUTF8*()](https://clickhouse.com/docs/en/sql-reference/functions/string-functions/) functions for that.
To match against literals `%`, `_` and `/` (which are LIKE metacharacters), prepend them with a backslash, i.e. `\%`, `\_` and `\\`.
The backslash loses its special meaning, i.e. is interpreted literally, if it prepends a character different than `%`, `_` or `\`.
To match against literal `%`, `_` and `/` (which are LIKE metacharacters), prepend them with a backslash: `\%`, `\_` and `\\`.
The backslash loses its special meaning (i.e. is interpreted literally) if it prepends a character different than `%`, `_` or `\`.
Note that ClickHouse requires backslashes in strings [to be quoted as well](../syntax.md#string), so you would actually need to write `\\%`, `\\_` and `\\\\`.
For patterns of the form `%needle%`, the function is as fast as the `position` function.
Other LIKE expressions are internally converted to a regular expression and executed with a performance similar to function `match`.
## notLike(haystack, pattern), haystack NOT LIKE pattern operator
The same thing as `like`, but negative.
## ilike
Case insensitive variant of [like](https://clickhouse.com/docs/en/sql-reference/functions/string-search-functions/#function-like) function. You can use `ILIKE` operator instead of the `ilike` function.
The function ignores the language, e.g. for Turkish (i/İ), the result might be incorrect.
For LIKE expressions of the form `%needle%`, the function is as fast as the `position` function.
All other LIKE expressions are internally converted to a regular expression and executed with a performance similar to function `match`.
**Syntax**
``` sql
ilike(haystack, pattern)
```sql
like(haystack, pattern)
```
**Arguments**
Alias: `haystack LIKE pattern` (operator)
- `haystack` — Input string. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `pattern` — If `pattern` does not contain percent signs or underscores, then the `pattern` only represents the string itself. An underscore (`_`) in `pattern` stands for (matches) any single character. A percent sign (`%`) matches any sequence of zero or more characters.
## notLike
Some `pattern` examples:
Like `like` but negates the result.
``` text
'abc' ILIKE 'abc' true
'abc' ILIKE 'a%' true
'abc' ILIKE '_b_' true
'abc' ILIKE 'c' false
Alias: `haystack NOT LIKE pattern` (operator)
## ilike
Like `like` but searches case-insensitively.
Alias: `haystack ILIKE pattern` (operator)
## notILike
Like `ilike` but negates the result.
Alias: `haystack NOT ILIKE pattern` (operator)
## ngramDistance
Calculates the 4-gram distance between a `haystack` string and a `needle` string. For that, it counts the symmetric difference between two multisets of 4-grams and normalizes it by the sum of their cardinalities. Returns a Float32 between 0 and 1. The smaller the result is, the more strings are similar to each other. Throws an exception if constant `needle` or `haystack` arguments are more than 32Kb in size. If any of non-constant `haystack` or `needle` arguments is more than 32Kb in size, the distance is always 1.
Functions `ngramDistanceCaseInsensitive, ngramDistanceUTF8, ngramDistanceCaseInsensitiveUTF8` provide case-insensitive and/or UTF-8 variants of this function.
**Syntax**
```sql
ngramDistance(haystack, needle)
```
**Returned values**
## ngramSearch
- True, if the string matches `pattern`.
- False, if the string does not match `pattern`.
Like `ngramDistance` but calculates the non-symmetric difference between a `needle` string and a `haystack` string, i.e. the number of n-grams from needle minus the common number of n-grams normalized by the number of `needle` n-grams. Returns a Float32 between 0 and 1. The bigger the result is, the more likely `needle` is in the `haystack`. This function is useful for fuzzy string search. Also see function `soundex`.
**Example**
Input table:
``` text
┌─id─┬─name─────┬─days─┐
│ 1 │ January │ 31 │
│ 2 │ February │ 29 │
│ 3 │ March │ 31 │
│ 4 │ April │ 30 │
└────┴──────────┴──────┘
```
Query:
``` sql
SELECT * FROM Months WHERE ilike(name, '%j%');
```
Result:
``` text
┌─id─┬─name────┬─days─┐
│ 1 │ January │ 31 │
└────┴─────────┴──────┘
```
## notILike(haystack, pattern), haystack NOT ILIKE pattern operator
The same thing as `ilike`, but negative.
## ngramDistance(haystack, needle)
Calculates the 4-gram distance between `haystack` and `needle`: counts the symmetric difference between two multisets of 4-grams and normalizes it by the sum of their cardinalities. Returns float number from 0 to 1 the closer to zero, the more strings are similar to each other. If the constant `needle` or `haystack` is more than 32Kb, throws an exception. If some of the non-constant `haystack` or `needle` strings are more than 32Kb, the distance is always one.
For case-insensitive search or/and in UTF-8 format use functions `ngramDistanceCaseInsensitive, ngramDistanceUTF8, ngramDistanceCaseInsensitiveUTF8`.
## ngramSearch(haystack, needle)
Same as `ngramDistance` but calculates the non-symmetric difference between `needle` and `haystack` the number of n-grams from needle minus the common number of n-grams normalized by the number of `needle` n-grams. The closer to one, the more likely `needle` is in the `haystack`. Can be useful for fuzzy string search.
For case-insensitive search or/and in UTF-8 format use functions `ngramSearchCaseInsensitive, ngramSearchUTF8, ngramSearchCaseInsensitiveUTF8`.
Functions `ngramSearchCaseInsensitive, ngramSearchUTF8, ngramSearchCaseInsensitiveUTF8` provide case-insensitive and/or UTF-8 variants of this function.
:::note
For UTF-8 case we use 3-gram distance. All these are not perfectly fair n-gram distances. We use 2-byte hashes to hash n-grams and then calculate the (non-)symmetric difference between these hash tables collisions may occur. With UTF-8 case-insensitive format we do not use fair `tolower` function we zero the 5-th bit (starting from zero) of each codepoint byte and first bit of zeroth byte if bytes more than one this works for Latin and mostly for all Cyrillic letters.
The UTF-8 variants use the 3-gram distance. These are not perfectly fair n-gram distances. We use 2-byte hashes to hash n-grams and then calculate the (non-)symmetric difference between these hash tables collisions may occur. With UTF-8 case-insensitive format we do not use fair `tolower` function we zero the 5-th bit (starting from zero) of each codepoint byte and first bit of zeroth byte if bytes more than one this works for Latin and mostly for all Cyrillic letters.
:::
**Syntax**
```sql
ngramSearch(haystack, needle)
```
## countSubstrings
Returns the number of substring occurrences.
Returns how often substring `needle` occurs in string `haystack`.
For a case-insensitive search, use [countSubstringsCaseInsensitive](../../sql-reference/functions/string-search-functions.md#countSubstringsCaseInsensitive) or [countSubstringsCaseInsensitiveUTF8](../../sql-reference/functions/string-search-functions.md#countSubstringsCaseInsensitiveUTF8) functions.
Functions `countSubstringsCaseInsensitive` and `countSubstringsCaseInsensitiveUTF8` provide a case-insensitive and case-insensitive + UTF-8 variants of this function.
**Syntax**
@ -645,34 +505,18 @@ countSubstrings(haystack, needle[, start_pos])
**Arguments**
- `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `start_pos` Position of the first character in the string to start search. Optional. [UInt](../../sql-reference/data-types/int-uint.md).
- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `start_pos` Position (1-based) in `haystack` at which the search starts. [UInt](../../sql-reference/data-types/int-uint.md). Optional.
**Returned values**
- Number of occurrences.
- The number of occurrences.
Type: [UInt64](../../sql-reference/data-types/int-uint.md).
**Examples**
Query:
``` sql
SELECT countSubstrings('foobar.com', '.');
```
Result:
``` text
┌─countSubstrings('foobar.com', '.')─┐
│ 1 │
└────────────────────────────────────┘
```
Query:
``` sql
SELECT countSubstrings('aaaa', 'aa');
```
@ -685,7 +529,7 @@ Result:
└───────────────────────────────┘
```
Query:
Example with `start_pos` argument:
```sql
SELECT countSubstrings('abc___abc', 'abc', 4);
@ -699,125 +543,7 @@ Result:
└────────────────────────────────────────┘
```
## countSubstringsCaseInsensitive
Returns the number of substring occurrences case-insensitive.
**Syntax**
``` sql
countSubstringsCaseInsensitive(haystack, needle[, start_pos])
```
**Arguments**
- `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `start_pos` — Position of the first character in the string to start search. Optional. [UInt](../../sql-reference/data-types/int-uint.md).
**Returned values**
- Number of occurrences.
Type: [UInt64](../../sql-reference/data-types/int-uint.md).
**Examples**
Query:
``` sql
SELECT countSubstringsCaseInsensitive('aba', 'B');
```
Result:
``` text
┌─countSubstringsCaseInsensitive('aba', 'B')─┐
│ 1 │
└────────────────────────────────────────────┘
```
Query:
``` sql
SELECT countSubstringsCaseInsensitive('foobar.com', 'CoM');
```
Result:
``` text
┌─countSubstringsCaseInsensitive('foobar.com', 'CoM')─┐
│ 1 │
└─────────────────────────────────────────────────────┘
```
Query:
``` sql
SELECT countSubstringsCaseInsensitive('abC___abC', 'aBc', 2);
```
Result:
``` text
┌─countSubstringsCaseInsensitive('abC___abC', 'aBc', 2)─┐
│ 1 │
└───────────────────────────────────────────────────────┘
```
## countSubstringsCaseInsensitiveUTF8
Returns the number of substring occurrences in `UTF-8` case-insensitive.
**Syntax**
``` sql
SELECT countSubstringsCaseInsensitiveUTF8(haystack, needle[, start_pos])
```
**Arguments**
- `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `start_pos` — Position of the first character in the string to start search. Optional. [UInt](../../sql-reference/data-types/int-uint.md).
**Returned values**
- Number of occurrences.
Type: [UInt64](../../sql-reference/data-types/int-uint.md).
**Examples**
Query:
``` sql
SELECT countSubstringsCaseInsensitiveUTF8('абв', 'A');
```
Result:
``` text
┌─countSubstringsCaseInsensitiveUTF8('абв', 'A')─┐
│ 1 │
└────────────────────────────────────────────────┘
```
Query:
```sql
SELECT countSubstringsCaseInsensitiveUTF8('аБв__АбВ__абв', 'Абв');
```
Result:
``` text
┌─countSubstringsCaseInsensitiveUTF8('аБв__АбВ__абв', 'Абв')─┐
│ 3 │
└────────────────────────────────────────────────────────────┘
```
## countMatches(haystack, pattern)
## countMatches
Returns the number of regular expression matches for a `pattern` in a `haystack`.
@ -840,8 +566,6 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md).
**Examples**
Query:
``` sql
SELECT countMatches('foobar.com', 'o+');
```
@ -854,8 +578,6 @@ Result:
└──────────────────────────────────┘
```
Query:
``` sql
SELECT countMatches('aaaa', 'aa');
```
@ -868,7 +590,7 @@ Result:
└───────────────────────────────┘
```
## regexpExtract(haystack, pattern[, index])
## regexpExtract
Extracts the first string in haystack that matches the regexp pattern and corresponds to the regex group index.
@ -899,8 +621,12 @@ SELECT
regexpExtract('100-200', '(\\d+)-(\\d+)', 1),
regexpExtract('100-200', '(\\d+)-(\\d+)', 2),
regexpExtract('100-200', '(\\d+)-(\\d+)', 0),
regexpExtract('100-200', '(\\d+)-(\\d+)')
regexpExtract('100-200', '(\\d+)-(\\d+)');
```
Result:
``` text
┌─regexpExtract('100-200', '(\\d+)-(\\d+)', 1)─┬─regexpExtract('100-200', '(\\d+)-(\\d+)', 2)─┬─regexpExtract('100-200', '(\\d+)-(\\d+)', 0)─┬─regexpExtract('100-200', '(\\d+)-(\\d+)')─┐
│ 100 │ 200 │ 100-200 │ 100 │
└──────────────────────────────────────────────┴──────────────────────────────────────────────┴──────────────────────────────────────────────┴───────────────────────────────────────────┘

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/time-window-functions
sidebar_position: 68
sidebar_position: 175
sidebar_label: Time Window
---

View File

@ -1,8 +1,7 @@
---
slug: /en/sql-reference/functions/tuple-functions
sidebar_position: 66
sidebar_position: 180
sidebar_label: Tuples
title: "Functions for Working with Tuples"
---
## tuple

View File

@ -1,8 +1,7 @@
---
slug: /en/sql-reference/functions/tuple-map-functions
sidebar_position: 46
sidebar_position: 120
sidebar_label: Maps
title: "Functions for Maps"
---
## map

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/type-conversion-functions
sidebar_position: 38
sidebar_position: 185
sidebar_label: Type Conversion
---
@ -8,12 +8,14 @@ sidebar_label: Type Conversion
## Common Issues with Data Conversion
Be aware of potential data loss if values of a datatype are converted to a smaller datatype (for example from `Int64` to `Int32`) or between
incompatible datatypes (for example from `String` to `Int`). Make sure to check carefully if the result is as expected.
ClickHouse generally uses the [same behavior as C++ programs](https://en.cppreference.com/w/cpp/language/implicit_conversion).
`to<type>` functions and [cast](#castx-t) have different behaviour in some cases, for example in case of [LowCardinality](../data-types/lowcardinality.md): [cast](#castx-t) removes [LowCardinality](../data-types/lowcardinality.md) trait `to<type>` functions don't. The same with [Nullable](../data-types/nullable.md), this behaviour is not compatible with SQL standard, and it can be changed using [cast_keep_nullable](../../operations/settings/settings.md/#cast_keep_nullable) setting.
`to<type>` functions and [cast](#castx-t) behave differently in some cases, for example in case of [LowCardinality](../data-types/lowcardinality.md): [cast](#castx-t) removes [LowCardinality](../data-types/lowcardinality.md) trait `to<type>` functions don't. The same with [Nullable](../data-types/nullable.md), this behaviour is not compatible with SQL standard, and it can be changed using [cast_keep_nullable](../../operations/settings/settings.md/#cast_keep_nullable) setting.
:::note
Be aware of potential data loss if values of a datatype are converted to a smaller datatype (for example from `Int64` to `Int32`) or between
incompatible datatypes (for example from `String` to `Int`). Make sure to check carefully if the result is as expected.
:::
Example:

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/ulid-functions
sidebar_position: 54
sidebar_position: 190
sidebar_label: ULID
---

View File

@ -1,5 +1,7 @@
---
slug: /en/sql-reference/functions/uniqtheta-functions
sidebar_position: 210
sidebar_label: uniqTheta
---
# uniqTheta Functions

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/url-functions
sidebar_position: 54
sidebar_position: 200
sidebar_label: URLs
---

View File

@ -1,12 +1,10 @@
---
slug: /en/sql-reference/functions/uuid-functions
sidebar_position: 53
sidebar_label: UUID
sidebar_position: 205
sidebar_label: UUIDs
---
# Functions for Working with UUID
The functions for working with UUID are listed below.
# Functions for Working with UUIDs
## generateUUIDv4

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/functions/ym-dict-functions
sidebar_position: 59
sidebar_position: 60
sidebar_label: Embedded Dictionaries
---

View File

@ -8,7 +8,7 @@ title: DELETE Statement
---
``` sql
DELETE FROM [db.]table [ON CLUSTER cluster] [WHERE expr]
DELETE FROM [db.]table [ON CLUSTER cluster] WHERE expr
```
`DELETE FROM` removes rows from the table `[db.]table` that match the expression `expr`. The deleted rows are marked as deleted immediately and will be automatically filtered out of all subsequent queries. Cleanup of data happens asynchronously in the background. This feature is only available for the MergeTree table engine family.

View File

@ -6,9 +6,9 @@ sidebar_label: file
# file
Creates a table from a file. This table function is similar to [url](/docs/en/sql-reference/table-functions/url.md) and [hdfs](/docs/en/sql-reference/table-functions/hdfs.md) ones.
Provides a table-like interface to SELECT from and INSERT to files. This table function is similar to the [s3](/docs/en/sql-reference/table-functions/url.md) table function. Use file() when working with local files, and s3() when working with buckets in S3, GCS, or MinIO.
`file` function can be used in `SELECT` and `INSERT` queries on data in [File](/docs/en/engines/table-engines/special/file.md) tables.
The `file` function can be used in `SELECT` and `INSERT` queries to read from or write to files.
**Syntax**
@ -27,7 +27,52 @@ file(path [,format] [,structure] [,compression])
A table with the specified structure for reading or writing data in the specified file.
**Examples**
## File Write Examples
### Write to a TSV file
```sql
INSERT INTO TABLE FUNCTION
file('test.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')
VALUES (1, 2, 3), (3, 2, 1), (1, 3, 2)
```
As a result, the data is written into the file `test.tsv`:
```bash
# cat /var/lib/clickhouse/user_files/test.tsv
1 2 3
3 2 1
1 3 2
```
### Partitioned Write to multiple TSV files
If you specify `PARTITION BY` expression when inserting data into a file() function, a separate file is created for each partition value. Splitting the data into separate files helps to improve reading operations efficiency.
```sql
INSERT INTO TABLE FUNCTION
file('test_{_partition_id}.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')
PARTITION BY column3
VALUES (1, 2, 3), (3, 2, 1), (1, 3, 2)
```
As a result, the data is written into three files: `test_1.tsv`, `test_2.tsv`, and `test_3.tsv`.
```bash
# cat /var/lib/clickhouse/user_files/test_1.tsv
3 2 1
# cat /var/lib/clickhouse/user_files/test_2.tsv
1 3 2
# cat /var/lib/clickhouse/user_files/test_3.tsv
1 2 3
```
## File Read Examples
### SELECT from a CSV file
Setting `user_files_path` and the contents of the file `test.csv`:
@ -44,7 +89,9 @@ $ cat /var/lib/clickhouse/user_files/test.csv
Getting data from a table in `test.csv` and selecting the first two rows from it:
``` sql
SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 2;
SELECT * FROM
file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32')
LIMIT 2;
```
``` text
@ -57,14 +104,21 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U
Getting the first 10 lines of a table that contains 3 columns of [UInt32](/docs/en/sql-reference/data-types/int-uint.md) type from a CSV file:
``` sql
SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 10;
SELECT * FROM
file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32')
LIMIT 10;
```
Inserting data from a file into a table:
### Inserting data from a file into a table:
``` sql
INSERT INTO FUNCTION file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') VALUES (1, 2, 3), (3, 2, 1);
SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32');
INSERT INTO FUNCTION
file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32')
VALUES (1, 2, 3), (3, 2, 1);
```
```sql
SELECT * FROM
file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32');
```
``` text

View File

@ -0,0 +1,260 @@
---
slug: /en/guides/developer/transactional
---
# Transactional (ACID) support
INSERT into one partition* in one table* of MergeTree* family up to max_insert_block_size rows* is transactional (ACID):
- Atomic: INSERT is succeeded or rejected as a whole: if confirmation is sent to the client, all rows INSERTed; if error is sent to the client, no rows INSERTed.
- Consistent: if there are no table constraints violated, then all rows in an INSERT are inserted and the INSERT succeeds; if constraints are violated, then no rows are inserted.
- Isolated: concurrent clients observe a consistent snapshot of the tablethe state of the table either as if before INSERT or after successful INSERT; no partial state is seen;
- Durable: successful INSERT is written to the filesystem before answering to the client, on single replica or multiple replicas (controlled by the `insert_quorum` setting), and ClickHouse can ask the OS to sync the filesystem data on the storage media (controlled by the `fsync_after_insert` setting).
* If table has many partitions and INSERT covers many partitionsthen insertion into every partition is transactional on its own;
* INSERT into multiple tables with one statement is possible if materialized views are involved;
* INSERT into Distributed table is not transactional as a whole, while insertion into every shard is transactional;
* another example: insert into Buffer tables is neither atomic nor isolated or consistent or durable;
* atomicity is ensured even if `async_insert` is enabled, but it can be turned off by the wait_for_async_insert setting;
* max_insert_block_size is 1 000 000 by default and can be adjusted as needed;
* if client did not receive the answer from the server, the client does not know if transaction succeeded, and it can repeat the transaction, using exactly-once insertion properties;
* ClickHouse is using MVCC with snapshot isolation internally;
* all ACID properties are valid even in case of server kill / crash;
* either insert_quorum into different AZ or fsync should be enabled to ensure durable inserts in typical setup;
* "consistency" in ACID terms does not cover the semantics of distributed systems, see https://jepsen.io/consistency which is controlled by different settings (select_sequential_consistency)
* this explanation does not cover a new transactions feature that allow to have full-featured transactions over multiple tables, materialized views, for multiple SELECTs, etc.
## Transactions, Commit, and Rollback
In addition to the functionality described at the top of this document, ClickHouse has experimental support for transactions, commits, and rollback functionality.
### Requirements
- Deploy ClickHouse Keeper or ZooKeeper to track transactions
- Atomic DB only (Default)
- Non-Replicated MergeTree table engine only
- Enable experimental transaction support by adding this setting in `config.d/transactions.xml`:
```xml
<clickhouse>
<allow_experimental_transactions>1</allow_experimental_transactions>
</clickhouse>
```
### Notes
- This is an experimental feature, and changes should be expected.
- If an exception occurs during a transaction, you cannot commit the transaction. This includes all exceptions, including `UNKNOWN_FUNCTION` exceptions caused by typos.
- Nested transactions are not supported; finish the current transaction and start a new one instead
### Configuration
These examples are with a single node ClickHouse server with ClickHouse Keeper enabled.
#### Enable experimental transaction support
```xml title=/etc/clickhouse-server/config.d/transactions.xml
<clickhouse>
<allow_experimental_transactions>1</allow_experimental_transactions>
</clickhouse>
```
#### Basic configuration for a single ClickHouse server node with ClickHouse Keeper enabled
:::note
See the [deployment](docs/en/deployment-guides/terminology.md) documentation for details on deploying ClickHouse server and a proper quorum of ClickHouse Keeper nodes. The configuration shown here is for experimental purposes.
:::
```xml title=/etc/clickhouse-server/config.d/config.xml
<clickhouse replace="true">
<logger>
<level>debug</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<size>1000M</size>
<count>3</count>
</logger>
<display_name>node 1</display_name>
<listen_host>0.0.0.0</listen_host>
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<zookeeper>
<node>
<host>clickhouse-01</host>
<port>9181</port>
</node>
</zookeeper>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms>
<session_timeout_ms>30000</session_timeout_ms>
<raft_logs_level>information</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>clickhouse-keeper-01</hostname>
<port>9234</port>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>
```
### Example
#### Verify that experimental transactions are enabled
Issue a `BEGIN TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions.
```sql
BEGIN TRANSACTION
```
```response
Ok.
```
:::tip
If you see the following error, then check your configuration file to make sure that `allow_experimental_transactions` is set to `1` (or any value other than `0` or `false`).
```
Code: 48. DB::Exception: Received from localhost:9000.
DB::Exception: Transactions are not supported.
(NOT_IMPLEMENTED)
```
You can also check ClickHouse Keeper by issuing
```
echo ruok | nc localhost 9181
```
ClickHouse Keeper should respond with `imok`.
:::
```sql
ROLLBACK
```
```response
Ok.
```
#### Create a table for testing
:::tip
Creation of tables is not transactional. Run this DDL query outside of a transaction.
:::
```sql
CREATE TABLE mergetree_table
(
`n` Int64
)
ENGINE = MergeTree
ORDER BY n
```
```response
Ok.
```
#### Begin a transaction and insert a row
```sql
BEGIN TRANSACTION
```
```response
Ok.
```
```sql
INSERT INTO mergetree_table FORMAT Values (10)
```
```response
Ok.
```
```sql
SELECT *
FROM mergetree_table
```
```response
┌──n─┐
│ 10 │
└────┘
```
:::note
You can query the table from within a transaction and see that the row was inserted even though it has not yet been committed.
:::
#### Rollback the transaction, and query the table again
Verify that the transaction is rolled back:
```sql
ROLLBACK
```
```response
Ok.
```
```sql
SELECT *
FROM mergetree_table
```
```response
Ok.
0 rows in set. Elapsed: 0.002 sec.
```
#### Complete a transaction and query the table again
```sql
BEGIN TRANSACTION
```
```response
Ok.
```
```sql
INSERT INTO mergetree_table FORMAT Values (42)
```
```response
Ok.
```
```sql
COMMIT
```
```response
Ok. Elapsed: 0.002 sec.
```
```sql
SELECT *
FROM mergetree_table
```
```response
┌──n─┐
│ 42 │
└────┘
```
### Transactions introspection
You can inspect transactions by querying the `system.transactions` table, but note that you cannot query that
table from a session that is in a transactionopen a second `clickhouse client` session to query that table.
```sql
SELECT *
FROM system.transactions
FORMAT Vertical
```
```response
Row 1:
──────
tid: (33,61,'51e60bce-6b82-4732-9e1d-b40705ae9ab8')
tid_hash: 11240433987908122467
elapsed: 210.017820947
is_readonly: 1
state: RUNNING
```
## More Details
See this [meta issue](https://github.com/ClickHouse/ClickHouse/issues/48794) to find much more extensive tests and to keep up to date with the progress.

View File

@ -41,6 +41,8 @@ contents:
dst: /usr/bin/clickhouse-library-bridge
- src: root/usr/bin/clickhouse-odbc-bridge
dst: /usr/bin/clickhouse-odbc-bridge
- src: root/usr/bin/clickhouse-static-files-disk-uploader
dst: /usr/bin/clickhouse-static-files-disk-uploader
- src: root/usr/share/bash-completion/completions
dst: /usr/share/bash-completion/completions
# docs

View File

@ -59,6 +59,8 @@ contents:
dst: /usr/bin/clickhouse-report
- src: root/usr/bin/clickhouse-server
dst: /usr/bin/clickhouse-server
- src: root/usr/bin/clickhouse-su
dst: /usr/bin/clickhouse-su
# docs
- src: ../AUTHORS
dst: /usr/share/doc/clickhouse-server/AUTHORS

View File

@ -638,6 +638,16 @@ public:
#pragma GCC optimize("-fno-var-tracking-assignments")
#endif
[[ maybe_unused ]] static std::string getHelpHeader()
{
return
"Usage: clickhouse benchmark [options] < queries.txt\n"
"Usage: clickhouse benchmark [options] --query \"query text\"\n"
"clickhouse-benchmark connects to ClickHouse server,"
" repeatedly sends specified queries and produces reports query statistics.\n";
}
int mainEntryClickHouseBenchmark(int argc, char ** argv)
{
using namespace DB;
@ -705,7 +715,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
if (options.count("help"))
{
std::cout << "Usage: " << argv[0] << " [options] < queries.txt\n";
std::cout << getHelpHeader();
std::cout << desc << "\n";
return 1;
}

View File

@ -959,12 +959,47 @@ bool Client::processWithFuzzing(const String & full_query)
}
[[ maybe_unused ]] static std::string getHelpHeader()
{
return
"Usage: clickhouse client [initial table definition] [--query <query>]\n"
"clickhouse-client is a client application that is used to connect to ClickHouse.\n"
"It can run queries as command line tool if you pass queries as an argument or as interactive client."
" Queries can run one at a time, or in in a multiquery mode with --multiquery option."
" To change settings you may use 'SET' statements and SETTINGS clause in queries or set is for a "
" session with corresponding clickhouse-client arguments.\n"
"'clickhouse client' command will try connect to clickhouse-server running on the same server."
" If you have credentials set up pass them with --user <username> --password <password>"
" or with --ask-password argument that will open command prompt.\n\n"
"This one will try connect to tcp native port(9000) without encryption:\n"
" clickhouse client --host clickhouse.example.com --password mysecretpassword\n"
"To connect to secure endpoint just set --secure argument. If you have "
" artered port set it with --port <your port>.\n"
" clickhouse client --secure --host clickhouse.example.com --password mysecretpassword\n";
}
[[ maybe_unused ]] static std::string getHelpFooter()
{
return
"Note: If you have clickhouse installed on your system you can use 'clickhouse-client'"
" invocation with a dash.\n\n"
"Example printing current longest running query on a server:\n"
" clickhouse client --query 'SELECT * FROM system.processes ORDER BY elapsed LIMIT 1 FORMAT Vertical'\n"
"Example creating table and inserting data:\n";
}
void Client::printHelpMessage(const OptionsDescription & options_description)
{
std::cout << getHelpHeader() << "\n";
std::cout << options_description.main_description.value() << "\n";
std::cout << options_description.external_description.value() << "\n";
std::cout << options_description.hosts_and_ports_description.value() << "\n";
std::cout << "In addition, --param_name=value can be specified for substitution of parameters for parametrized queries.\n";
std::cout << getHelpFooter() << "\n";
}

View File

@ -97,8 +97,9 @@ int mainEntryClickHouseCompressor(int argc, char ** argv)
if (options.count("help"))
{
std::cout << "Usage: " << argv[0] << " [options] < INPUT > OUTPUT" << std::endl;
std::cout << "Usage: " << argv[0] << " [options] INPUT OUTPUT" << std::endl;
std::cout << "Usage: clickhouse compressor [options] < INPUT > OUTPUT" << std::endl;
std::cout << "Alternative usage: clickhouse compressor [options] INPUT OUTPUT" << std::endl;
std::cout << desc << std::endl;
return 0;
}

View File

@ -72,10 +72,10 @@ void ClusterCopierApp::handleHelp(const std::string &, const std::string &)
Poco::Util::HelpFormatter help_formatter(options());
if (terminal_width)
help_formatter.setWidth(terminal_width);
help_formatter.setCommand(commandName());
help_formatter.setCommand(commandName() == "clickhouse-copier" ? "clickhouse-copier" : commandName() + " copier");
help_formatter.setHeader("Copies tables from one cluster to another");
help_formatter.setUsage("--config-file <config-file> --task-path <task-path>");
help_formatter.format(std::cerr);
help_formatter.format(std::cout);
stopOptionsProcessing();
}

View File

@ -31,8 +31,7 @@ void DisksApp::printHelpMessage(ProgramOptionsDescription & command_option_descr
help_description->add(command_option_description);
std::cout << "ClickHouse disk management tool\n";
std::cout << "Usage: ./clickhouse-disks [OPTION]\n";
std::cout << "clickhouse-disks\n\n";
std::cout << "usage clickhouse disks [OPTION]\n" << "clickhouse-disks\n\n";
for (const auto & current_command : supported_commands)
std::cout << command_descriptions[current_command]->command_name

View File

@ -146,11 +146,11 @@ int mainEntryClickHouseExtractFromConfig(int argc, char ** argv)
if (options.count("help"))
{
std::cerr << "Preprocess config file and extract value of the given key." << std::endl
std::cout << "Preprocess config file and extract value of the given key." << std::endl
<< std::endl;
std::cerr << "Usage: clickhouse extract-from-config [options]" << std::endl
std::cout << "Usage: clickhouse extract-from-config [options]" << std::endl
<< std::endl;
std::cerr << options_desc << std::endl;
std::cout << options_desc << std::endl;
return 0;
}

View File

@ -74,7 +74,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
if (options.count("help"))
{
std::cout << "Usage: " << argv[0] << " [options] < query" << std::endl;
std::cout << "Usage: clickhouse format [options] < query" << std::endl;
std::cout << desc << std::endl;
return 1;
}

View File

@ -1189,7 +1189,7 @@ try
{
using namespace DB;
po::options_description desc("Allowed options", getTerminalWidth());
po::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth());
desc.add_options()
("help,h", "produce help message")
("skip-commits-without-parents", po::value<bool>()->default_value(true),
@ -1218,7 +1218,7 @@ try
if (options.count("help"))
{
std::cout << documentation << '\n'
<< "Usage: " << argv[0] << '\n'
<< "Usage: clickhouse git-import\n"
<< desc << '\n'
<< "\nExample:\n"
<< "\nclickhouse git-import --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch '\n";

View File

@ -220,8 +220,8 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
("log-path", po::value<std::string>()->default_value("var/log/clickhouse-server"), "where to create log directory")
("data-path", po::value<std::string>()->default_value("var/lib/clickhouse"), "directory for data")
("pid-path", po::value<std::string>()->default_value("var/run/clickhouse-server"), "directory for pid file")
("user", po::value<std::string>()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user to create")
("group", po::value<std::string>()->default_value(DEFAULT_CLICKHOUSE_SERVER_GROUP), "clickhouse group to create")
("user", po::value<std::string>()->implicit_value("")->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user")
("group", po::value<std::string>()->implicit_value("")->default_value(DEFAULT_CLICKHOUSE_SERVER_GROUP), "clickhouse group")
("noninteractive,y", "run non-interactively")
("link", "create symlink to the binary instead of copying to binary-path")
;
@ -231,7 +231,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
if (options.count("help"))
{
std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " install [options]", getuid() != 0) << '\n';
std::cout << "Usage: " << formatWithSudo("clickhouse", getuid() != 0) << " install [options]\n";
std::cout << desc << '\n';
return 1;
}
@ -592,6 +592,11 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
fs::permissions(data_file, fs::perms::owner_read, fs::perm_options::replace);
fmt::print("Data path configuration override is saved to file {}.\n", data_file);
}
else
{
fmt::print("WARNING: Configuration of data paths already exists in {}."
" If you want to apply new paths, remove {} and run install again.\n", data_file, data_file);
}
/// Logger.
const std::string logger_file = config_d / "logger.xml";
@ -609,6 +614,11 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
fs::permissions(logger_file, fs::perms::owner_read, fs::perm_options::replace);
fmt::print("Log path configuration override is saved to file {}.\n", logger_file);
}
else
{
fmt::print("WARNING: Configuration of logger paths already exists in {}."
" If you want to apply new paths, remove {} and run install again.\n", logger_file, logger_file);
}
/// User directories.
const std::string user_directories_file = config_d / "user-directories.xml";
@ -627,6 +637,11 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
fs::permissions(user_directories_file, fs::perms::owner_read, fs::perm_options::replace);
fmt::print("User directory path configuration override is saved to file {}.\n", user_directories_file);
}
else
{
fmt::print("WARNING: Configuration of user directories paths already exists in {}."
" If you want to apply new paths, remove {} and run install again.\n", user_directories_file, user_directories_file);
}
/// OpenSSL.
const std::string openssl_file = config_d / "openssl.xml";
@ -647,6 +662,11 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
fs::permissions(openssl_file, fs::perms::owner_read, fs::perm_options::replace);
fmt::print("OpenSSL path configuration override is saved to file {}.\n", openssl_file);
}
else
{
fmt::print("WARNING: Configuration of OpenSSL paths already exists in {}."
" If you want to apply new paths, remove {} and run install again.\n", openssl_file, openssl_file);
}
}
}
else
@ -658,17 +678,46 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
if (configuration->has("path"))
{
data_path = configuration->getString("path");
fmt::print("{} has {} as data path.\n", main_config_file.string(), data_path.string());
std::string config_data_path = configuration->getString("path");
fmt::print("{} has {} as data path.\n", main_config_file.string(), config_data_path);
if (options.count("data-path"))
{
fmt::print("WARNING: Will use {} instead of {} as data path as it is set in {}.\n",
config_data_path, data_path.string(), main_config_file.string());
data_path = config_data_path;
}
}
if (configuration->has("logger.log"))
{
log_path = fs::path(configuration->getString("logger.log")).remove_filename();
fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path.string());
std::string config_log_path = fs::path(configuration->getString("logger.log")).remove_filename();
fmt::print("{} has {} as log path.\n", main_config_file.string(), config_log_path);
if (options.count("log-path"))
{
fmt::print("WARNING: Will use {} instead of {} as log path as it is set in {}.\n",
config_log_path, data_path.string(), main_config_file.string());
log_path = config_log_path;
}
}
if (configuration->has("user_directories.local_directory.path"))
{
std::string config_user_directory_path = fs::path(configuration->getString("user_directories.local_directory.path")).remove_filename();
fmt::print("{} has {} as log path.\n", main_config_file.string(), config_user_directory_path);
if (options.count("data-path"))
{
fmt::print("WARNING: Will use {} instead of {} as user directory path as it is set in {}.\n",
config_user_directory_path, (data_path / "access").string(), main_config_file.string());
}
}
/// OpenSSL.
const std::string openssl_file = config_d / "openssl.xml";
if (options.count("data-path") && !fs::exists(openssl_file))
{
fmt::print("WARNING: Paths for OpenSSL are not configured automatically.\n");
}
}
if (!fs::exists(users_config_file))
{
@ -905,6 +954,33 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
if (has_password_for_default_user)
maybe_password = " --password";
// If user specified --prefix, --pid-path, --config-path, --binary-path, --user, --group
// in install args we need to pass them to start command
std::string maybe_prefix;
if (options.count("prefix"))
maybe_prefix = " --prefix " + prefix.string();
std::string maybe_pid_path;
if (options.count("pid-path"))
maybe_pid_path = " --pid-path " + options["pid-path"].as<std::string>();
std::string maybe_config_path;
if (options.count("config-path"))
maybe_config_path = " --config-path " + options["config-path"].as<std::string>();
std::string maybe_binary_path;
if (options.count("binary-path"))
maybe_binary_path = " --binary-path " + options["binary-path"].as<std::string>();
std::string maybe_user;
if (options.count("user"))
maybe_user = " --user " + options["user"].as<std::string>();
std::string maybe_group;
if (options.count("group"))
maybe_group = " --group " + options["group"].as<std::string>();
fs::path pid_file = pid_path / "clickhouse-server.pid";
if (fs::exists(pid_file))
{
@ -913,9 +989,15 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
"\nRestart clickhouse-server with:\n"
" {}\n"
"\nStart clickhouse-client with:\n"
" clickhouse-client{}\n\n",
formatWithSudo("clickhouse restart"),
maybe_password);
" clickhouse-client{}{}{}{}{}{}{}\n\n",
formatWithSudo("clickhouse restart", getuid() != 0),
maybe_password,
maybe_prefix,
maybe_pid_path,
maybe_config_path,
maybe_binary_path,
maybe_user,
maybe_group);
}
else
{
@ -924,9 +1006,15 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
"\nStart clickhouse-server with:\n"
" {}\n"
"\nStart clickhouse-client with:\n"
" clickhouse-client{}\n\n",
formatWithSudo("clickhouse start"),
maybe_password);
" clickhouse-client{}{}{}{}{}{}{}\n\n",
formatWithSudo("clickhouse start", getuid() != 0),
maybe_password,
maybe_prefix,
maybe_pid_path,
maybe_config_path,
maybe_binary_path,
maybe_user,
maybe_group);
}
}
catch (const fs::filesystem_error &)
@ -947,10 +1035,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
return 0;
}
namespace
{
int start(const std::string & user, const fs::path & executable, const fs::path & config, const fs::path & pid_file, unsigned max_tries)
int start(const std::string & user, const std::string & group, const fs::path & binary, const fs::path & executable, const fs::path & config, const fs::path & pid_file, unsigned max_tries, bool no_sudo)
{
if (fs::exists(pid_file))
{
@ -989,11 +1076,20 @@ namespace
executable.string(), config.string(), pid_file.string());
if (!user.empty())
{
if (no_sudo)
{
/// Sometimes there is no sudo available like in some docker images
/// We will use clickhouse su instead
command = fmt::format("{} su {}:{} {}", binary.string(), user, group, command);
}
else
{
/// sudo respects limits in /etc/security/limits.conf e.g. open files,
/// that's why we are using it instead of the 'clickhouse su' tool.
command = fmt::format("sudo -u '{}' {}", user, command);
}
}
fmt::print("Will run {}\n", command);
executeScript(command, true);
@ -1194,28 +1290,35 @@ int mainEntryClickHouseStart(int argc, char ** argv)
("binary-path", po::value<std::string>()->default_value("usr/bin"), "directory with binary")
("config-path", po::value<std::string>()->default_value("etc/clickhouse-server"), "directory with configs")
("pid-path", po::value<std::string>()->default_value("var/run/clickhouse-server"), "directory for pid file")
("user", po::value<std::string>()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user")
("user", po::value<std::string>()->implicit_value("")->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user")
("group", po::value<std::string>()->implicit_value("")->default_value(DEFAULT_CLICKHOUSE_SERVER_GROUP), "clickhouse group")
("max-tries", po::value<unsigned>()->default_value(60), "Max number of tries for waiting the server (with 1 second delay)")
("no-sudo", po::bool_switch(), "Use clickhouse su if sudo is unavailable")
;
po::variables_map options;
po::store(po::parse_command_line(argc, argv, desc), options);
bool no_sudo = options["no-sudo"].as<bool>();
if (options.count("help"))
{
std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " start", getuid() != 0) << '\n';
std::cout << "Usage: " << formatWithSudo("clickhouse", !no_sudo && (getuid() != 0)) <<" start\n";
std::cout << desc << '\n';
return 1;
}
std::string user = options["user"].as<std::string>();
std::string group = options["group"].as<std::string>();
fs::path prefix = options["prefix"].as<std::string>();
fs::path binary = prefix / options["binary-path"].as<std::string>() / "clickhouse";
fs::path executable = prefix / options["binary-path"].as<std::string>() / "clickhouse-server";
fs::path config = prefix / options["config-path"].as<std::string>() / "config.xml";
fs::path pid_file = prefix / options["pid-path"].as<std::string>() / "clickhouse-server.pid";
unsigned max_tries = options["max-tries"].as<unsigned>();
return start(user, executable, config, pid_file, max_tries);
return start(user, group, binary, executable, config, pid_file, max_tries, no_sudo);
}
catch (...)
{
@ -1244,7 +1347,8 @@ int mainEntryClickHouseStop(int argc, char ** argv)
if (options.count("help"))
{
std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " stop", getuid() != 0) << '\n';
std::cout << "Usage: " << formatWithSudo("clickhouse", getuid() != 0) << " stop\n";
std::cout << desc << '\n';
return 1;
}
@ -1280,7 +1384,8 @@ int mainEntryClickHouseStatus(int argc, char ** argv)
if (options.count("help"))
{
std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " status", getuid() != 0) << '\n';
std::cout << "Usage: " << formatWithSudo("clickhouse", getuid() != 0) << " status\n";
std::cout << desc << '\n';
return 1;
}
@ -1310,24 +1415,31 @@ int mainEntryClickHouseRestart(int argc, char ** argv)
("binary-path", po::value<std::string>()->default_value("usr/bin"), "directory with binary")
("config-path", po::value<std::string>()->default_value("etc/clickhouse-server"), "directory with configs")
("pid-path", po::value<std::string>()->default_value("var/run/clickhouse-server"), "directory for pid file")
("user", po::value<std::string>()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user")
("user", po::value<std::string>()->implicit_value("")->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user")
("group", po::value<std::string>()->implicit_value("")->default_value(DEFAULT_CLICKHOUSE_SERVER_GROUP), "clickhouse group")
("force", po::value<bool>()->default_value(false), "Stop with KILL signal instead of TERM")
("do-not-kill", po::bool_switch(), "Do not send KILL even if TERM did not help")
("max-tries", po::value<unsigned>()->default_value(60), "Max number of tries for waiting the server (with 1 second delay)")
("no-sudo", po::bool_switch(), "Use clickhouse su if sudo is unavailable")
;
po::variables_map options;
po::store(po::parse_command_line(argc, argv, desc), options);
bool no_sudo = options["no-sudo"].as<bool>();
if (options.count("help"))
{
std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " restart", getuid() != 0) << '\n';
std::cout << "Usage: " << formatWithSudo("clickhouse", !no_sudo && getuid() != 0) << " restart\n";
std::cout << desc << '\n';
return 1;
}
std::string user = options["user"].as<std::string>();
std::string group = options["group"].as<std::string>();
fs::path prefix = options["prefix"].as<std::string>();
fs::path binary = prefix / options["binary-path"].as<std::string>() / "clickhouse";
fs::path executable = prefix / options["binary-path"].as<std::string>() / "clickhouse-server";
fs::path config = prefix / options["config-path"].as<std::string>() / "config.xml";
fs::path pid_file = prefix / options["pid-path"].as<std::string>() / "clickhouse-server.pid";
@ -1338,7 +1450,7 @@ int mainEntryClickHouseRestart(int argc, char ** argv)
if (int res = stop(pid_file, force, do_not_kill, max_tries))
return res;
return start(user, executable, config, pid_file, max_tries);
return start(user, group, binary, executable, config, pid_file, max_tries, no_sudo);
}
catch (...)
{

View File

@ -32,7 +32,7 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv)
if (options.count("help"))
{
std::cout << "Usage: " << argv[0] << " --zookeeper-logs-dir /var/lib/zookeeper/data/version-2 --zookeeper-snapshots-dir /var/lib/zookeeper/data/version-2 --output-dir /var/lib/clickhouse/coordination/snapshots" << std::endl;
std::cout << "Usage: clickhouse keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/data/version-2 --zookeeper-snapshots-dir /var/lib/zookeeper/data/version-2 --output-dir /var/lib/clickhouse/coordination/snapshots" << std::endl;
std::cout << desc << std::endl;
return 0;
}

View File

@ -213,7 +213,7 @@ int Keeper::run()
Poco::Util::HelpFormatter help_formatter(Keeper::options());
auto header_str = fmt::format("{} [OPTION] [-- [ARG]...]\n"
"positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010",
commandName());
commandName() == "clickhouse-keeper" ? "clickhouse-keeper" : commandName() + " keeper");
help_formatter.setHeader(header_str);
help_formatter.format(std::cout);
return 0;

View File

@ -699,27 +699,46 @@ void LocalServer::processConfig()
[[ maybe_unused ]] static std::string getHelpHeader()
{
return
"usage: clickhouse-local [initial table definition] [--query <query>]\n"
"Usage: clickhouse local [initial table definition] [--query <query>]\n"
"clickhouse-local allows to execute SQL queries on your data files via single command line call."
" To do so, initially you need to define your data source and its format."
" After you can execute your SQL queries in usual manner.\n"
"clickhouse-local allows to execute SQL queries on your data files without running clickhouse-server.\n\n"
"It can run as command line tool that does single action or as interactive client."
" For interactive experience you can just run 'clickhouse local' or add --interactive argument to your command."
" It will set up tables, run queries and pass control as if it is clickhouse-client."
" Then you can execute your SQL queries in usual manner."
" Non-interactive mode requires query as an argument and exits when queries finish."
" Multiple SQL queries can be passed as --query argument.\n\n"
"To configure initial environment two ways are supported: queries or command line parameters."
"There are two ways to define initial table keeping your data."
" Either just in first query like this:\n"
" CREATE TABLE <table> (<structure>) ENGINE = File(<input-format>, <file>);\n"
"Either through corresponding command line parameters --table --structure --input-format and --file.";
"Or through corresponding command line parameters --table --structure --input-format and --file.\n\n"
"clickhouse-local supports all features and engines of ClickHouse."
" You can query data from remote engines and store results locally or other way around."
" For table engines that actually store data on a disk like Log and MergeTree"
" clickhouse-local puts data to temporary directory that is not reused between runs.\n\n"
"clickhouse-local can be used to query data from stopped clickhouse-server installation with --path to"
" local directory with data.\n";
}
[[ maybe_unused ]] static std::string getHelpFooter()
{
return
"Note: If you have clickhouse installed on your system you can use 'clickhouse-local'"
" invocation with a dash.\n\n"
"Example printing memory used by each Unix user:\n"
"ps aux | tail -n +2 | awk '{ printf(\"%s\\t%s\\n\", $1, $4) }' | "
"clickhouse-local -S \"user String, mem Float64\" -q"
" \"SELECT user, round(sum(mem), 2) as mem_total FROM table GROUP BY user ORDER"
" BY mem_total DESC FORMAT PrettyCompact\"";
" BY mem_total DESC FORMAT PrettyCompact\"\n\n"
"Example reading file from S3, converting format and writing to a file:\n"
"clickhouse-local --query \"SELECT c1 as version, c2 as date "
"FROM url('https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/utils/list-versions/version_date.tsv')"
" INTO OUTFILE '/tmp/versions.json'\"";
}
@ -727,7 +746,7 @@ void LocalServer::printHelpMessage([[maybe_unused]] const OptionsDescription & o
{
#if defined(FUZZING_MODE)
std::cout <<
"usage: clickhouse <clickhouse-local arguments> -- <libfuzzer arguments>\n"
"Usage: clickhouse <clickhouse-local arguments> -- <libfuzzer arguments>\n"
"Note: It is important not to use only one letter keys with single dash for \n"
"for clickhouse-local arguments. It may work incorrectly.\n"
@ -895,4 +914,5 @@ catch (...)
{
return 1;
}
#endif

View File

@ -79,17 +79,43 @@ int mainEntryClickHouseRestart(int argc, char ** argv);
int mainEntryClickHouseDisks(int argc, char ** argv);
#endif
int mainEntryClickHouseHashBinary(int, char **)
bool hasHelpArg (char* arg)
{
/// Intentionally without newline. So you can run:
/// objcopy --add-section .clickhouse.hash=<(./clickhouse hash-binary) clickhouse
return (strcmp(arg, "--help") == 0 || (strcmp(arg, "-h") == 0) || (strcmp(arg, "help") == 0));
}
int mainEntryClickHouseHashBinary(int argc_, char ** argv_)
{
std::vector<char *> argv(argv_, argv_ + argc_);
auto it = std::find_if(argv.begin(), argv.end(), hasHelpArg);
if (it != argv.end())
{
std::cout << "Usage: clickhouse hash\nPrints hash of clickhouse binary.\n";
std::cout << " -h, --help Prints this message\n";
std::cout << "Result is intentionally without newline. So you can run:\n";
std::cout << "objcopy --add-section .clickhouse.hash=<(./clickhouse hash-binary) clickhouse.\n\n";
std::cout << "Current binary hash: ";
}
std::cout << getHashOfLoadedBinaryHex();
return 0;
}
namespace
{
void printHelp();
int mainEntryHelp(int, char **)
{
printHelp();
return 0;
}
int printHelpOnError(int, char **)
{
printHelp();
return -1;
}
using MainFunc = int (*)(int, char**);
#if !defined(FUZZING_MODE)
@ -150,14 +176,14 @@ std::pair<const char *, MainFunc> clickhouse_applications[] =
#if ENABLE_CLICKHOUSE_DISKS
{"disks", mainEntryClickHouseDisks},
#endif
{"help", mainEntryHelp},
};
int printHelp(int, char **)
void printHelp()
{
std::cerr << "Use one of the following commands:" << std::endl;
std::cout << "Use one of the following commands:" << std::endl;
for (auto & application : clickhouse_applications)
std::cerr << "clickhouse " << application.first << " [args] " << std::endl;
return -1;
std::cout << "clickhouse " << application.first << " [args] " << std::endl;
}
bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv)
@ -467,7 +493,7 @@ int main(int argc_, char ** argv_)
std::vector<char *> argv(argv_, argv_ + argc_);
/// Print a basic help if nothing was matched
MainFunc main_func = printHelp;
MainFunc main_func = printHelpOnError;
for (auto & application : clickhouse_applications)
{

View File

@ -1243,10 +1243,10 @@ try
|| !options.count("output-format"))
{
std::cout << documentation << "\n"
<< "\nUsage: " << argv[0] << " [options] < in > out\n"
<< "\nUsage: clickhouse obfuscator [options] < in > out\n"
<< "\nInput must be seekable file (it will be read twice).\n"
<< "\n" << description << "\n"
<< "\nExample:\n " << argv[0] << " --seed \"$(head -c16 /dev/urandom | base64)\" --input-format TSV --output-format TSV --structure 'CounterID UInt32, URLDomain String, URL String, SearchPhrase String, Title String' < stats.tsv\n";
<< "\nExample:\n clickhouse obfuscator --seed \"$(head -c16 /dev/urandom | base64)\" --input-format TSV --output-format TSV --structure 'CounterID UInt32, URLDomain String, URL String, SearchPhrase String, Title String' < stats.tsv\n";
return 0;
}

View File

@ -463,7 +463,7 @@ int Server::run()
Poco::Util::HelpFormatter help_formatter(Server::options());
auto header_str = fmt::format("{} [OPTION] [-- [ARG]...]\n"
"positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010",
commandName());
commandName() == "clickhouse-server" ? "clickhouse-server" : commandName() + " server");
help_formatter.setHeader(header_str);
help_formatter.format(std::cout);
return 0;

View File

@ -145,7 +145,7 @@ try
using namespace DB;
namespace po = boost::program_options;
po::options_description description("Allowed options", getTerminalWidth());
po::options_description description = createOptionsDescription("Allowed options", getTerminalWidth());
description.add_options()
("help,h", "produce help message")
("metadata-path", po::value<std::string>(), "Metadata path (SELECT data_paths FROM system.tables WHERE name = 'table_name' AND database = 'database_name')")

View File

@ -107,7 +107,7 @@ try
if (argc < 3)
{
std::cout << "Usage: ./clickhouse su user:group ..." << std::endl;
std::cout << "Usage: clickhouse su user:group ..." << std::endl;
exit(0); // NOLINT(concurrency-mt-unsafe)
}

View File

@ -130,6 +130,8 @@ static IAggregateFunction * createWithNumericBasedType(const IDataType & argumen
if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate<UInt16>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTemplate<UInt32>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::UUID) return new AggregateFunctionTemplate<UUID>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::IPv4) return new AggregateFunctionTemplate<IPv4>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::IPv6) return new AggregateFunctionTemplate<IPv6>(std::forward<TArgs>(args)...);
return nullptr;
}

View File

@ -5124,7 +5124,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
/// Create constant set column for constant folding
auto column_set = ColumnSet::create(1, std::move(set));
auto column_set = ColumnSet::create(1, FutureSet(std::move(set)));
argument_columns[1].column = ColumnConst::create(std::move(column_set), 1);
}

View File

@ -335,6 +335,7 @@ ReplxxLineReader::ReplxxLineReader(
rx.set_complete_on_empty(false);
rx.set_word_break_characters(word_break_characters);
rx.set_ignore_case(true);
rx.set_indent_multiline(false);
if (highlighter)
rx.set_highlighter_callback(highlighter);

View File

@ -1,5 +1,6 @@
#pragma once
#include <Interpreters/PreparedSets.h>
#include <Columns/IColumnDummy.h>
#include <Core/Field.h>
@ -20,7 +21,7 @@ class ColumnSet final : public COWHelper<IColumnDummy, ColumnSet>
private:
friend class COWHelper<IColumnDummy, ColumnSet>;
ColumnSet(size_t s_, const ConstSetPtr & data_) : data(data_) { s = s_; }
ColumnSet(size_t s_, FutureSet data_) : data(std::move(data_)) { s = s_; }
ColumnSet(const ColumnSet &) = default;
public:
@ -28,13 +29,13 @@ public:
TypeIndex getDataType() const override { return TypeIndex::Set; }
MutableColumnPtr cloneDummy(size_t s_) const override { return ColumnSet::create(s_, data); }
ConstSetPtr getData() const { return data; }
ConstSetPtr getData() const { if (!data.isReady()) return nullptr; return data.get(); }
// Used only for debugging, making it DUMPABLE
Field operator[](size_t) const override { return {}; }
private:
ConstSetPtr data;
FutureSet data;
};
}

View File

@ -140,6 +140,8 @@
M(RestartReplicaThreadsActive, "Number of threads in the RESTART REPLICA thread pool running a task.") \
M(QueryPipelineExecutorThreads, "Number of threads in the PipelineExecutor thread pool.") \
M(QueryPipelineExecutorThreadsActive, "Number of threads in the PipelineExecutor thread pool running a task.") \
M(ParquetDecoderThreads, "Number of threads in the ParquetBlockInputFormat thread pool running a task.") \
M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool.") \
M(DistributedFilesToInsert, "Number of pending files to process for asynchronous insertion into Distributed tables. Number of files for every shard is summed.") \
M(BrokenDistributedFilesToInsert, "Number of files for asynchronous insertion into Distributed tables that has been marked as broken. This metric will starts from 0 on start. Number of files for every shard is summed.") \
M(TablesToDropQueueSize, "Number of dropped tables, that are waiting for background data removal.") \

View File

@ -344,6 +344,8 @@ DEFINE_HASH(DB::Int256)
DEFINE_HASH(DB::Float32)
DEFINE_HASH(DB::Float64)
DEFINE_HASH(DB::UUID)
DEFINE_HASH(DB::IPv4)
DEFINE_HASH(DB::IPv6)
#undef DEFINE_HASH

View File

@ -75,7 +75,7 @@ template <typename T>
bool check(const T x) { return x == T{}; }
template <typename T>
void set(T & x) { x = {}; }
void set(T & x) { x = T{}; }
}

View File

@ -63,7 +63,7 @@
M(DiskReadElapsedMicroseconds, "Total time spent waiting for read syscall. This include reads from page cache.") \
M(DiskWriteElapsedMicroseconds, "Total time spent waiting for write syscall. This include writes to page cache.") \
M(NetworkReceiveElapsedMicroseconds, "Total time spent waiting for data to receive or receiving data from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
M(NetworkSendElapsedMicroseconds, "Total time spent waiting for data to send to network or sending data to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries..") \
M(NetworkSendElapsedMicroseconds, "Total time spent waiting for data to send to network or sending data to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
M(NetworkReceiveBytes, "Total number of bytes received from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
M(NetworkSendBytes, "Total number of bytes send to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
\
@ -249,7 +249,7 @@ The server successfully detected this situation and will download merged part fr
M(RWLockWritersWaitMilliseconds, "Total time spent waiting for a write lock to be acquired (in a heavy RWLock).") \
M(DNSError, "Total count of errors in DNS resolution") \
\
M(RealTimeMicroseconds, "Total (wall clock) time spent in processing (queries and other tasks) threads (not that this is a sum).") \
M(RealTimeMicroseconds, "Total (wall clock) time spent in processing (queries and other tasks) threads (note that this is a sum).") \
M(UserTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in user space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \
M(SystemTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in OS kernel space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \
M(MemoryOvercommitWaitTimeMicroseconds, "Total time spent in waiting for memory to be freed in OvercommitTracker.") \
@ -419,7 +419,7 @@ The server successfully detected this situation and will download merged part fr
M(MainConfigLoads, "Number of times the main configuration was reloaded.") \
\
M(AggregationPreallocatedElementsInHashTables, "How many elements were preallocated in hash tables for aggregation.") \
M(AggregationHashTablesInitializedAsTwoLevel, "How many hash tables were inited as two-level for aggregation.") \
M(AggregationHashTablesInitializedAsTwoLevel, "How many hash tables were initialiseded as two-level for aggregation.") \
\
M(MergeTreeMetadataCacheGet, "Number of rocksdb reads(used for merge tree metadata cache)") \
M(MergeTreeMetadataCachePut, "Number of rocksdb puts(used for merge tree metadata cache)") \

View File

@ -1,46 +0,0 @@
#pragma once
#include <optional>
#include <cmath>
namespace DB
{
class RangeGenerator
{
public:
explicit RangeGenerator(size_t total_size_, size_t range_step_, size_t range_start = 0)
: from(range_start), range_step(range_step_), total_size(total_size_)
{
}
size_t totalRanges() const { return static_cast<size_t>(ceil(static_cast<float>(total_size - from) / range_step)); }
using Range = std::pair<size_t, size_t>;
// return upper exclusive range of values, i.e. [from_range, to_range>
std::optional<Range> nextRange()
{
if (from >= total_size)
{
return std::nullopt;
}
auto to = from + range_step;
if (to >= total_size)
{
to = total_size;
}
Range range{from, to};
from = to;
return range;
}
private:
size_t from;
size_t range_step;
size_t total_size;
};
}

View File

@ -323,7 +323,7 @@ using ThreadFromGlobalPool = ThreadFromGlobalPoolImpl<true>;
/// one is at GlobalThreadPool level, the other is at ThreadPool level, so tracing context will be initialized on the same thread twice.
///
/// Once the worker on ThreadPool gains the control of execution, it won't return until it's shutdown,
/// which means the tracing context initialized at underlying worker level won't be delete for a very long time.
/// which means the tracing context initialized at underlying worker level won't be deleted for a very long time.
/// This would cause wrong context for further jobs scheduled in ThreadPool.
///
/// To make sure the tracing context is correctly propagated, we explicitly disable context propagation(including initialization and de-initialization) at underlying worker level.

View File

@ -299,10 +299,7 @@ ZooKeeper::~ZooKeeper()
{
finalize(false, false, "Destructor called");
if (send_thread.joinable())
send_thread.join();
if (receive_thread.joinable())
receive_thread.join();
}
catch (...)
@ -365,10 +362,7 @@ ZooKeeper::ZooKeeper(
{
tryLogCurrentException(log, "Failed to connect to ZooKeeper");
if (send_thread.joinable())
send_thread.join();
if (receive_thread.joinable())
receive_thread.join();
throw;
@ -914,7 +908,6 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea
}
/// Send thread will exit after sending close request or on expired flag
if (send_thread.joinable())
send_thread.join();
}
@ -932,7 +925,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea
tryLogCurrentException(log);
}
if (!error_receive && receive_thread.joinable())
if (!error_receive)
receive_thread.join();
{

View File

@ -255,8 +255,30 @@ private:
Watches watches TSA_GUARDED_BY(watches_mutex);
std::mutex watches_mutex;
ThreadFromGlobalPool send_thread;
ThreadFromGlobalPool receive_thread;
/// A wrapper around ThreadFromGlobalPool that allows to call join() on it from multiple threads.
class ThreadReference
{
public:
const ThreadReference & operator = (ThreadFromGlobalPool && thread_)
{
std::lock_guard<std::mutex> l(lock);
thread = std::move(thread_);
return *this;
}
void join()
{
std::lock_guard<std::mutex> l(lock);
if (thread.joinable())
thread.join();
}
private:
std::mutex lock;
ThreadFromGlobalPool thread;
};
ThreadReference send_thread;
ThreadReference receive_thread;
Poco::Logger * log;

View File

@ -1,22 +0,0 @@
#include <Common/RangeGenerator.h>
#include <gtest/gtest.h>
using namespace DB;
TEST(RangeGenerator, Common)
{
RangeGenerator g{25, 10};
EXPECT_EQ(g.totalRanges(), 3);
std::vector<RangeGenerator::Range> ranges{{0, 10}, {10, 20}, {20, 25}};
for (size_t i = 0; i < 3; ++i)
{
auto r = g.nextRange();
EXPECT_TRUE(r);
EXPECT_EQ(r, ranges[i]);
}
auto r = g.nextRange();
EXPECT_TRUE(!r);
}

View File

@ -11,6 +11,9 @@
namespace DB
{
namespace ErrorCodes { extern const int CANNOT_SCHEDULE_TASK; }
BackgroundSchedulePoolTaskInfo::BackgroundSchedulePoolTaskInfo(
BackgroundSchedulePool & pool_, const std::string & log_name_, const BackgroundSchedulePool::TaskFunc & function_)
: pool(pool_), log_name(log_name_), function(function_)
@ -158,10 +161,26 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size_, CurrentMetrics::Met
LOG_INFO(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with {} threads", size_);
threads.resize(size_);
try
{
for (auto & thread : threads)
thread = ThreadFromGlobalPoolNoTracingContextPropagation([this] { threadFunction(); });
delayed_thread = std::make_unique<ThreadFromGlobalPoolNoTracingContextPropagation>([this] { delayExecutionThreadFunction(); });
}
catch (...)
{
LOG_FATAL(
&Poco::Logger::get("BackgroundSchedulePool/" + thread_name),
"Couldn't get {} threads from global thread pool: {}",
size_,
getCurrentExceptionCode() == DB::ErrorCodes::CANNOT_SCHEDULE_TASK
? "Not enough threads. Please make sure max_thread_pool_size is considerably "
"bigger than background_schedule_pool_size."
: getCurrentExceptionMessage(/* with_stacktrace */ true));
abort();
}
}

View File

@ -302,6 +302,7 @@ class IColumn;
M(Bool, http_skip_not_found_url_for_globs, true, "Skip url's for globs with HTTP_NOT_FOUND error", 0) \
M(Bool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \
M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \
M(UInt64, use_index_for_in_with_subqueries_max_values, 0, "The maximum size of set in the right hand side of the IN operator to use table index for filtering. It allows to avoid performance degradation and higher memory usage due to preparation of additional data structures for large queries. Zero means no limit.", 0) \
M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \
M(Bool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \
M(Bool, empty_result_for_aggregation_by_constant_keys_on_empty_set, true, "Return empty result when aggregating by constant keys on empty set.", 0) \
@ -569,6 +570,7 @@ class IColumn;
M(Bool, query_cache_squash_partial_results, true, "Squash partial result blocks to blocks of size 'max_block_size'. Reduces performance of inserts into the query cache but improves the compressability of cache entries.", 0) \
M(Seconds, query_cache_ttl, 60, "After this time in seconds entries in the query cache become stale", 0) \
M(Bool, query_cache_share_between_users, false, "Allow other users to read entry in the query cache", 0) \
M(Bool, enable_sharing_sets_for_mutations, true, "Allow sharing set objects build for IN subqueries between different tasks of the same mutation. This reduces memory usage and CPU consumption", 0) \
\
M(Bool, optimize_rewrite_sum_if_to_count_if, false, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \
M(Bool, optimize_rewrite_aggregate_function_with_if, true, "Rewrite aggregate functions with if expression as argument when logically equivalent. For example, avg(if(cond, col, null)) can be rewritten to avgIf(cond, col)", 0) \
@ -635,10 +637,10 @@ class IColumn;
M(String, remote_filesystem_read_method, "threadpool", "Method of reading data from remote filesystem, one of: read, threadpool.", 0) \
M(Bool, local_filesystem_read_prefetch, false, "Should use prefetching when reading data from local filesystem.", 0) \
M(Bool, remote_filesystem_read_prefetch, true, "Should use prefetching when reading data from remote filesystem.", 0) \
M(Int64, read_priority, 0, "Priority to read data from local filesystem. Only supported for 'pread_threadpool' method.", 0) \
M(Int64, read_priority, 0, "Priority to read data from local filesystem or remote filesystem. Only supported for 'pread_threadpool' method for local filesystem and for `threadpool` method for remote filesystem.", 0) \
M(UInt64, merge_tree_min_rows_for_concurrent_read_for_remote_filesystem, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \
M(UInt64, merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \
M(UInt64, remote_read_min_bytes_for_seek, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes required for remote read (url, s3) to do seek, instead for read with ignore.", 0) \
M(UInt64, remote_read_min_bytes_for_seek, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes required for remote read (url, s3) to do seek, instead of read with ignore.", 0) \
\
M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \
M(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background. Makes sense only for inserts via HTTP protocol. If wait_for_async_insert is false, INSERT query is processed almost instantly, otherwise client will wait until data will be flushed to table", 0) \
@ -813,6 +815,8 @@ class IColumn;
M(Bool, input_format_orc_case_insensitive_column_matching, false, "Ignore case when matching ORC columns with CH columns.", 0) \
M(Bool, input_format_parquet_import_nested, false, "Allow to insert array of structs into Nested table in Parquet input format.", 0) \
M(Bool, input_format_parquet_case_insensitive_column_matching, false, "Ignore case when matching Parquet columns with CH columns.", 0) \
/* TODO: Consider unifying this with https://github.com/ClickHouse/ClickHouse/issues/38755 */ \
M(Bool, input_format_parquet_preserve_order, true, "Avoid reordering rows when reading from Parquet files. Usually makes it much slower.", 0) \
M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \
M(Bool, input_format_orc_allow_missing_columns, false, "Allow missing columns while reading ORC input formats", 0) \
M(Bool, input_format_parquet_allow_missing_columns, false, "Allow missing columns while reading Parquet input formats", 0) \

View File

@ -37,6 +37,7 @@ public:
bool textCanContainOnlyValidUTF8() const override { return true; }
bool isComparable() const override { return true; }
bool isValueRepresentedByNumber() const override { return true; }
bool isValueRepresentedByInteger() const override { return true; }
bool isValueRepresentedByUnsignedInteger() const override { return true; }
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const override { return true; }

View File

@ -20,7 +20,7 @@ public:
bool isParametric() const override { return true; }
// Used for expressions analysis.
MutableColumnPtr createColumn() const override { return ColumnSet::create(0, nullptr); }
MutableColumnPtr createColumn() const override { return ColumnSet::create(0, FutureSet{}); }
// Used only for debugging, making it DUMPABLE
Field getDefault() const override { return Tuple(); }

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