Merge remote-tracking branch 'upstream/master' into async-insert

This commit is contained in:
Ivan Lezhankin 2021-02-17 21:48:48 +03:00
commit dee051dc8e
405 changed files with 9232 additions and 1978 deletions

6
.gitignore vendored
View File

@ -137,3 +137,9 @@ website/package-lock.json
/prof
*.iml
# data store
/programs/server/data
/programs/server/metadata
/programs/server/store

2
.gitmodules vendored
View File

@ -220,4 +220,4 @@
url = https://github.com/ClickHouse-Extras/boringssl.git
[submodule "contrib/NuRaft"]
path = contrib/NuRaft
url = https://github.com/eBay/NuRaft.git
url = https://github.com/ClickHouse-Extras/NuRaft.git

View File

@ -13,6 +13,3 @@ ClickHouse® is an open-source column-oriented database management system that a
* [Code Browser](https://clickhouse.tech/codebrowser/html_report/ClickHouse/index.html) with syntax highlight and navigation.
* [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any.
* You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person.
## Upcoming Events
* [Chinese ClickHouse Meetup (online)](http://hdxu.cn/8KxZE) on 6 February 2021.

View File

@ -1,6 +1,8 @@
#pragma once
#include <common/extended_types.h>
#include <common/defines.h>
namespace common
{
@ -156,4 +158,11 @@ namespace common
return false;
return (x * y) / y != x;
}
/// Multiply and ignore overflow.
template <typename T1, typename T2>
inline auto NO_SANITIZE_UNDEFINED mulIgnoreOverflow(T1 x, T2 y)
{
return x * y;
}
}

View File

@ -152,7 +152,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context)
if (sig != SIGTSTP) /// This signal is used for debugging.
{
/// The time that is usually enough for separate thread to print info into log.
sleepForSeconds(10);
sleepForSeconds(20); /// FIXME: use some feedback from threads that process stacktrace
call_default_signal_handler(sig);
}
@ -311,7 +311,8 @@ private:
if (stack_trace.getSize())
{
/// Write bare stack trace (addresses) just in case if we will fail to print symbolized stack trace.
/// NOTE This still require memory allocations and mutex lock inside logger. BTW we can also print it to stderr using write syscalls.
/// NOTE: This still require memory allocations and mutex lock inside logger.
/// BTW we can also print it to stderr using write syscalls.
std::stringstream bare_stacktrace;
bare_stacktrace << "Stack trace:";
@ -324,7 +325,7 @@ private:
/// Write symbolized stack trace line by line for better grep-ability.
stack_trace.toStringEveryLine([&](const std::string & s) { LOG_FATAL(log, s); });
#if defined(__linux__)
#if defined(OS_LINUX)
/// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace.
String calculated_binary_hash = getHashOfLoadedBinaryHex();
if (daemon.stored_binary_hash.empty())
@ -561,6 +562,7 @@ void debugIncreaseOOMScore()
{
DB::WriteBufferFromFile buf("/proc/self/oom_score_adj");
buf.write(new_score.c_str(), new_score.size());
buf.close();
}
catch (const Poco::Exception & e)
{
@ -783,7 +785,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing()
/// Setup signal handlers.
/// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime.
addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP}, signalHandler, &handled_signals);
addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP, SIGTRAP}, signalHandler, &handled_signals);
addSignalHandler({SIGHUP, SIGUSR1}, closeLogsSignalHandler, &handled_signals);
addSignalHandler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler, &handled_signals);

View File

@ -11,7 +11,7 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/NuRaft/CMakeLists.txt")
return()
endif ()
if (NOT OS_FREEBSD)
if (NOT OS_FREEBSD AND NOT OS_DARWIN)
set (USE_NURAFT 1)
set (NURAFT_LIBRARY nuraft)
@ -20,5 +20,5 @@ if (NOT OS_FREEBSD)
message (STATUS "Using NuRaft=${USE_NURAFT}: ${NURAFT_INCLUDE_DIR} : ${NURAFT_LIBRARY}")
else()
set (USE_NURAFT 0)
message (STATUS "Using internal NuRaft library on FreeBSD is not supported")
message (STATUS "Using internal NuRaft library on FreeBSD and Darwin is not supported")
endif()

2
contrib/NuRaft vendored

@ -1 +1 @@
Subproject commit 410bd149da84cdde60b4436b02b738749f4e87e1
Subproject commit 7adf7ae33e7d5c307342431b577c8ab1025ee793

2
contrib/boost vendored

@ -1 +1 @@
Subproject commit 8e259cd2a6b60d75dd17e73432f11bb7b9351bb1
Subproject commit 48f40ebb539220d328958f8823b094c0b07a4e79

View File

@ -30,7 +30,12 @@ set(SRCS
add_library(nuraft ${SRCS})
target_compile_definitions(nuraft PRIVATE USE_BOOST_ASIO=1 BOOST_ASIO_STANDALONE=1)
if (NOT OPENSSL_SSL_LIBRARY OR NOT OPENSSL_CRYPTO_LIBRARY)
target_compile_definitions(nuraft PRIVATE USE_BOOST_ASIO=1 BOOST_ASIO_STANDALONE=1 SSL_LIBRARY_NOT_FOUND=1)
else()
target_compile_definitions(nuraft PRIVATE USE_BOOST_ASIO=1 BOOST_ASIO_STANDALONE=1)
endif()
target_include_directories (nuraft SYSTEM PRIVATE ${LIBRARY_DIR}/include/libnuraft)
# for some reason include "asio.h" directly without "boost/" prefix.

View File

@ -113,10 +113,10 @@ $ docker run --rm -e CLICKHOUSE_UID=0 -e CLICKHOUSE_GID=0 --name clickhouse-serv
### How to create default database and user on starting
Sometimes you may want to create user (user named `default` is used by default) and database on image starting. You can do it using environment variables `CLICKHOUSE_DB`, `CLICKHOUSE_USER` and `CLICKHOUSE_PASSWORD`:
Sometimes you may want to create user (user named `default` is used by default) and database on image starting. You can do it using environment variables `CLICKHOUSE_DB`, `CLICKHOUSE_USER`, `CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT` and `CLICKHOUSE_PASSWORD`:
```
$ docker run --rm -e CLICKHOUSE_DB=my_database -e CLICKHOUSE_USER=username -e CLICKHOUSE_PASSWORD=password -p 9000:9000/tcp yandex/clickhouse-server
$ docker run --rm -e CLICKHOUSE_DB=my_database -e CLICKHOUSE_USER=username -e CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT=1 -e CLICKHOUSE_PASSWORD=password -p 9000:9000/tcp yandex/clickhouse-server
```
## How to extend this image

View File

@ -54,8 +54,10 @@ docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/libm.so.6 "${CONTAIN
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/libpthread.so.0 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/librt.so.1 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/libnss_dns.so.2 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/libnss_files.so.2 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib/x86_64-linux-gnu/libresolv.so.2 "${CONTAINER_ROOT_FOLDER}/lib"
docker cp -L "${ubuntu20image}":/lib64/ld-linux-x86-64.so.2 "${CONTAINER_ROOT_FOLDER}/lib64"
docker cp -L "${ubuntu20image}":/etc/nsswitch.conf "${CONTAINER_ROOT_FOLDER}/etc"
docker build "$DOCKER_BUILD_FOLDER" -f Dockerfile.alpine -t "${DOCKER_IMAGE}:${VERSION}-alpine" --pull
rm -rf "$CONTAINER_ROOT_FOLDER"

View File

@ -54,6 +54,7 @@ FORMAT_SCHEMA_PATH="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_
CLICKHOUSE_USER="${CLICKHOUSE_USER:-default}"
CLICKHOUSE_PASSWORD="${CLICKHOUSE_PASSWORD:-}"
CLICKHOUSE_DB="${CLICKHOUSE_DB:-}"
CLICKHOUSE_ACCESS_MANAGEMENT="${CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT:-0}"
for dir in "$DATA_DIR" \
"$ERROR_LOG_DIR" \
@ -97,6 +98,7 @@ if [ -n "$CLICKHOUSE_USER" ] && [ "$CLICKHOUSE_USER" != "default" ] || [ -n "$CL
</networks>
<password>${CLICKHOUSE_PASSWORD}</password>
<quota>default</quota>
<access_management>${CLICKHOUSE_ACCESS_MANAGEMENT}</access_management>
</${CLICKHOUSE_USER}>
</users>
</yandex>

View File

@ -163,6 +163,7 @@ function clone_submodules
contrib/xz
contrib/dragonbox
contrib/fast_float
contrib/NuRaft
)
git submodule sync
@ -182,6 +183,7 @@ function run_cmake
"-DENABLE_EMBEDDED_COMPILER=0"
"-DENABLE_THINLTO=0"
"-DUSE_UNWIND=1"
"-DENABLE_NURAFT=1"
)
# TODO remove this? we don't use ccache anyway. An option would be to download it

View File

@ -64,7 +64,7 @@ clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits"
clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits"
clickhouse-client --query "SHOW TABLES FROM test"
./stress --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION"
./stress --hung-check --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" && echo "OK" > /test_output/script_exit_code.txt || echo "FAIL" > /test_output/script_exit_code.txt
stop
start

View File

@ -1,8 +1,9 @@
#!/usr/bin/env python3
# -*- coding: utf-8 -*-
from multiprocessing import cpu_count
from subprocess import Popen, check_call
from subprocess import Popen, call, STDOUT
import os
import sys
import shutil
import argparse
import logging
@ -64,7 +65,8 @@ if __name__ == "__main__":
parser.add_argument("--server-log-folder", default='/var/log/clickhouse-server')
parser.add_argument("--output-folder")
parser.add_argument("--global-time-limit", type=int, default=3600)
parser.add_argument("--num-parallel", default=cpu_count());
parser.add_argument("--num-parallel", default=cpu_count())
parser.add_argument('--hung-check', action='store_true', default=False)
args = parser.parse_args()
func_pipes = []
@ -81,4 +83,13 @@ if __name__ == "__main__":
logging.info("Finished %s from %s processes", len(retcodes), len(func_pipes))
time.sleep(5)
logging.info("All processes finished")
if args.hung_check:
logging.info("Checking if some queries hung")
cmd = "{} {} {}".format(args.test_cmd, "--hung-check", "00001_select_1")
res = call(cmd, shell=True, stderr=STDOUT)
if res != 0:
logging.info("Hung check failed with exit code {}".format(res))
sys.exit(1)
logging.info("Stress test finished")

View File

@ -0,0 +1,29 @@
---
toc_priority:
toc_title:
---
# data_type_name {#data_type-name}
Description.
**Parameters** (Optional)
- `x` — Description. [Type name](relative/path/to/type/dscr.md#type).
- `y` — Description. [Type name](relative/path/to/type/dscr.md#type).
**Examples**
```sql
```
## Additional Info {#additional-info} (Optional)
The name of an additional section can be any, for example, **Usage**.
**See Also** (Optional)
- [link](#)
[Original article](https://clickhouse.tech/docs/en/data_types/<data-type-name>/) <!--hide-->

View File

@ -12,16 +12,20 @@ Alias: `<alias name>`. (Optional)
More text (Optional).
**Parameters** (Optional)
**Arguments** (Optional)
- `x` — Description. [Type name](relative/path/to/type/dscr.md#type).
- `y` — Description. [Type name](relative/path/to/type/dscr.md#type).
**Parameters** (Optional, only for parametric aggregate functions)
- `z` — Description. [Type name](relative/path/to/type/dscr.md#type).
**Returned value(s)**
- Returned values list.
Type: [Type](relative/path/to/type/dscr.md#type).
Type: [Type name](relative/path/to/type/dscr.md#type).
**Example**

View File

@ -8,10 +8,14 @@ Columns:
**Example**
Query:
``` sql
SELECT * FROM system.table_name
```
Result:
``` text
Some output. It shouldn't be too long.
```

View File

@ -59,10 +59,11 @@ Optional parameters:
- `rabbitmq_max_block_size`
- `rabbitmq_flush_interval_ms`
Required configuration:
The RabbitMQ server configuration should be added using the ClickHouse config file.
Required configuration:
``` xml
<rabbitmq>
<username>root</username>
@ -70,6 +71,14 @@ The RabbitMQ server configuration should be added using the ClickHouse config fi
</rabbitmq>
```
Additional configuration:
``` xml
<rabbitmq>
<vhost>clickhouse</vhost>
</rabbitmq>
```
Example:
``` sql

View File

@ -136,8 +136,7 @@ The following settings can be specified in configuration file for given endpoint
- `access_key_id` and `secret_access_key` — Optional. Specifies credentials to use with given endpoint.
- `use_environment_credentials` — Optional, default value is `false`. If set to `true`, S3 client will try to obtain credentials from environment variables and Amazon EC2 metadata for given endpoint.
- `header` — Optional, can be speficied multiple times. Adds specified HTTP header to a request to given endpoint.
This configuration also applies to S3 disks in `MergeTree` table engine family.
- `server_side_encryption_customer_key_base64` — Optional. If specified, required headers for accessing S3 objects with SSE-C encryption will be set.
Example:
@ -149,6 +148,7 @@ Example:
<!-- <secret_access_key>SECRET_ACCESS_KEY</secret_access_key> -->
<!-- <use_environment_credentials>false</use_environment_credentials> -->
<!-- <header>Authorization: Bearer SOME-TOKEN</header> -->
<!-- <server_side_encryption_customer_key_base64>BASE64-ENCODED-KEY</server_side_encryption_customer_key_base64> -->
</endpoint-name>
</s3>
```

View File

@ -715,6 +715,7 @@ Configuration markup:
<endpoint>https://storage.yandexcloud.net/my-bucket/root-path/</endpoint>
<access_key_id>your_access_key_id</access_key_id>
<secret_access_key>your_secret_access_key</secret_access_key>
<server_side_encryption_customer_key_base64>your_base64_encoded_customer_key</server_side_encryption_customer_key_base64>
<proxy>
<uri>http://proxy1</uri>
<uri>http://proxy2</uri>
@ -750,7 +751,8 @@ Optional parameters:
- `metadata_path` — Path on local FS to store metadata files for S3. Default value is `/var/lib/clickhouse/disks/<disk_name>/`.
- `cache_enabled` — Allows to cache mark and index files on local FS. Default value is `true`.
- `cache_path` — Path on local FS where to store cached mark and index files. Default value is `/var/lib/clickhouse/disks/<disk_name>/cache/`.
- `skip_access_check` — If true disk access checks will not be performed on disk start-up. Default value is `false`.
- `skip_access_check` — If true, disk access checks will not be performed on disk start-up. Default value is `false`.
- `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set.
S3 disk can be configured as `main` or `cold` storage:

View File

@ -39,4 +39,4 @@ More details on [manipulating partitions](../../sql-reference/statements/alter/p
Its rather radical to drop all data from a table, but in some cases it might be exactly what you need.
More details on [table truncation](../../sql-reference/statements/alter/partition.md#alter_drop-partition).
More details on [table truncation](../../sql-reference/statements/truncate.md).

View File

@ -644,7 +644,7 @@ If there are no replicas at the moment on replicated table creation, a new first
``` sql
CREATE TABLE tutorial.hits_replica (...)
ENGINE = ReplcatedMergeTree(
ENGINE = ReplicatedMergeTree(
'/clickhouse_perftest/tables/{shard}/hits',
'{replica}'
)

View File

@ -31,8 +31,8 @@ The supported formats are:
| [JSONCompactString](#jsoncompactstring) | ✗ | ✔ |
| [JSONEachRow](#jsoneachrow) | ✔ | ✔ |
| [JSONEachRowWithProgress](#jsoneachrowwithprogress) | ✗ | ✔ |
| [JSONStringEachRow](#jsonstringeachrow) | ✔ | ✔ |
| [JSONStringEachRowWithProgress](#jsonstringeachrowwithprogress) | ✗ | ✔ |
| [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ |
| [JSONStringsEachRowWithProgress](#jsonstringseachrowwithprogress) | ✗ | ✔ |
| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ |
| [JSONCompactEachRowWithNamesAndTypes](#jsoncompacteachrowwithnamesandtypes) | ✔ | ✔ |
| [JSONCompactStringEachRow](#jsoncompactstringeachrow) | ✔ | ✔ |
@ -612,7 +612,7 @@ Example:
```
## JSONEachRow {#jsoneachrow}
## JSONStringEachRow {#jsonstringeachrow}
## JSONStringsEachRow {#jsonstringseachrow}
## JSONCompactEachRow {#jsoncompacteachrow}
## JSONCompactStringEachRow {#jsoncompactstringeachrow}
@ -627,9 +627,9 @@ When using these formats, ClickHouse outputs rows as separated, newline-delimite
When inserting the data, you should provide a separate JSON value for each row.
## JSONEachRowWithProgress {#jsoneachrowwithprogress}
## JSONStringEachRowWithProgress {#jsonstringeachrowwithprogress}
## JSONStringsEachRowWithProgress {#jsonstringseachrowwithprogress}
Differs from `JSONEachRow`/`JSONStringEachRow` in that ClickHouse will also yield progress information as JSON values.
Differs from `JSONEachRow`/`JSONStringsEachRow` in that ClickHouse will also yield progress information as JSON values.
```json
{"row":{"'hello'":"hello","multiply(42, number)":"0","range(5)":[0,1,2,3,4]}}

View File

@ -5,7 +5,7 @@ toc_title: Data Backup
# Data Backup {#data-backup}
While [replication](../engines/table-engines/mergetree-family/replication.md) provides protection from hardware failures, it does not protect against human errors: accidental deletion of data, deletion of the wrong table or a table on the wrong cluster, and software bugs that result in incorrect data processing or data corruption. In many cases mistakes like these will affect all replicas. ClickHouse has built-in safeguards to prevent some types of mistakes — for example, by default [you cant just drop tables with a MergeTree-like engine containing more than 50 Gb of data](https://github.com/ClickHouse/ClickHouse/blob/v18.14.18-stable/programs/server/config.xml#L322-L330). However, these safeguards dont cover all possible cases and can be circumvented.
While [replication](../engines/table-engines/mergetree-family/replication.md) provides protection from hardware failures, it does not protect against human errors: accidental deletion of data, deletion of the wrong table or a table on the wrong cluster, and software bugs that result in incorrect data processing or data corruption. In many cases mistakes like these will affect all replicas. ClickHouse has built-in safeguards to prevent some types of mistakes — for example, by default [you cant just drop tables with a MergeTree-like engine containing more than 50 Gb of data](server-configuration-parameters/settings.md#max-table-size-to-drop). However, these safeguards dont cover all possible cases and can be circumvented.
In order to effectively mitigate possible human errors, you should carefully prepare a strategy for backing up and restoring your data **in advance**.

View File

@ -312,7 +312,7 @@ Enables or disables parsing enum values as enum ids for TSV input format.
Possible values:
- 0 — Enum values are parsed as values.
- 1 — Enum values are parsed as enum IDs
- 1 — Enum values are parsed as enum IDs.
Default value: 0.
@ -2592,6 +2592,18 @@ Possible values:
Default value: `16`.
## opentelemetry_start_trace_probability {#opentelemetry-start-trace-probability}
Sets the probability that the ClickHouse can start a trace for executed queries (if no parent [trace context](https://www.w3.org/TR/trace-context/) is supplied).
Possible values:
- 0 — The trace for all executed queries is disabled (if no parent trace context is supplied).
- Positive floating-point number in the range [0..1]. For example, if the setting value is `0,5`, ClickHouse can start a trace on average for half of the queries.
- 1 — The trace for all executed queries is enabled.
Default value: `0`.
## optimize_on_insert {#optimize-on-insert}
Enables or disables data transformation before the insertion, as if merge was done on this block (according to table engine).

View File

@ -0,0 +1,53 @@
# system.opentelemetry_span_log {#system_tables-opentelemetry_span_log}
Contains information about [trace spans](https://opentracing.io/docs/overview/spans/) for executed queries.
Columns:
- `trace_id` ([UUID](../../sql-reference/data-types/uuid.md) — ID of the trace for executed query.
- `span_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of the `trace span`.
- `parent_span_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of the parent `trace span`.
- `operation_name` ([String](../../sql-reference/data-types/string.md)) — The name of the operation.
- `start_time_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The start time of the `trace span` (in microseconds).
- `finish_time_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The finish time of the `trace span` (in microseconds).
- `finish_date` ([Date](../../sql-reference/data-types/date.md)) — The finish date of the `trace span`.
- `attribute.names` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — [Attribute](https://opentelemetry.io/docs/go/instrumentation/#attributes) names depending on the `trace span`. They are filled in according to the recommendations in the [OpenTelemetry](https://opentelemetry.io/) standard.
- `attribute.values` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Attribute values depending on the `trace span`. They are filled in according to the recommendations in the `OpenTelemetry` standard.
**Example**
Query:
``` sql
SELECT * FROM system.opentelemetry_span_log LIMIT 1 FORMAT Vertical;
```
Result:
``` text
Row 1:
──────
trace_id: cdab0847-0d62-61d5-4d38-dd65b19a1914
span_id: 701487461015578150
parent_span_id: 2991972114672045096
operation_name: DB::Block DB::InterpreterSelectQuery::getSampleBlockImpl()
start_time_us: 1612374594529090
finish_time_us: 1612374594529108
finish_date: 2021-02-03
attribute.names: []
attribute.values: []
```
**See Also**
- [OpenTelemetry](../../operations/opentelemetry.md)
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/opentelemetry_span_log) <!--hide-->

View File

@ -1,9 +1,9 @@
---
toc_priority: 47
toc_title: ClickHouse Update
toc_title: ClickHouse Upgrade
---
# ClickHouse Update {#clickhouse-update}
# ClickHouse Upgrade {#clickhouse-upgrade}
If ClickHouse was installed from `deb` packages, execute the following commands on the server:
@ -16,3 +16,19 @@ $ sudo service clickhouse-server restart
If you installed ClickHouse using something other than the recommended `deb` packages, use the appropriate update method.
ClickHouse does not support a distributed update. The operation should be performed consecutively on each separate server. Do not update all the servers on a cluster simultaneously, or the cluster will be unavailable for some time.
The upgrade of older version of ClickHouse to specific version:
As an example:
`xx.yy.a.b` is a current stable version. The latest stable version could be found [here](https://github.com/ClickHouse/ClickHouse/releases)
```bash
$ sudo apt-get update
$ sudo apt-get install clickhouse-server=xx.yy.a.b clickhouse-client=xx.yy.a.b clickhouse-common-static=xx.yy.a.b
$ sudo service clickhouse-server restart
```

View File

@ -72,7 +72,7 @@ If an aggregate function doesnt have input values, with this combinator it re
<aggFunction>OrDefault(x)
```
**Parameters**
**Arguments**
- `x` — Aggregate function parameters.
@ -132,7 +132,7 @@ This combinator converts a result of an aggregate function to the [Nullable](../
<aggFunction>OrNull(x)
```
**Parameters**
**Arguments**
- `x` — Aggregate function parameters.
@ -189,7 +189,7 @@ Lets you divide data into groups, and then separately aggregates the data in tho
<aggFunction>Resample(start, end, step)(<aggFunction_params>, resampling_key)
```
**Parameters**
**Arguments**
- `start` — Starting value of the whole required interval for `resampling_key` values.
- `stop` — Ending value of the whole required interval for `resampling_key` values. The whole interval doesnt include the `stop` value `[start, stop)`.

View File

@ -17,10 +17,13 @@ histogram(number_of_bins)(values)
The functions uses [A Streaming Parallel Decision Tree Algorithm](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf). The borders of histogram bins are adjusted as new data enters a function. In common case, the widths of bins are not equal.
**Arguments**
`values` — [Expression](../../sql-reference/syntax.md#syntax-expressions) resulting in input values.
**Parameters**
`number_of_bins` — Upper limit for the number of bins in the histogram. The function automatically calculates the number of bins. It tries to reach the specified number of bins, but if it fails, it uses fewer bins.
`values` — [Expression](../../sql-reference/syntax.md#syntax-expressions) resulting in input values.
**Returned values**
@ -89,14 +92,16 @@ sequenceMatch(pattern)(timestamp, cond1, cond2, ...)
!!! warning "Warning"
Events that occur at the same second may lay in the sequence in an undefined order affecting the result.
**Parameters**
- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax).
**Arguments**
- `timestamp` — Column considered to contain time data. Typical data types are `Date` and `DateTime`. You can also use any of the supported [UInt](../../sql-reference/data-types/int-uint.md) data types.
- `cond1`, `cond2` — Conditions that describe the chain of events. Data type: `UInt8`. You can pass up to 32 condition arguments. The function takes only the events described in these conditions into account. If the sequence contains data that isnt described in a condition, the function skips them.
**Parameters**
- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax).
**Returned values**
- 1, if the pattern is matched.
@ -176,14 +181,16 @@ Counts the number of event chains that matched the pattern. The function searche
sequenceCount(pattern)(timestamp, cond1, cond2, ...)
```
**Parameters**
- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax).
**Arguments**
- `timestamp` — Column considered to contain time data. Typical data types are `Date` and `DateTime`. You can also use any of the supported [UInt](../../sql-reference/data-types/int-uint.md) data types.
- `cond1`, `cond2` — Conditions that describe the chain of events. Data type: `UInt8`. You can pass up to 32 condition arguments. The function takes only the events described in these conditions into account. If the sequence contains data that isnt described in a condition, the function skips them.
**Parameters**
- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax).
**Returned values**
- Number of non-overlapping event chains that are matched.
@ -239,13 +246,16 @@ The function works according to the algorithm:
windowFunnel(window, [mode])(timestamp, cond1, cond2, ..., condN)
```
**Arguments**
- `timestamp` — Name of the column containing the timestamp. Data types supported: [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md#data_type-datetime) and other unsigned integer types (note that even though timestamp supports the `UInt64` type, its value cant exceed the Int64 maximum, which is 2^63 - 1).
- `cond` — Conditions or data describing the chain of events. [UInt8](../../sql-reference/data-types/int-uint.md).
**Parameters**
- `window` — Length of the sliding window. The unit of `window` depends on the timestamp itself and varies. Determined using the expression `timestamp of cond2 <= timestamp of cond1 + window`.
- `mode` - It is an optional argument.
- `mode` - It is an optional parameter.
- `'strict'` - When the `'strict'` is set, the windowFunnel() applies conditions only for the unique values.
- `timestamp` — Name of the column containing the timestamp. Data types supported: [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md#data_type-datetime) and other unsigned integer types (note that even though timestamp supports the `UInt64` type, its value cant exceed the Int64 maximum, which is 2^63 - 1).
- `cond` — Conditions or data describing the chain of events. [UInt8](../../sql-reference/data-types/int-uint.md).
**Returned value**
@ -324,7 +334,7 @@ The conditions, except the first, apply in pairs: the result of the second will
retention(cond1, cond2, ..., cond32);
```
**Parameters**
**Arguments**
- `cond` — an expression that returns a `UInt8` result (1 or 0).

View File

@ -20,7 +20,7 @@ or
argMax(tuple(arg, val))
```
**Parameters**
**Arguments**
- `arg` — Argument.
- `val` — Value.

View File

@ -20,7 +20,7 @@ or
argMin(tuple(arg, val))
```
**Parameters**
**Arguments**
- `arg` — Argument.
- `val` — Value.

View File

@ -12,7 +12,7 @@ Calculates the arithmetic mean.
avgWeighted(x)
```
**Parameter**
**Arguments**
- `x` — Values.

View File

@ -12,7 +12,7 @@ Calculates the [weighted arithmetic mean](https://en.wikipedia.org/wiki/Weighted
avgWeighted(x, weight)
```
**Parameters**
**Arguments**
- `x` — Values.
- `weight` — Weights of the values.

View File

@ -10,7 +10,7 @@ ClickHouse supports the following syntaxes for `count`:
- `count(expr)` or `COUNT(DISTINCT expr)`.
- `count()` or `COUNT(*)`. The `count()` syntax is ClickHouse-specific.
**Parameters**
**Arguments**
The function can take:

View File

@ -17,7 +17,7 @@ If in one query several values are inserted into the same position, the function
- If a query is executed in a single thread, the first one of the inserted values is used.
- If a query is executed in multiple threads, the resulting value is an undetermined one of the inserted values.
**Parameters**
**Arguments**
- `x` — Value to be inserted. [Expression](../../../sql-reference/syntax.md#syntax-expressions) resulting in one of the [supported data types](../../../sql-reference/data-types/index.md).
- `pos` — Position at which the specified element `x` is to be inserted. Index numbering in the array starts from zero. [UInt32](../../../sql-reference/data-types/int-uint.md#uint-ranges).

View File

@ -13,7 +13,7 @@ groupArrayMovingAvg(window_size)(numbers_for_summing)
The function can take the window size as a parameter. If left unspecified, the function takes the window size equal to the number of rows in the column.
**Parameters**
**Arguments**
- `numbers_for_summing` — [Expression](../../../sql-reference/syntax.md#syntax-expressions) resulting in a numeric data type value.
- `window_size` — Size of the calculation window.

View File

@ -13,7 +13,7 @@ groupArrayMovingSum(window_size)(numbers_for_summing)
The function can take the window size as a parameter. If left unspecified, the function takes the window size equal to the number of rows in the column.
**Parameters**
**Arguments**
- `numbers_for_summing` — [Expression](../../../sql-reference/syntax.md#syntax-expressions) resulting in a numeric data type value.
- `window_size` — Size of the calculation window.

View File

@ -12,7 +12,7 @@ Creates an array of sample argument values. The size of the resulting array is l
groupArraySample(max_size[, seed])(x)
```
**Parameters**
**Arguments**
- `max_size` — Maximum size of the resulting array. [UInt64](../../data-types/int-uint.md).
- `seed` — Seed for the random number generator. Optional. [UInt64](../../data-types/int-uint.md). Default value: `123456`.

View File

@ -10,7 +10,7 @@ Applies bitwise `AND` for series of numbers.
groupBitAnd(expr)
```
**Parameters**
**Arguments**
`expr` An expression that results in `UInt*` type.

View File

@ -10,7 +10,7 @@ Bitmap or Aggregate calculations from a unsigned integer column, return cardinal
groupBitmap(expr)
```
**Parameters**
**Arguments**
`expr` An expression that results in `UInt*` type.

View File

@ -10,7 +10,7 @@ Calculations the AND of a bitmap column, return cardinality of type UInt64, if a
groupBitmapAnd(expr)
```
**Parameters**
**Arguments**
`expr` An expression that results in `AggregateFunction(groupBitmap, UInt*)` type.

View File

@ -10,7 +10,7 @@ Calculations the OR of a bitmap column, return cardinality of type UInt64, if ad
groupBitmapOr(expr)
```
**Parameters**
**Arguments**
`expr` An expression that results in `AggregateFunction(groupBitmap, UInt*)` type.

View File

@ -10,7 +10,7 @@ Calculations the XOR of a bitmap column, return cardinality of type UInt64, if a
groupBitmapOr(expr)
```
**Parameters**
**Arguments**
`expr` An expression that results in `AggregateFunction(groupBitmap, UInt*)` type.

View File

@ -10,7 +10,7 @@ Applies bitwise `OR` for series of numbers.
groupBitOr(expr)
```
**Parameters**
**Arguments**
`expr` An expression that results in `UInt*` type.

View File

@ -10,7 +10,7 @@ Applies bitwise `XOR` for series of numbers.
groupBitXor(expr)
```
**Parameters**
**Arguments**
`expr` An expression that results in `UInt*` type.

View File

@ -13,7 +13,7 @@ Use it for tests or to process columns of types `AggregateFunction` and `Aggrega
initializeAggregation (aggregate_function, column_1, column_2);
```
**Parameters**
**Arguments**
- `aggregate_function` — Name of the aggregation function. The state of this function — the creating one. [String](../../../sql-reference/data-types/string.md#string).
- `column_n` — The column to translate it into the function as it's argument. [String](../../../sql-reference/data-types/string.md#string).

View File

@ -10,7 +10,7 @@ Computes the [kurtosis](https://en.wikipedia.org/wiki/Kurtosis) of a sequence.
kurtPop(expr)
```
**Parameters**
**Arguments**
`expr` — [Expression](../../../sql-reference/syntax.md#syntax-expressions) returning a number.

View File

@ -12,7 +12,7 @@ It represents an unbiased estimate of the kurtosis of a random variable if passe
kurtSamp(expr)
```
**Parameters**
**Arguments**
`expr` — [Expression](../../../sql-reference/syntax.md#syntax-expressions) returning a number.

View File

@ -16,6 +16,11 @@ mannWhitneyUTest[(alternative[, continuity_correction])](sample_data, sample_ind
Values of both samples are in the `sample_data` column. If `sample_index` equals to 0 then the value in that row belongs to the sample from the first population. Otherwise it belongs to the sample from the second population.
The null hypothesis is that two populations are stochastically equal. Also one-sided hypothesises can be tested. This test does not assume that data have normal distribution.
**Arguments**
- `sample_data` — sample data. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md).
- `sample_index` — sample index. [Integer](../../../sql-reference/data-types/int-uint.md).
**Parameters**
- `alternative` — alternative hypothesis. (Optional, default: `'two-sided'`.) [String](../../../sql-reference/data-types/string.md).
@ -23,9 +28,6 @@ The null hypothesis is that two populations are stochastically equal. Also one-s
- `'greater'`;
- `'less'`.
- `continuity_correction` - if not 0 then continuity correction in the normal approximation for the p-value is applied. (Optional, default: 1.) [UInt64](../../../sql-reference/data-types/int-uint.md).
- `sample_data` — sample data. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md).
- `sample_index` — sample index. [Integer](../../../sql-reference/data-types/int-uint.md).
**Returned values**

View File

@ -18,7 +18,7 @@ quantile(level)(expr)
Alias: `median`.
**Parameters**
**Arguments**
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md).

View File

@ -18,7 +18,7 @@ quantileDeterministic(level)(expr, determinator)
Alias: `medianDeterministic`.
**Parameters**
**Arguments**
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md).

View File

@ -18,7 +18,7 @@ quantileExact(level)(expr)
Alias: `medianExact`.
**Parameters**
**Arguments**
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md).
@ -77,7 +77,7 @@ quantileExact(level)(expr)
Alias: `medianExactLow`.
**Parameters**
**Arguments**
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md).
@ -128,7 +128,7 @@ quantileExactHigh(level)(expr)
Alias: `medianExactHigh`.
**Parameters**
**Arguments**
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md).

View File

@ -18,7 +18,7 @@ quantileExactWeighted(level)(expr, weight)
Alias: `medianExactWeighted`.
**Parameters**
**Arguments**
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md).

View File

@ -20,7 +20,7 @@ quantileTDigest(level)(expr)
Alias: `medianTDigest`.
**Parameters**
**Arguments**
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md).

View File

@ -20,7 +20,7 @@ quantileTDigest(level)(expr)
Alias: `medianTDigest`.
**Parameters**
**Arguments**
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md).

View File

@ -18,7 +18,7 @@ quantileTiming(level)(expr)
Alias: `medianTiming`.
**Parameters**
**Arguments**
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).

View File

@ -18,7 +18,7 @@ quantileTimingWeighted(level)(expr, weight)
Alias: `medianTimingWeighted`.
**Parameters**
**Arguments**
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).

View File

@ -8,7 +8,7 @@ Computes a rank correlation coefficient.
rankCorr(x, y)
```
**Parameters**
**Arguments**
- `x` — Arbitrary value. [Float32](../../../sql-reference/data-types/float.md#float32-float64) or [Float64](../../../sql-reference/data-types/float.md#float32-float64).
- `y` — Arbitrary value. [Float32](../../../sql-reference/data-types/float.md#float32-float64) or [Float64](../../../sql-reference/data-types/float.md#float32-float64).

View File

@ -10,7 +10,7 @@ Computes the [skewness](https://en.wikipedia.org/wiki/Skewness) of a sequence.
skewPop(expr)
```
**Parameters**
**Arguments**
`expr` — [Expression](../../../sql-reference/syntax.md#syntax-expressions) returning a number.

View File

@ -12,7 +12,7 @@ It represents an unbiased estimate of the skewness of a random variable if passe
skewSamp(expr)
```
**Parameters**
**Arguments**
`expr` — [Expression](../../../sql-reference/syntax.md#syntax-expressions) returning a number.

View File

@ -16,7 +16,7 @@ studentTTest(sample_data, sample_index)
Values of both samples are in the `sample_data` column. If `sample_index` equals to 0 then the value in that row belongs to the sample from the first population. Otherwise it belongs to the sample from the second population.
The null hypothesis is that means of populations are equal. Normal distribution with equal variances is assumed.
**Parameters**
**Arguments**
- `sample_data` — sample data. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md).
- `sample_index` — sample index. [Integer](../../../sql-reference/data-types/int-uint.md).

View File

@ -16,7 +16,7 @@ This function doesnt provide a guaranteed result. In certain situations, erro
We recommend using the `N < 10` value; performance is reduced with large `N` values. Maximum value of `N = 65536`.
**Parameters**
**Arguments**
- N is the number of elements to return.

View File

@ -12,7 +12,7 @@ Similar to `topK` but takes one additional argument of integer type - `weight`.
topKWeighted(N)(x, weight)
```
**Parameters**
**Arguments**
- `N` — The number of elements to return.

View File

@ -10,7 +10,7 @@ Calculates the approximate number of different values of the argument.
uniq(x[, ...])
```
**Parameters**
**Arguments**
The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types.

View File

@ -12,7 +12,7 @@ uniqCombined(HLL_precision)(x[, ...])
The `uniqCombined` function is a good choice for calculating the number of different values.
**Parameters**
**Arguments**
The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types.

View File

@ -14,7 +14,7 @@ Use the `uniqExact` function if you absolutely need an exact result. Otherwise u
The `uniqExact` function uses more memory than `uniq`, because the size of the state has unbounded growth as the number of different values increases.
**Parameters**
**Arguments**
The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types.

View File

@ -10,7 +10,7 @@ Calculates the approximate number of different argument values, using the [Hyper
uniqHLL12(x[, ...])
```
**Parameters**
**Arguments**
The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types.

View File

@ -16,7 +16,7 @@ welchTTest(sample_data, sample_index)
Values of both samples are in the `sample_data` column. If `sample_index` equals to 0 then the value in that row belongs to the sample from the first population. Otherwise it belongs to the sample from the second population.
The null hypothesis is that means of populations are equal. Normal distribution is assumed. Populations may have unequal variance.
**Parameters**
**Arguments**
- `sample_data` — sample data. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md).
- `sample_index` — sample index. [Integer](../../../sql-reference/data-types/int-uint.md).

View File

@ -0,0 +1,83 @@
---
toc_priority: 65
toc_title: Map(key, value)
---
# Map(key, value) {#data_type-map}
`Map(key, value)` data type stores `key:value` pairs.
**Parameters**
- `key` — The key part of the pair. [String](../../sql-reference/data-types/string.md) or [Integer](../../sql-reference/data-types/int-uint.md).
- `value` — The value part of the pair. [String](../../sql-reference/data-types/string.md), [Integer](../../sql-reference/data-types/int-uint.md) or [Array](../../sql-reference/data-types/array.md).
!!! warning "Warning"
Currently `Map` data type is an experimental feature. To work with it you must set `allow_experimental_map_type = 1`.
To get the value from an `a Map('key', 'value')` column, use `a['key']` syntax. This lookup works now with a linear complexity.
**Examples**
Consider the table:
``` sql
CREATE TABLE table_map (a Map(String, UInt64)) ENGINE=Memory;
INSERT INTO table_map VALUES ({'key1':1, 'key2':10}), ({'key1':2,'key2':20}), ({'key1':3,'key2':30});
```
Select all `key2` values:
```sql
SELECT a['key2'] FROM table_map;
```
Result:
```text
┌─arrayElement(a, 'key2')─┐
│ 10 │
│ 20 │
│ 30 │
└─────────────────────────┘
```
If there's no such `key` in the `Map()` column, the query returns zeros for numerical values, empty strings or empty arrays.
```sql
INSERT INTO table_map VALUES ({'key3':100}), ({});
SELECT a['key3'] FROM table_map;
```
Result:
```text
┌─arrayElement(a, 'key3')─┐
│ 100 │
│ 0 │
└─────────────────────────┘
┌─arrayElement(a, 'key3')─┐
│ 0 │
│ 0 │
│ 0 │
└─────────────────────────┘
```
## Convert Tuple to Map Type {#map-and-tuple}
You can cast `Tuple()` as `Map()` using [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) function:
``` sql
SELECT CAST(([1, 2, 3], ['Ready', 'Steady', 'Go']), 'Map(UInt8, String)') AS map;
```
``` text
┌─map───────────────────────────┐
│ {1:'Ready',2:'Steady',3:'Go'} │
└───────────────────────────────┘
```
**See Also**
- [map()](../../sql-reference/functions/tuple-map-functions.md#function-map) function
- [CAST()](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) function
[Original article](https://clickhouse.tech/docs/en/data-types/map/) <!--hide-->

View File

@ -61,7 +61,7 @@ Combines arrays passed as arguments.
arrayConcat(arrays)
```
**Parameters**
**Arguments**
- `arrays` Arbitrary number of arguments of [Array](../../sql-reference/data-types/array.md) type.
**Example**
@ -111,7 +111,7 @@ Checks whether one array is a subset of another.
hasAll(set, subset)
```
**Parameters**
**Arguments**
- `set` Array of any type with a set of elements.
- `subset` Array of any type with elements that should be tested to be a subset of `set`.
@ -149,7 +149,7 @@ Checks whether two arrays have intersection by some elements.
hasAny(array1, array2)
```
**Parameters**
**Arguments**
- `array1` Array of any type with a set of elements.
- `array2` Array of any type with a set of elements.
@ -191,7 +191,7 @@ 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`.
**Parameters**
**Arguments**
- `array1` Array of any type with a set of elements.
- `array2` Array of any type with a set of elements.
@ -369,7 +369,7 @@ Removes the last item from the array.
arrayPopBack(array)
```
**Parameters**
**Arguments**
- `array` Array.
@ -393,7 +393,7 @@ Removes the first item from the array.
arrayPopFront(array)
```
**Parameters**
**Arguments**
- `array` Array.
@ -417,7 +417,7 @@ Adds one item to the end of the array.
arrayPushBack(array, single_value)
```
**Parameters**
**Arguments**
- `array` Array.
- `single_value` A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about the types of data in ClickHouse, see “[Data types](../../sql-reference/data-types/index.md#data_types)”. Can be `NULL`. The function adds a `NULL` element to an array, and the type of array elements converts to `Nullable`.
@ -442,7 +442,7 @@ Adds one element to the beginning of the array.
arrayPushFront(array, single_value)
```
**Parameters**
**Arguments**
- `array` Array.
- `single_value` A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about the types of data in ClickHouse, see “[Data types](../../sql-reference/data-types/index.md#data_types)”. Can be `NULL`. The function adds a `NULL` element to an array, and the type of array elements converts to `Nullable`.
@ -467,7 +467,7 @@ Changes the length of the array.
arrayResize(array, size[, extender])
```
**Parameters:**
**Arguments:**
- `array` — Array.
- `size` — Required length of the array.
@ -509,7 +509,7 @@ Returns a slice of the array.
arraySlice(array, offset[, length])
```
**Parameters**
**Arguments**
- `array` Array of data.
- `offset` Indent from the edge of the array. A positive value indicates an offset on the left, and a negative value is an indent on the right. Numbering of the array items begins with 1.
@ -751,7 +751,7 @@ Calculates the difference between adjacent array elements. Returns an array wher
arrayDifference(array)
```
**Parameters**
**Arguments**
- `array` [Array](https://clickhouse.tech/docs/en/data_types/array/).
@ -803,7 +803,7 @@ Takes an array, returns an array containing the distinct elements only.
arrayDistinct(array)
```
**Parameters**
**Arguments**
- `array` [Array](https://clickhouse.tech/docs/en/data_types/array/).
@ -871,7 +871,7 @@ Applies an aggregate function to array elements and returns its result. The name
arrayReduce(agg_func, arr1, arr2, ..., arrN)
```
**Parameters**
**Arguments**
- `agg_func` — The name of an aggregate function which should be a constant [string](../../sql-reference/data-types/string.md).
- `arr` — Any number of [array](../../sql-reference/data-types/array.md) type columns as the parameters of the aggregation function.
@ -936,7 +936,7 @@ Applies an aggregate function to array elements in given ranges and returns an a
arrayReduceInRanges(agg_func, ranges, arr1, arr2, ..., arrN)
```
**Parameters**
**Arguments**
- `agg_func` — The name of an aggregate function which should be a constant [string](../../sql-reference/data-types/string.md).
- `ranges` — The ranges to aggretate which should be an [array](../../sql-reference/data-types/array.md) of [tuples](../../sql-reference/data-types/tuple.md) which containing the index and the length of each range.
@ -1007,7 +1007,7 @@ flatten(array_of_arrays)
Alias: `flatten`.
**Parameters**
**Arguments**
- `array_of_arrays` — [Array](../../sql-reference/data-types/array.md) of arrays. For example, `[[1,2,3], [4,5]]`.
@ -1033,7 +1033,7 @@ Removes consecutive duplicate elements from an array. The order of result values
arrayCompact(arr)
```
**Parameters**
**Arguments**
`arr` — The [array](../../sql-reference/data-types/array.md) to inspect.
@ -1069,7 +1069,7 @@ Combines multiple arrays into a single array. The resulting array contains the c
arrayZip(arr1, arr2, ..., arrN)
```
**Parameters**
**Arguments**
- `arrN` — [Array](../../sql-reference/data-types/array.md).
@ -1107,7 +1107,7 @@ Calculate AUC (Area Under the Curve, which is a concept in machine learning, see
arrayAUC(arr_scores, arr_labels)
```
**Parameters**
**Arguments**
- `arr_scores` — scores prediction model gives.
- `arr_labels` — labels of samples, usually 1 for positive sample and 0 for negtive sample.
@ -1302,7 +1302,7 @@ Note that the `arrayMin` is a [higher-order function](../../sql-reference/functi
arrayMin([func,] arr)
```
**Parameters**
**Arguments**
- `func` — Function. [Expression](../../sql-reference/data-types/special-data-types/expression.md).
- `arr` — Array. [Array](../../sql-reference/data-types/array.md).
@ -1357,7 +1357,7 @@ Note that the `arrayMax` is a [higher-order function](../../sql-reference/functi
arrayMax([func,] arr)
```
**Parameters**
**Arguments**
- `func` — Function. [Expression](../../sql-reference/data-types/special-data-types/expression.md).
- `arr` — Array. [Array](../../sql-reference/data-types/array.md).
@ -1412,7 +1412,7 @@ Note that the `arraySum` is a [higher-order function](../../sql-reference/functi
arraySum([func,] arr)
```
**Parameters**
**Arguments**
- `func` — Function. [Expression](../../sql-reference/data-types/special-data-types/expression.md).
- `arr` — Array. [Array](../../sql-reference/data-types/array.md).
@ -1467,7 +1467,7 @@ Note that the `arrayAvg` is a [higher-order function](../../sql-reference/functi
arrayAvg([func,] arr)
```
**Parameters**
**Arguments**
- `func` — Function. [Expression](../../sql-reference/data-types/special-data-types/expression.md).
- `arr` — Array. [Array](../../sql-reference/data-types/array.md).

View File

@ -35,7 +35,7 @@ Takes any integer and converts it into [binary form](https://en.wikipedia.org/wi
SELECT bitTest(number, index)
```
**Parameters**
**Arguments**
- `number` integer number.
- `index` position of bit.
@ -100,7 +100,7 @@ The conjuction for bitwise operations:
SELECT bitTestAll(number, index1, index2, index3, index4, ...)
```
**Parameters**
**Arguments**
- `number` integer number.
- `index1`, `index2`, `index3`, `index4` positions of bit. For example, for set of positions (`index1`, `index2`, `index3`, `index4`) is true if and only if all of its positions are true (`index1` ⋀ `index2`, ⋀ `index3``index4`).
@ -165,7 +165,7 @@ The disjunction for bitwise operations:
SELECT bitTestAny(number, index1, index2, index3, index4, ...)
```
**Parameters**
**Arguments**
- `number` integer number.
- `index1`, `index2`, `index3`, `index4` positions of bit.
@ -220,7 +220,7 @@ Calculates the number of bits set to one in the binary representation of a numbe
bitCount(x)
```
**Parameters**
**Arguments**
- `x` — [Integer](../../sql-reference/data-types/int-uint.md) or [floating-point](../../sql-reference/data-types/float.md) number. The function uses the value representation in memory. It allows supporting floating-point numbers.

View File

@ -21,7 +21,7 @@ Build a bitmap from unsigned integer array.
bitmapBuild(array)
```
**Parameters**
**Arguments**
- `array` unsigned integer array.
@ -45,7 +45,7 @@ Convert bitmap to integer array.
bitmapToArray(bitmap)
```
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -69,7 +69,7 @@ Return subset in specified range (not include the range_end).
bitmapSubsetInRange(bitmap, range_start, range_end)
```
**Parameters**
**Arguments**
- `bitmap` [Bitmap object](#bitmap_functions-bitmapbuild).
- `range_start` range start point. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
@ -97,7 +97,7 @@ Creates a subset of bitmap with n elements taken between `range_start` and `card
bitmapSubsetLimit(bitmap, range_start, cardinality_limit)
```
**Parameters**
**Arguments**
- `bitmap` [Bitmap object](#bitmap_functions-bitmapbuild).
- `range_start` The subset starting point. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
@ -133,7 +133,7 @@ Checks whether the bitmap contains an element.
bitmapContains(haystack, needle)
```
**Parameters**
**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).
@ -167,7 +167,7 @@ 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.
**Parameters**
**Arguments**
- `bitmap*` bitmap object.
@ -197,7 +197,7 @@ If the second argument is an empty bitmap then returns 1.
bitmapHasAll(bitmap,bitmap)
```
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -221,7 +221,7 @@ Retrun bitmap cardinality of type UInt64.
bitmapCardinality(bitmap)
```
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -243,7 +243,7 @@ Retrun the smallest value of type UInt64 in the set, UINT32_MAX if the set is em
bitmapMin(bitmap)
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -263,7 +263,7 @@ Retrun the greatest value of type UInt64 in the set, 0 if the set is empty.
bitmapMax(bitmap)
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -283,7 +283,7 @@ Transform an array of values in a bitmap to another array of values, the result
bitmapTransform(bitmap, from_array, to_array)
**Parameters**
**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.
@ -307,7 +307,7 @@ Two bitmap and calculation, the result is a new bitmap.
bitmapAnd(bitmap,bitmap)
```
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -331,7 +331,7 @@ Two bitmap or calculation, the result is a new bitmap.
bitmapOr(bitmap,bitmap)
```
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -355,7 +355,7 @@ Two bitmap xor calculation, the result is a new bitmap.
bitmapXor(bitmap,bitmap)
```
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -379,7 +379,7 @@ Two bitmap andnot calculation, the result is a new bitmap.
bitmapAndnot(bitmap,bitmap)
```
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -403,7 +403,7 @@ Two bitmap and calculation, return cardinality of type UInt64.
bitmapAndCardinality(bitmap,bitmap)
```
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -427,7 +427,7 @@ Two bitmap or calculation, return cardinality of type UInt64.
bitmapOrCardinality(bitmap,bitmap)
```
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -451,7 +451,7 @@ Two bitmap xor calculation, return cardinality of type UInt64.
bitmapXorCardinality(bitmap,bitmap)
```
**Parameters**
**Arguments**
- `bitmap` bitmap object.
@ -475,7 +475,7 @@ Two bitmap andnot calculation, return cardinality of type UInt64.
bitmapAndnotCardinality(bitmap,bitmap)
```
**Parameters**
**Arguments**
- `bitmap` bitmap object.

View File

@ -17,7 +17,7 @@ SELECT 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.
**Parameters**
**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.
@ -117,7 +117,7 @@ Allows you to write the [CASE](../../sql-reference/operators/index.md#operator_c
Syntax: `multiIf(cond_1, then_1, cond_2, then_2, ..., else)`
**Parameters:**
**Arguments:**
- `cond_N` — The condition for the function to return `then_N`.
- `then_N` — The result of the function when executed.

View File

@ -186,7 +186,7 @@ Truncates sub-seconds.
toStartOfSecond(value[, timezone])
```
**Parameters**
**Arguments**
- `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md).
- `timezone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../../sql-reference/data-types/string.md).
@ -328,7 +328,7 @@ For mode values with a meaning of “contains January 1”, the week contains Ja
toWeek(date, [, mode][, Timezone])
```
**Parameters**
**Arguments**
- `date` Date or DateTime.
- `mode` Optional parameter, Range of values is \[0,9\], default is 0.
@ -378,7 +378,7 @@ date_trunc(unit, value[, timezone])
Alias: `dateTrunc`.
**Parameters**
**Arguments**
- `unit` — The type of interval to truncate the result. [String Literal](../syntax.md#syntax-string-literal).
Possible values:
@ -447,7 +447,7 @@ date_add(unit, value, date)
Aliases: `dateAdd`, `DATE_ADD`.
**Parameters**
**Arguments**
- `unit` — The type of interval to add. [String](../../sql-reference/data-types/string.md).
@ -484,7 +484,7 @@ date_diff('unit', startdate, enddate, [timezone])
Aliases: `dateDiff`, `DATE_DIFF`.
**Parameters**
**Arguments**
- `unit` — The type of interval for result [String](../../sql-reference/data-types/string.md).
@ -530,7 +530,7 @@ date_sub(unit, value, date)
Aliases: `dateSub`, `DATE_SUB`.
**Parameters**
**Arguments**
- `unit` — The type of interval to subtract. [String](../../sql-reference/data-types/string.md).
@ -570,7 +570,7 @@ timestamp_add(date, INTERVAL value unit)
Aliases: `timeStampAdd`, `TIMESTAMP_ADD`.
**Parameters**
**Arguments**
- `date` — Date or Date with time - [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md).
- `value` - Value in specified unit - [Int](../../sql-reference/data-types/int-uint.md)
@ -606,7 +606,7 @@ timestamp_sub(unit, value, date)
Aliases: `timeStampSub`, `TIMESTAMP_SUB`.
**Parameters**
**Arguments**
- `unit` — The type of interval to add. [String](../../sql-reference/data-types/string.md).
@ -640,7 +640,7 @@ Returns the current date and time.
now([timezone])
```
**Parameters**
**Arguments**
- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). [String](../../sql-reference/data-types/string.md).
@ -855,7 +855,7 @@ Converts a [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Prolepti
toModifiedJulianDay(date)
```
**Parameters**
**Arguments**
- `date` — Date in text form. [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md).
@ -891,7 +891,7 @@ Similar to [toModifiedJulianDay()](#tomodifiedjulianday), but instead of raising
toModifiedJulianDayOrNull(date)
```
**Parameters**
**Arguments**
- `date` — Date in text form. [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md).
@ -927,7 +927,7 @@ Converts a [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Varian
fromModifiedJulianDay(day)
```
**Parameters**
**Arguments**
- `day` — Modified Julian Day number. [Any integral types](../../sql-reference/data-types/int-uint.md).
@ -963,7 +963,7 @@ Similar to [fromModifiedJulianDayOrNull()](#frommodifiedjuliandayornull), but in
fromModifiedJulianDayOrNull(day)
```
**Parameters**
**Arguments**
- `day` — Modified Julian Day number. [Any integral types](../../sql-reference/data-types/int-uint.md).

View File

@ -15,7 +15,7 @@ Returns the string with the length as the number of passed arguments and each by
char(number_1, [number_2, ..., number_n]);
```
**Parameters**
**Arguments**
- `number_1, number_2, ..., number_n` — Numerical arguments interpreted as integers. Types: [Int](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md).
@ -107,7 +107,7 @@ For `String` and `FixedString`, all bytes are simply encoded as two hexadecimal
Values of floating point and Decimal types are encoded as their representation in memory. As we support little endian architecture, they are encoded in little endian. Zero leading/trailing bytes are not omitted.
**Parameters**
**Arguments**
- `arg` — A value to convert to hexadecimal. Types: [String](../../sql-reference/data-types/string.md), [UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md), [Decimal](../../sql-reference/data-types/decimal.md), [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md).

View File

@ -31,7 +31,7 @@ This function encrypts data using these modes:
encrypt('mode', 'plaintext', 'key' [, iv, aad])
```
**Parameters**
**Arguments**
- `mode` — Encryption mode. [String](../../sql-reference/data-types/string.md#string).
- `plaintext` — Text thats need to be encrypted. [String](../../sql-reference/data-types/string.md#string).
@ -127,7 +127,7 @@ Supported encryption modes:
aes_encrypt_mysql('mode', 'plaintext', 'key' [, iv])
```
**Parameters**
**Arguments**
- `mode` — Encryption mode. [String](../../sql-reference/data-types/string.md#string).
- `plaintext` — Text that needs to be encrypted. [String](../../sql-reference/data-types/string.md#string).
@ -238,7 +238,7 @@ This function decrypts ciphertext into a plaintext using these modes:
decrypt('mode', 'ciphertext', 'key' [, iv, aad])
```
**Parameters**
**Arguments**
- `mode` — Decryption mode. [String](../../sql-reference/data-types/string.md#string).
- `ciphertext` — Encrypted text that needs to be decrypted. [String](../../sql-reference/data-types/string.md#string).
@ -317,7 +317,7 @@ Supported decryption modes:
aes_decrypt_mysql('mode', 'ciphertext', 'key' [, iv])
```
**Parameters**
**Arguments**
- `mode` — Decryption mode. [String](../../sql-reference/data-types/string.md#string).
- `ciphertext` — Encrypted text that needs to be decrypted. [String](../../sql-reference/data-types/string.md#string).

View File

@ -19,7 +19,7 @@ dictGet('dict_name', 'attr_name', id_expr)
dictGetOrDefault('dict_name', 'attr_name', id_expr, default_value_expr)
```
**Parameters**
**Arguments**
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `attr_name` — Name of the column of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
@ -108,7 +108,7 @@ Checks whether a key is present in a dictionary.
dictHas('dict_name', id_expr)
```
**Parameters**
**Arguments**
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `id_expr` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md) or [Tuple](../../sql-reference/data-types/tuple.md)-type value depending on the dictionary configuration.
@ -130,7 +130,7 @@ Creates an array, containing all the parents of a key in the [hierarchical dicti
dictGetHierarchy('dict_name', key)
```
**Parameters**
**Arguments**
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `key` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md)-type value.
@ -149,7 +149,7 @@ Checks the ancestor of a key through the whole hierarchical chain in the diction
dictIsIn('dict_name', child_id_expr, ancestor_id_expr)
```
**Parameters**
**Arguments**
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `child_id_expr` — Key to be checked. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md)-type value.
@ -185,7 +185,7 @@ dictGet[Type]('dict_name', 'attr_name', id_expr)
dictGet[Type]OrDefault('dict_name', 'attr_name', id_expr, default_value_expr)
```
**Parameters**
**Arguments**
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `attr_name` — Name of the column of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).

View File

@ -13,7 +13,7 @@ Checks whether the argument is [NULL](../../sql-reference/syntax.md#null-literal
isNull(x)
```
**Parameters**
**Arguments**
- `x` — A value with a non-compound data type.
@ -53,7 +53,7 @@ Checks whether the argument is [NULL](../../sql-reference/syntax.md#null-literal
isNotNull(x)
```
**Parameters:**
**Arguments:**
- `x` — A value with a non-compound data type.
@ -93,7 +93,7 @@ Checks from left to right whether `NULL` arguments were passed and returns the f
coalesce(x,...)
```
**Parameters:**
**Arguments:**
- Any number of parameters of a non-compound type. All parameters must be compatible by data type.
@ -136,7 +136,7 @@ Returns an alternative value if the main argument is `NULL`.
ifNull(x,alt)
```
**Parameters:**
**Arguments:**
- `x` — The value to check for `NULL`.
- `alt` — The value that the function returns if `x` is `NULL`.
@ -176,7 +176,7 @@ Returns `NULL` if the arguments are equal.
nullIf(x, y)
```
**Parameters:**
**Arguments:**
`x`, `y` — Values for comparison. They must be compatible types, or ClickHouse will generate an exception.
@ -215,7 +215,7 @@ Results in a value of type [Nullable](../../sql-reference/data-types/nullable.md
assumeNotNull(x)
```
**Parameters:**
**Arguments:**
- `x` — The original value.
@ -277,7 +277,7 @@ Converts the argument type to `Nullable`.
toNullable(x)
```
**Parameters:**
**Arguments:**
- `x` — The value of any non-compound type.

View File

@ -72,7 +72,7 @@ Returns an array of [geohash](#geohash)-encoded strings of given precision that
geohashesInBox(longitude_min, latitude_min, longitude_max, latitude_max, precision)
```
**Parameters**
**Arguments**
- `longitude_min` — Minimum longitude. Range: `[-180°, 180°]`. Type: [Float](../../../sql-reference/data-types/float.md).
- `latitude_min` — Minimum latitude. Range: `[-90°, 90°]`. Type: [Float](../../../sql-reference/data-types/float.md).

View File

@ -162,7 +162,7 @@ Returns [H3](#h3index) point index `(lon, lat)` with specified resolution.
geoToH3(lon, lat, resolution)
```
**Parameters**
**Arguments**
- `lon` — Longitude. Type: [Float64](../../../sql-reference/data-types/float.md).
- `lat` — Latitude. Type: [Float64](../../../sql-reference/data-types/float.md).
@ -201,7 +201,7 @@ Result:
h3kRing(h3index, k)
```
**Parameters**
**Arguments**
- `h3index` — Hexagon index number. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
- `k` — Raduis. Type: [integer](../../../sql-reference/data-types/int-uint.md)
@ -315,7 +315,7 @@ Returns whether or not the provided [H3](#h3index) indexes are neighbors.
h3IndexesAreNeighbors(index1, index2)
```
**Parameters**
**Arguments**
- `index1` — Hexagon index number. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
- `index2` — Hexagon index number. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
@ -353,7 +353,7 @@ Returns an array of child indexes for the given [H3](#h3index) index.
h3ToChildren(index, resolution)
```
**Parameters**
**Arguments**
- `index` — Hexagon index number. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
- `resolution` — Index resolution. Range: `[0, 15]`. Type: [UInt8](../../../sql-reference/data-types/int-uint.md).
@ -390,7 +390,7 @@ Returns the parent (coarser) index containing the given [H3](#h3index) index.
h3ToParent(index, resolution)
```
**Parameters**
**Arguments**
- `index` — Hexagon index number. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
- `resolution` — Index resolution. Range: `[0, 15]`. Type: [UInt8](../../../sql-reference/data-types/int-uint.md).

View File

@ -18,9 +18,9 @@ halfMD5(par1, ...)
The function is relatively slow (5 million short strings per second per processor core).
Consider using the [sipHash64](#hash_functions-siphash64) function instead.
**Parameters**
**Arguments**
The function takes a variable number of input parameters. Parameters can be any of the [supported data types](../../sql-reference/data-types/index.md).
The function takes a variable number of input parameters. Arguments can be any of the [supported data types](../../sql-reference/data-types/index.md).
**Returned Value**
@ -61,9 +61,9 @@ Function [interprets](../../sql-reference/functions/type-conversion-functions.md
3. Then the function takes the hash value, calculated at the previous step, and the third element of the initial hash array, and calculates a hash for the array of them.
4. The previous step is repeated for all the remaining elements of the initial hash array.
**Parameters**
**Arguments**
The function takes a variable number of input parameters. Parameters can be any of the [supported data types](../../sql-reference/data-types/index.md).
The function takes a variable number of input parameters. Arguments can be any of the [supported data types](../../sql-reference/data-types/index.md).
**Returned Value**
@ -97,9 +97,9 @@ cityHash64(par1,...)
This is a fast non-cryptographic hash function. It uses the CityHash algorithm for string parameters and implementation-specific fast non-cryptographic hash function for parameters with other data types. The function uses the CityHash combinator to get the final results.
**Parameters**
**Arguments**
The function takes a variable number of input parameters. Parameters can be any of the [supported data types](../../sql-reference/data-types/index.md).
The function takes a variable number of input parameters. Arguments can be any of the [supported data types](../../sql-reference/data-types/index.md).
**Returned Value**
@ -166,9 +166,9 @@ farmHash64(par1, ...)
These functions use the `Fingerprint64` and `Hash64` methods respectively from all [available methods](https://github.com/google/farmhash/blob/master/src/farmhash.h).
**Parameters**
**Arguments**
The function takes a variable number of input parameters. Parameters can be any of the [supported data types](../../sql-reference/data-types/index.md).
The function takes a variable number of input parameters. Arguments can be any of the [supported data types](../../sql-reference/data-types/index.md).
**Returned Value**
@ -226,7 +226,7 @@ Calculates [JavaHash](http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add97
javaHashUTF16LE(stringUtf16le)
```
**Parameters**
**Arguments**
- `stringUtf16le` — a string in UTF-16LE encoding.
@ -292,9 +292,9 @@ Produces a 64-bit [MetroHash](http://www.jandrewrogers.com/2015/05/27/metrohash/
metroHash64(par1, ...)
```
**Parameters**
**Arguments**
The function takes a variable number of input parameters. Parameters can be any of the [supported data types](../../sql-reference/data-types/index.md).
The function takes a variable number of input parameters. Arguments can be any of the [supported data types](../../sql-reference/data-types/index.md).
**Returned Value**
@ -327,9 +327,9 @@ murmurHash2_32(par1, ...)
murmurHash2_64(par1, ...)
```
**Parameters**
**Arguments**
Both functions take a variable number of input parameters. Parameters can be any of the [supported data types](../../sql-reference/data-types/index.md).
Both functions take a variable number of input parameters. Arguments can be any of the [supported data types](../../sql-reference/data-types/index.md).
**Returned Value**
@ -358,7 +358,7 @@ Calculates a 64-bit [MurmurHash2](https://github.com/aappleby/smhasher) hash val
gccMurmurHash(par1, ...);
```
**Parameters**
**Arguments**
- `par1, ...` — A variable number of parameters that can be any of the [supported data types](../../sql-reference/data-types/index.md#data_types).
@ -395,9 +395,9 @@ murmurHash3_32(par1, ...)
murmurHash3_64(par1, ...)
```
**Parameters**
**Arguments**
Both functions take a variable number of input parameters. Parameters can be any of the [supported data types](../../sql-reference/data-types/index.md).
Both functions take a variable number of input parameters. Arguments can be any of the [supported data types](../../sql-reference/data-types/index.md).
**Returned Value**
@ -424,7 +424,7 @@ Produces a 128-bit [MurmurHash3](https://github.com/aappleby/smhasher) hash valu
murmurHash3_128( expr )
```
**Parameters**
**Arguments**
- `expr` — [Expressions](../../sql-reference/syntax.md#syntax-expressions) returning a [String](../../sql-reference/data-types/string.md)-type value.

View File

@ -32,7 +32,7 @@ If you use official ClickHouse packages, you need to install the `clickhouse-com
addressToLine(address_of_binary_instruction)
```
**Parameters**
**Arguments**
- `address_of_binary_instruction` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Address of instruction in a running process.
@ -123,7 +123,7 @@ Converts virtual memory address inside ClickHouse server process to the symbol f
addressToSymbol(address_of_binary_instruction)
```
**Parameters**
**Arguments**
- `address_of_binary_instruction` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Address of instruction in a running process.
@ -220,7 +220,7 @@ Converts a symbol that you can get using the [addressToSymbol](#addresstosymbol)
demangle(symbol)
```
**Parameters**
**Arguments**
- `symbol` ([String](../../sql-reference/data-types/string.md)) — Symbol from an object file.
@ -345,7 +345,7 @@ Emits trace log message to server log for each [Block](https://clickhouse.tech/d
logTrace('message')
```
**Parameters**
**Arguments**
- `message` — Message that is emitted to server log. [String](../../sql-reference/data-types/string.md#string).

View File

@ -265,32 +265,81 @@ SELECT toIPv6('127.0.0.1')
└─────────────────────┘
```
## isIPv4String
## isIPv4String {#isipv4string}
Determines if the input string is an IPv4 address or not. Returns `1` if true `0` otherwise.
Determines whether the input string is an IPv4 address or not. If `string` is IPv6 address returns `0`.
``` sql
SELECT isIPv4String('127.0.0.1')
**Syntax**
```sql
isIPv4String(string)
```
**Arguments**
- `string` — IP address. [String](../../sql-reference/data-types/string.md).
**Returned value**
- `1` if `string` is IPv4 address, `0` otherwise.
Type: [UInt8](../../sql-reference/data-types/int-uint.md).
**Examples**
Query:
```sql
SELECT addr, isIPv4String(addr) FROM ( SELECT ['0.0.0.0', '127.0.0.1', '::ffff:127.0.0.1'] AS addr ) ARRAY JOIN addr
```
Result:
``` text
┌─isIPv4String('127.0.0.1')─┐
│ 1 │
└───────────────────────────┘
┌─addr─────────────┬─isIPv4String(addr)─┐
│ 0.0.0.0 │ 1 │
│ 127.0.0.1 │ 1 │
│ ::ffff:127.0.0.1 │ 0 │
└──────────────────┴────────────────────┘
```
## isIPv6String
## isIPv6String {#isipv6string}
Determines if the input string is an IPv6 address or not. Returns `1` if true `0` otherwise.
Determines whether the input string is an IPv6 address or not. If `string` is IPv4 address returns `0`.
**Syntax**
```sql
isIPv6String(string)
```
**Arguments**
- `string` — IP address. [String](../../sql-reference/data-types/string.md).
**Returned value**
- `1` if `string` is IPv6 address, `0` otherwise.
Type: [UInt8](../../sql-reference/data-types/int-uint.md).
**Examples**
Query:
``` sql
SELECT isIPv6String('2001:438:ffff::407d:1bc1')
SELECT addr, isIPv6String(addr) FROM ( SELECT ['::', '1111::ffff', '::ffff:127.0.0.1', '127.0.0.1'] AS addr ) ARRAY JOIN addr
```
Result:
``` text
┌─isIPv6String('2001:438:ffff::407d:1bc1')─┐
│ 1 │
└──────────────────────────────────────────┘
┌─addr─────────────┬─isIPv6String(addr)─┐
│ :: │ 1 │
│ 1111::ffff │ 1 │
│ ::ffff:127.0.0.1 │ 1 │
│ 127.0.0.1 │ 0 │
└──────────────────┴────────────────────┘
```
[Original article](https://clickhouse.tech/docs/en/query_language/functions/ip_address_functions/) <!--hide-->

View File

@ -236,7 +236,7 @@ Extracts raw data from a JSON object.
JSONExtractKeysAndValuesRaw(json[, p, a, t, h])
```
**Parameters**
**Arguments**
- `json` — [String](../../sql-reference/data-types/string.md) with valid JSON.
- `p, a, t, h` — Comma-separated indices or keys that specify the path to the inner field in a nested JSON object. Each argument can be either a [string](../../sql-reference/data-types/string.md) to get the field by the key or an [integer](../../sql-reference/data-types/int-uint.md) to get the N-th field (indexed from 1, negative integers count from the end). If not set, the whole JSON is parsed as the top-level object. Optional parameter.

View File

@ -27,7 +27,7 @@ Compares test groups (variants) and calculates for each group the probability to
bayesAB(distribution_name, higher_is_better, variant_names, x, y)
```
**Parameters**
**Arguments**
- `distribution_name` — Name of the probability distribution. [String](../../sql-reference/data-types/string.md). Possible values:

View File

@ -121,7 +121,7 @@ Accepts a numeric argument and returns a UInt64 number close to 10 to the power
cosh(x)
```
**Parameters**
**Arguments**
- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64).
@ -157,7 +157,7 @@ Result:
acosh(x)
```
**Parameters**
**Arguments**
- `x` — Hyperbolic cosine of angle. Values from the interval: `1 <= x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64).
@ -197,7 +197,7 @@ Result:
sinh(x)
```
**Parameters**
**Arguments**
- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64).
@ -233,7 +233,7 @@ Result:
asinh(x)
```
**Parameters**
**Arguments**
- `x` — Hyperbolic sine of angle. Values from the interval: `-∞ < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64).
@ -273,7 +273,7 @@ Result:
atanh(x)
```
**Parameters**
**Arguments**
- `x` — Hyperbolic tangent of angle. Values from the interval: `1 < x < 1`. [Float64](../../sql-reference/data-types/float.md#float32-float64).
@ -309,7 +309,7 @@ The [function](https://en.wikipedia.org/wiki/Atan2) calculates the angle in the
atan2(y, x)
```
**Parameters**
**Arguments**
- `y` — y-coordinate of the point through which the ray passes. [Float64](../../sql-reference/data-types/float.md#float32-float64).
- `x` — x-coordinate of the point through which the ray passes. [Float64](../../sql-reference/data-types/float.md#float32-float64).
@ -346,7 +346,7 @@ Calculates the length of the hypotenuse of a right-angle triangle. The [function
hypot(x, y)
```
**Parameters**
**Arguments**
- `x` — The first cathetus of a right-angle triangle. [Float64](../../sql-reference/data-types/float.md#float32-float64).
- `y` — The second cathetus of a right-angle triangle. [Float64](../../sql-reference/data-types/float.md#float32-float64).
@ -383,7 +383,7 @@ Calculates `log(1+x)`. The [function](https://en.wikipedia.org/wiki/Natural_loga
log1p(x)
```
**Parameters**
**Arguments**
- `x` — Values from the interval: `-1 < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64).
@ -423,7 +423,7 @@ The `sign` function can extract the sign of a real number.
sign(x)
```
**Parameters**
**Arguments**
- `x` — Values from `-∞` to `+∞`. Support all numeric types in ClickHouse.

View File

@ -19,7 +19,7 @@ Gets a named value from the [macros](../../operations/server-configuration-param
getMacro(name);
```
**Parameters**
**Arguments**
- `name` — Name to retrieve from the `macros` section. [String](../../sql-reference/data-types/string.md#string).
@ -108,7 +108,7 @@ Extracts the trailing part of a string after the last slash or backslash. This f
basename( expr )
```
**Parameters**
**Arguments**
- `expr` — Expression resulting in a [String](../../sql-reference/data-types/string.md) type value. All the backslashes must be escaped in the resulting value.
@ -192,7 +192,7 @@ Returns estimation of uncompressed byte size of its arguments in memory.
byteSize(argument [, ...])
```
**Parameters**
**Arguments**
- `argument` — Value.
@ -349,7 +349,7 @@ The function is intended for development, debugging and demonstration.
isConstant(x)
```
**Parameters**
**Arguments**
- `x` — Expression to check.
@ -420,7 +420,7 @@ Checks whether floating point value is finite.
ifNotFinite(x,y)
**Parameters**
**Arguments**
- `x` — Value to be checked for infinity. Type: [Float\*](../../sql-reference/data-types/float.md).
- `y` — Fallback value. Type: [Float\*](../../sql-reference/data-types/float.md).
@ -460,7 +460,7 @@ Allows building a unicode-art diagram.
`bar(x, min, max, width)` draws a band with a width proportional to `(x - min)` and equal to `width` characters when `x = max`.
Parameters:
**Arguments**
- `x` — Size to display.
- `min, max` — Integer constants. The value must fit in `Int64`.
@ -645,7 +645,7 @@ Accepts the time delta in seconds. Returns a time delta with (year, month, day,
formatReadableTimeDelta(column[, maximum_unit])
```
**Parameters**
**Arguments**
- `column` — A column with numeric time delta.
- `maximum_unit` — Optional. Maximum unit to show. Acceptable values seconds, minutes, hours, days, months, years.
@ -730,7 +730,7 @@ The result of the function depends on the affected data blocks and the order of
The rows order used during the calculation of `neighbor` can differ from the order of rows returned to the user.
To prevent that you can make a subquery with ORDER BY and call the function from outside the subquery.
**Parameters**
**Arguments**
- `column` — A column name or scalar expression.
- `offset` — The number of rows forwards or backwards from the current row of `column`. [Int64](../../sql-reference/data-types/int-uint.md).
@ -909,6 +909,66 @@ WHERE diff != 1
Same as for [runningDifference](../../sql-reference/functions/other-functions.md#other_functions-runningdifference), the difference is the value of the first row, returned the value of the first row, and each subsequent row returns the difference from the previous row.
## runningConcurrency {#runningconcurrency}
Given a series of beginning time and ending time of events, this function calculates concurrency of the events at each of the data point, that is, the beginning time.
!!! warning "Warning"
Events spanning multiple data blocks will not be processed correctly. The function resets its state for each new data block.
The result of the function depends on the order of data in the block. It assumes the beginning time is sorted in ascending order.
**Syntax**
``` sql
runningConcurrency(begin, end)
```
**Arguments**
- `begin` — A column for the beginning time of events (inclusive). [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), or [DateTime64](../../sql-reference/data-types/datetime64.md).
- `end` — A column for the ending time of events (exclusive). [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), or [DateTime64](../../sql-reference/data-types/datetime64.md).
Note that two columns `begin` and `end` must have the same type.
**Returned values**
- The concurrency of events at the data point.
Type: [UInt32](../../sql-reference/data-types/int-uint.md)
**Example**
Input table:
``` text
┌───────────────begin─┬─────────────────end─┐
│ 2020-12-01 00:00:00 │ 2020-12-01 00:59:59 │
│ 2020-12-01 00:30:00 │ 2020-12-01 00:59:59 │
│ 2020-12-01 00:40:00 │ 2020-12-01 01:30:30 │
│ 2020-12-01 01:10:00 │ 2020-12-01 01:30:30 │
│ 2020-12-01 01:50:00 │ 2020-12-01 01:59:59 │
└─────────────────────┴─────────────────────┘
```
Query:
``` sql
SELECT runningConcurrency(begin, end) FROM example
```
Result:
``` text
┌─runningConcurrency(begin, end)─┐
│ 1 │
│ 2 │
│ 3 │
│ 2 │
│ 1 │
└────────────────────────────────┘
```
## MACNumToString(num) {#macnumtostringnum}
Accepts a UInt64 number. Interprets it as a MAC address in big endian. Returns a string containing the corresponding MAC address in the format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form).
@ -929,7 +989,7 @@ Returns the number of fields in [Enum](../../sql-reference/data-types/enum.md).
getSizeOfEnumType(value)
```
**Parameters:**
**Arguments:**
- `value` — Value of type `Enum`.
@ -958,7 +1018,7 @@ Returns size on disk (without taking into account compression).
blockSerializedSize(value[, value[, ...]])
```
**Parameters**
**Arguments**
- `value` — Any value.
@ -990,7 +1050,7 @@ Returns the name of the class that represents the data type of the column in RAM
toColumnTypeName(value)
```
**Parameters:**
**Arguments:**
- `value` — Any type of value.
@ -1030,7 +1090,7 @@ Outputs a detailed description of data structures in RAM
dumpColumnStructure(value)
```
**Parameters:**
**Arguments:**
- `value` — Any type of value.
@ -1060,7 +1120,7 @@ Does not include default values for custom columns set by the user.
defaultValueOfArgumentType(expression)
```
**Parameters:**
**Arguments:**
- `expression` — Arbitrary type of value or an expression that results in a value of an arbitrary type.
@ -1102,7 +1162,7 @@ Does not include default values for custom columns set by the user.
defaultValueOfTypeName(type)
```
**Parameters:**
**Arguments:**
- `type` — A string representing a type name.
@ -1144,7 +1204,7 @@ Used for internal implementation of [arrayJoin](../../sql-reference/functions/ar
SELECT replicate(x, arr);
```
**Parameters:**
**Arguments:**
- `arr` — Original array. ClickHouse creates a new array of the same length as the original and fills it with the value `x`.
- `x` — The value that the resulting array will be filled with.
@ -1277,7 +1337,7 @@ Takes state of aggregate function. Returns result of aggregation (or finalized s
finalizeAggregation(state)
```
**Parameters**
**Arguments**
- `state` — State of aggregation. [AggregateFunction](../../sql-reference/data-types/aggregatefunction.md#data-type-aggregatefunction).
@ -1381,7 +1441,7 @@ Accumulates states of an aggregate function for each row of a data block.
runningAccumulate(agg_state[, grouping]);
```
**Parameters**
**Arguments**
- `agg_state` — State of the aggregate function. [AggregateFunction](../../sql-reference/data-types/aggregatefunction.md#data-type-aggregatefunction).
- `grouping` — Grouping key. Optional. The state of the function is reset if the `grouping` value is changed. It can be any of the [supported data types](../../sql-reference/data-types/index.md) for which the equality operator is defined.
@ -1487,7 +1547,7 @@ Only supports tables created with the `ENGINE = Join(ANY, LEFT, <join_keys>)` st
joinGet(join_storage_table_name, `value_column`, join_keys)
```
**Parameters**
**Arguments**
- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicates where search is performed. The identifier is searched in the default database (see parameter `default_database` in the config file). To override the default database, use the `USE db_name` or specify the database and the table through the separator `db_name.db_table`, see the example.
- `value_column` — name of the column of the table that contains required data.
@ -1591,7 +1651,7 @@ Generates a string with a random set of [ASCII](https://en.wikipedia.org/wiki/AS
randomPrintableASCII(length)
```
**Parameters**
**Arguments**
- `length` — Resulting string length. Positive integer.
@ -1627,7 +1687,7 @@ Generates a binary string of the specified length filled with random bytes (incl
randomString(length)
```
**Parameters**
**Arguments**
- `length` — String length. Positive integer.
@ -1675,7 +1735,7 @@ Generates a binary string of the specified length filled with random bytes (incl
randomFixedString(length);
```
**Parameters**
**Arguments**
- `length` — String length in bytes. [UInt64](../../sql-reference/data-types/int-uint.md).
@ -1713,7 +1773,7 @@ Generates a random string of a specified length. Result string contains valid UT
randomStringUTF8(length);
```
**Parameters**
**Arguments**
- `length` — Required length of the resulting string in code points. [UInt64](../../sql-reference/data-types/int-uint.md).
@ -1785,7 +1845,7 @@ Checks whether the [Decimal](../../sql-reference/data-types/decimal.md) value is
isDecimalOverflow(d, [p])
```
**Parameters**
**Arguments**
- `d` — value. [Decimal](../../sql-reference/data-types/decimal.md).
- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. Using of this paratemer could be helpful for data extraction to another DBMS or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges).
@ -1822,7 +1882,7 @@ Returns number of decimal digits you need to represent the value.
countDigits(x)
```
**Parameters**
**Arguments**
- `x` — [Int](../../sql-reference/data-types/int-uint.md) or [Decimal](../../sql-reference/data-types/decimal.md) value.
@ -1881,7 +1941,7 @@ Returns [native interface](../../interfaces/tcp.md) TCP port number listened by
tcpPort()
```
**Parameters**
**Arguments**
- None.

View File

@ -32,7 +32,7 @@ Produces a constant column with a random value.
randConstant([x])
```
**Parameters**
**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.
@ -81,7 +81,7 @@ fuzzBits([s], [prob])
Inverts bits of `s`, each with probability `prob`.
**Parameters**
**Arguments**
- `s` - `String` or `FixedString`
- `prob` - constant `Float32/64`

View File

@ -35,7 +35,7 @@ The function returns the nearest number of the specified order. In case when giv
round(expression [, decimal_places])
```
**Parameters:**
**Arguments:**
- `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../../sql-reference/data-types/index.md#data_types).
- `decimal-places` — An integer value.
@ -114,7 +114,7 @@ For example, sum numbers 1.5, 2.5, 3.5, 4.5 with different rounding:
roundBankers(expression [, decimal_places])
```
**Parameters**
**Arguments**
- `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../../sql-reference/data-types/index.md#data_types).
- `decimal-places` — Decimal places. An integer number.

View File

@ -16,7 +16,7 @@ Returns an array of selected substrings. Empty substrings may be selected if the
splitByChar(<separator>, <s>)
```
**Parameters**
**Arguments**
- `separator` — The separator which should contain exactly one character. [String](../../sql-reference/data-types/string.md).
- `s` — The string to split. [String](../../sql-reference/data-types/string.md).
@ -53,7 +53,7 @@ Splits a string into substrings separated by a string. It uses a constant string
splitByString(<separator>, <s>)
```
**Parameters**
**Arguments**
- `separator` — The separator. [String](../../sql-reference/data-types/string.md).
- `s` — The string to split. [String](../../sql-reference/data-types/string.md).
@ -121,7 +121,7 @@ Extracts all groups from non-overlapping substrings matched by a regular express
extractAllGroups(text, regexp)
```
**Parameters**
**Arguments**
- `text` — [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md).
- `regexp` — Regular expression. Constant. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md).

View File

@ -76,7 +76,7 @@ Replaces invalid UTF-8 characters by the `<60>` (U+FFFD) character. All running i
toValidUTF8( input_string )
```
Parameters:
**Arguments**
- input_string — Any set of bytes represented as the [String](../../sql-reference/data-types/string.md) data type object.
@ -104,7 +104,7 @@ Repeats a string as many times as specified and concatenates the replicated valu
repeat(s, n)
```
**Parameters**
**Arguments**
- `s` — The string to repeat. [String](../../sql-reference/data-types/string.md).
- `n` — The number of times to repeat the string. [UInt](../../sql-reference/data-types/int-uint.md).
@ -173,7 +173,7 @@ Concatenates the strings listed in the arguments, without a separator.
concat(s1, s2, ...)
```
**Parameters**
**Arguments**
Values of type String or FixedString.
@ -211,7 +211,7 @@ The function is named “injective” if it always returns different result for
concatAssumeInjective(s1, s2, ...)
```
**Parameters**
**Arguments**
Values of type String or FixedString.
@ -328,7 +328,7 @@ By default removes all consecutive occurrences of common whitespace (ASCII chara
trim([[LEADING|TRAILING|BOTH] trim_character FROM] input_string)
```
**Parameters**
**Arguments**
- `trim_character` — specified characters for trim. [String](../../sql-reference/data-types/string.md).
- `input_string` — string for trim. [String](../../sql-reference/data-types/string.md).
@ -367,7 +367,7 @@ trimLeft(input_string)
Alias: `ltrim(input_string)`.
**Parameters**
**Arguments**
- `input_string` — string to trim. [String](../../sql-reference/data-types/string.md).
@ -405,7 +405,7 @@ trimRight(input_string)
Alias: `rtrim(input_string)`.
**Parameters**
**Arguments**
- `input_string` — string to trim. [String](../../sql-reference/data-types/string.md).
@ -443,7 +443,7 @@ trimBoth(input_string)
Alias: `trim(input_string)`.
**Parameters**
**Arguments**
- `input_string` — string to trim. [String](../../sql-reference/data-types/string.md).
@ -496,7 +496,7 @@ Replaces literals, sequences of literals and complex aliases with placeholders.
normalizeQuery(x)
```
**Parameters**
**Arguments**
- `x` — Sequence of characters. [String](../../sql-reference/data-types/string.md).
@ -532,7 +532,7 @@ Returns identical 64bit hash values without the values of literals for similar q
normalizedQueryHash(x)
```
**Parameters**
**Arguments**
- `x` — Sequence of characters. [String](../../sql-reference/data-types/string.md).
@ -570,7 +570,7 @@ The following five XML predefined entities will be replaced: `<`, `&`, `>`, `"`,
encodeXMLComponent(x)
```
**Parameters**
**Arguments**
- `x` — The sequence of characters. [String](../../sql-reference/data-types/string.md).

View File

@ -24,7 +24,7 @@ position(haystack, needle[, start_pos])
Alias: `locate(haystack, needle[, start_pos])`.
**Parameters**
**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).
@ -95,7 +95,7 @@ Works under the assumption that the string contains a set of bytes representing
positionCaseInsensitive(haystack, needle[, start_pos])
```
**Parameters**
**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).
@ -138,7 +138,7 @@ For a case-insensitive search, use the function [positionCaseInsensitiveUTF8](#p
positionUTF8(haystack, needle[, start_pos])
```
**Parameters**
**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).
@ -211,7 +211,7 @@ Works under the assumption that the string contains a set of bytes representing
positionCaseInsensitiveUTF8(haystack, needle[, start_pos])
```
**Parameters**
**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).
@ -256,7 +256,7 @@ The search is performed on sequences of bytes without respect to string encoding
multiSearchAllPositions(haystack, [needle1, needle2, ..., needlen])
```
**Parameters**
**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).
@ -371,7 +371,7 @@ Matches all groups of the `haystack` string using the `pattern` regular expressi
extractAllGroupsHorizontal(haystack, pattern)
```
**Parameters**
**Arguments**
- `haystack` — Input string. Type: [String](../../sql-reference/data-types/string.md).
- `pattern` — Regular expression with [re2 syntax](https://github.com/google/re2/wiki/Syntax). Must contain groups, each group enclosed in parentheses. If `pattern` contains no groups, an exception is thrown. Type: [String](../../sql-reference/data-types/string.md).
@ -412,7 +412,7 @@ Matches all groups of the `haystack` string using the `pattern` regular expressi
extractAllGroupsVertical(haystack, pattern)
```
**Parameters**
**Arguments**
- `haystack` — Input string. Type: [String](../../sql-reference/data-types/string.md).
- `pattern` — Regular expression with [re2 syntax](https://github.com/google/re2/wiki/Syntax). Must contain groups, each group enclosed in parentheses. If `pattern` contains no groups, an exception is thrown. Type: [String](../../sql-reference/data-types/string.md).
@ -471,7 +471,7 @@ Case insensitive variant of [like](https://clickhouse.tech/docs/en/sql-reference
ilike(haystack, pattern)
```
**Parameters**
**Arguments**
- `haystack` — Input string. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `pattern` — If `pattern` doesn't 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.
@ -548,7 +548,7 @@ For a case-insensitive search, use [countSubstringsCaseInsensitive](../../sql-re
countSubstrings(haystack, needle[, start_pos])
```
**Parameters**
**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).
@ -614,7 +614,7 @@ Returns the number of substring occurrences case-insensitive.
countSubstringsCaseInsensitive(haystack, needle[, start_pos])
```
**Parameters**
**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).
@ -680,7 +680,7 @@ Returns the number of substring occurrences in `UTF-8` case-insensitive.
SELECT countSubstringsCaseInsensitiveUTF8(haystack, needle[, start_pos])
```
**Parameters**
**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).
@ -732,7 +732,7 @@ Returns the number of regular expression matches for a `pattern` in a `haystack`
countMatches(haystack, pattern)
```
**Parameters**
**Arguments**
- `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `pattern` — The regular expression with [re2 syntax](https://github.com/google/re2/wiki/Syntax). [String](../../sql-reference/data-types/string.md).

View File

@ -45,7 +45,7 @@ untuple(x)
You can use the `EXCEPT` expression to skip columns as a result of the query.
**Parameters**
**Arguments**
- `x` - A `tuple` function, column, or tuple of elements. [Tuple](../../sql-reference/data-types/tuple.md).

View File

@ -5,6 +5,68 @@ toc_title: Working with maps
# Functions for maps {#functions-for-working-with-tuple-maps}
## map {#function-map}
Arranges `key:value` pairs into [Map(key, value)](../../sql-reference/data-types/map.md) data type.
**Syntax**
``` sql
map(key1, value1[, key2, value2, ...])
```
**Arguments**
- `key` — The key part of the pair. [String](../../sql-reference/data-types/string.md) or [Integer](../../sql-reference/data-types/int-uint.md).
- `value` — The value part of the pair. [String](../../sql-reference/data-types/string.md), [Integer](../../sql-reference/data-types/int-uint.md) or [Array](../../sql-reference/data-types/array.md).
**Returned value**
- Data structure as `key:value` pairs.
Type: [Map(key, value)](../../sql-reference/data-types/map.md).
**Examples**
Query:
``` sql
SELECT map('key1', number, 'key2', number * 2) FROM numbers(3);
```
Result:
``` text
┌─map('key1', number, 'key2', multiply(number, 2))─┐
│ {'key1':0,'key2':0} │
│ {'key1':1,'key2':2} │
│ {'key1':2,'key2':4} │
└──────────────────────────────────────────────────┘
```
Query:
``` sql
CREATE TABLE table_map (a Map(String, UInt64)) ENGINE = MergeTree() ORDER BY a;
INSERT INTO table_map SELECT map('key1', number, 'key2', number * 2) FROM numbers(3);
SELECT a['key2'] FROM table_map;
```
Result:
``` text
┌─arrayElement(a, 'key2')─┐
│ 0 │
│ 2 │
│ 4 │
└─────────────────────────┘
```
**See Also**
- [Map(key, value)](../../sql-reference/data-types/map.md) data type
## mapAdd {#function-mapadd}
Collect all the keys and sum corresponding values.
@ -15,7 +77,7 @@ Collect all the keys and sum corresponding values.
mapAdd(Tuple(Array, Array), Tuple(Array, Array) [, ...])
```
**Parameters**
**Arguments**
Arguments are [tuples](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array), where items in the first array represent keys, and the second array contains values for the each key. All key arrays should have same type, and all value arrays should contain items which are promote to the one type ([Int64](../../sql-reference/data-types/int-uint.md#int-ranges), [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges) or [Float64](../../sql-reference/data-types/float.md#float32-float64)). The common promoted type is used as a type for the result array.
@ -49,7 +111,7 @@ Collect all the keys and subtract corresponding values.
mapSubtract(Tuple(Array, Array), Tuple(Array, Array) [, ...])
```
**Parameters**
**Arguments**
Arguments are [tuples](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array), where items in the first array represent keys, and the second array contains values for the each key. All key arrays should have same type, and all value arrays should contain items which are promote to the one type ([Int64](../../sql-reference/data-types/int-uint.md#int-ranges), [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges) or [Float64](../../sql-reference/data-types/float.md#float32-float64)). The common promoted type is used as a type for the result array.
@ -87,7 +149,7 @@ Generates a map, where keys are a series of numbers, from minimum to maximum key
The number of elements in `keys` and `values` must be the same for each row.
**Parameters**
**Arguments**
- `keys` — Array of keys. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#uint-ranges)).
- `values` — Array of values. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#uint-ranges)).
@ -112,4 +174,4 @@ Result:
└──────────────────────────────┴───────────────────────────────────┘
```
[Original article](https://clickhouse.tech/docs/en/query_language/functions/tuple-map-functions/) <!--hide-->
[Original article](https://clickhouse.tech/docs/en/sql-reference/functions/tuple-map-functions/) <!--hide-->

View File

@ -22,7 +22,7 @@ Converts an input value to the [Int](../../sql-reference/data-types/int-uint.md)
- `toInt128(expr)` — Results in the `Int128` data type.
- `toInt256(expr)` — Results in the `Int256` data type.
**Parameters**
**Arguments**
- `expr` — [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a number or a string with the decimal representation of a number. Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped.
@ -88,7 +88,7 @@ Converts an input value to the [UInt](../../sql-reference/data-types/int-uint.md
- `toUInt64(expr)` — Results in the `UInt64` data type.
- `toUInt256(expr)` — Results in the `UInt256` data type.
**Parameters**
**Arguments**
- `expr` — [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a number or a string with the decimal representation of a number. Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped.
@ -154,7 +154,7 @@ Converts an input string to a [Nullable(Decimal(P,S))](../../sql-reference/data-
These functions should be used instead of `toDecimal*()` functions, if you prefer to get a `NULL` value instead of an exception in the event of an input value parsing error.
**Parameters**
**Arguments**
- `expr` — [Expression](../../sql-reference/syntax.md#syntax-expressions), returns a value in the [String](../../sql-reference/data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`.
- `S` — Scale, the number of decimal places in the resulting value.
@ -199,7 +199,7 @@ Converts an input value to the [Decimal(P,S)](../../sql-reference/data-types/dec
These functions should be used instead of `toDecimal*()` functions, if you prefer to get a `0` value instead of an exception in the event of an input value parsing error.
**Parameters**
**Arguments**
- `expr` — [Expression](../../sql-reference/syntax.md#syntax-expressions), returns a value in the [String](../../sql-reference/data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`.
- `S` — Scale, the number of decimal places in the resulting value.
@ -467,7 +467,7 @@ toIntervalQuarter(number)
toIntervalYear(number)
```
**Parameters**
**Arguments**
- `number` — Duration of interval. Positive integer number.
@ -505,7 +505,7 @@ The function parses [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601), [RFC 112
parseDateTimeBestEffort(time_string [, time_zone]);
```
**Parameters**
**Arguments**
- `time_string` — String containing a date and time to convert. [String](../../sql-reference/data-types/string.md).
- `time_zone` — Time zone. The function parses `time_string` according to the time zone. [String](../../sql-reference/data-types/string.md).
@ -617,7 +617,7 @@ This function is similar to [parseDateTimeBestEffort](#parsedatetimebestef
parseDateTimeBestEffortUS(time_string [, time_zone]);
```
**Parameters**
**Arguments**
- `time_string` — String containing a date and time to convert. [String](../../sql-reference/data-types/string.md).
- `time_zone` — Time zone. The function parses `time_string` according to the time zone. [String](../../sql-reference/data-types/string.md).
@ -701,7 +701,7 @@ To convert data from the `LowCardinality` data type use the [CAST](#type_convers
toLowCardinality(expr)
```
**Parameters**
**Arguments**
- `expr` — [Expression](../../sql-reference/syntax.md#syntax-expressions) resulting in one of the [supported data types](../../sql-reference/data-types/index.md#data_types).
@ -741,7 +741,7 @@ Converts a `DateTime64` to a `Int64` value with fixed sub-second precision. Inpu
toUnixTimestamp64Milli(value)
```
**Parameters**
**Arguments**
- `value` — DateTime64 value with any precision.
@ -793,7 +793,7 @@ Converts an `Int64` to a `DateTime64` value with fixed sub-second precision and
fromUnixTimestamp64Milli(value [, ti])
```
**Parameters**
**Arguments**
- `value``Int64` value with any precision.
- `timezone``String` (optional) timezone name of the result.
@ -825,7 +825,7 @@ Converts arbitrary expressions into a string via given format.
formatRow(format, x, y, ...)
```
**Parameters**
**Arguments**
- `format` — Text format. For example, [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated).
- `x`,`y`, ... — Expressions.
@ -866,7 +866,7 @@ Converts arbitrary expressions into a string via given format. The function trim
formatRowNoNewline(format, x, y, ...)
```
**Parameters**
**Arguments**
- `format` — Text format. For example, [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated).
- `x`,`y`, ... — Expressions.

View File

@ -25,7 +25,7 @@ Extracts the hostname from a URL.
domain(url)
```
**Parameters**
**Arguments**
- `url` — URL. Type: [String](../../sql-reference/data-types/string.md).
@ -76,7 +76,7 @@ Extracts the the top-level domain from a URL.
topLevelDomain(url)
```
**Parameters**
**Arguments**
- `url` — URL. Type: [String](../../sql-reference/data-types/string.md).
@ -133,10 +133,9 @@ For example:
### cutToFirstSignificantSubdomainCustom {#cuttofirstsignificantsubdomaincustom}
Same as `cutToFirstSignificantSubdomain` but accept custom TLD list name, useful if:
Returns the part of the domain that includes top-level subdomains up to the first significant subdomain. Accepts custom [TLD list](https://en.wikipedia.org/wiki/List_of_Internet_top-level_domains) name.
- you need fresh TLD list,
- or you have custom.
Can be useful if you need fresh TLD list or you have custom.
Configuration example:
@ -149,21 +148,150 @@ Configuration example:
</top_level_domains_lists>
```
Example:
**Syntax**
- `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/', 'public_suffix_list') = 'yandex.com.tr'`.
``` sql
cutToFirstSignificantSubdomain(URL, TLD)
```
**Parameters**
- `URL` — URL. [String](../../sql-reference/data-types/string.md).
- `TLD` — Custom TLD list name. [String](../../sql-reference/data-types/string.md).
**Returned value**
- Part of the domain that includes top-level subdomains up to the first significant subdomain.
Type: [String](../../sql-reference/data-types/string.md).
**Example**
Query:
```sql
SELECT cutToFirstSignificantSubdomainCustom('bar.foo.there-is-no-such-domain', 'public_suffix_list');
```
Result:
```text
┌─cutToFirstSignificantSubdomainCustom('bar.foo.there-is-no-such-domain', 'public_suffix_list')─┐
│ foo.there-is-no-such-domain │
└───────────────────────────────────────────────────────────────────────────────────────────────┘
```
**See Also**
- [firstSignificantSubdomain](#firstsignificantsubdomain).
### cutToFirstSignificantSubdomainCustomWithWWW {#cuttofirstsignificantsubdomaincustomwithwww}
Same as `cutToFirstSignificantSubdomainWithWWW` but accept custom TLD list name.
Returns the part of the domain that includes top-level subdomains up to the first significant subdomain without stripping `www`. Accepts custom TLD list name.
Can be useful if you need fresh TLD list or you have custom.
Configuration example:
```xml
<!-- <top_level_domains_path>/var/lib/clickhouse/top_level_domains/</top_level_domains_path> -->
<top_level_domains_lists>
<!-- https://publicsuffix.org/list/public_suffix_list.dat -->
<public_suffix_list>public_suffix_list.dat</public_suffix_list>
<!-- NOTE: path is under top_level_domains_path -->
</top_level_domains_lists>
```
**Syntax**
```sql
cutToFirstSignificantSubdomainCustomWithWWW(URL, TLD)
```
**Parameters**
- `URL` — URL. [String](../../sql-reference/data-types/string.md).
- `TLD` — Custom TLD list name. [String](../../sql-reference/data-types/string.md).
**Returned value**
- Part of the domain that includes top-level subdomains up to the first significant subdomain without stripping `www`.
Type: [String](../../sql-reference/data-types/string.md).
**Example**
Query:
```sql
SELECT cutToFirstSignificantSubdomainCustomWithWWW('www.foo', 'public_suffix_list');
```
Result:
```text
┌─cutToFirstSignificantSubdomainCustomWithWWW('www.foo', 'public_suffix_list')─┐
│ www.foo │
└──────────────────────────────────────────────────────────────────────────────┘
```
**See Also**
- [firstSignificantSubdomain](#firstsignificantsubdomain).
### firstSignificantSubdomainCustom {#firstsignificantsubdomaincustom}
Same as `firstSignificantSubdomain` but accept custom TLD list name.
Returns the first significant subdomain. Accepts customs TLD list name.
### cutToFirstSignificantSubdomainCustomWithWWW {#cuttofirstsignificantsubdomaincustomwithwww}
Can be useful if you need fresh TLD list or you have custom.
Same as `cutToFirstSignificantSubdomainWithWWW` but accept custom TLD list name.
Configuration example:
```xml
<!-- <top_level_domains_path>/var/lib/clickhouse/top_level_domains/</top_level_domains_path> -->
<top_level_domains_lists>
<!-- https://publicsuffix.org/list/public_suffix_list.dat -->
<public_suffix_list>public_suffix_list.dat</public_suffix_list>
<!-- NOTE: path is under top_level_domains_path -->
</top_level_domains_lists>
```
**Syntax**
```sql
firstSignificantSubdomainCustom(URL, TLD)
```
**Parameters**
- `URL` — URL. [String](../../sql-reference/data-types/string.md).
- `TLD` — Custom TLD list name. [String](../../sql-reference/data-types/string.md).
**Returned value**
- First significant subdomain.
Type: [String](../../sql-reference/data-types/string.md).
**Example**
Query:
```sql
SELECT firstSignificantSubdomainCustom('bar.foo.there-is-no-such-domain', 'public_suffix_list');
```
Result:
```text
┌─firstSignificantSubdomainCustom('bar.foo.there-is-no-such-domain', 'public_suffix_list')─┐
│ foo │
└──────────────────────────────────────────────────────────────────────────────────────────┘
```
**See Also**
- [firstSignificantSubdomain](#firstsignificantsubdomain).
### port(URL\[, default_port = 0\]) {#port}
@ -242,7 +370,7 @@ Extracts network locality (`username:password@host:port`) from a URL.
netloc(URL)
```
**Parameters**
**Arguments**
- `url` — URL. [String](../../sql-reference/data-types/string.md).

View File

@ -115,7 +115,7 @@ Finds the highest continent in the hierarchy for the region.
regionToTopContinent(id[, geobase]);
```
**Parameters**
**Arguments**
- `id` — Region ID from the Yandex geobase. [UInt32](../../sql-reference/data-types/int-uint.md).
- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional.

View File

@ -20,6 +20,7 @@ The following actions are supported:
- [ADD COLUMN](#alter_add-column) — Adds a new column to the table.
- [DROP COLUMN](#alter_drop-column) — Deletes the column.
- [RENAME COLUMN](#alter_rename-column) — Renames the column.
- [CLEAR COLUMN](#alter_clear-column) — Resets column values.
- [COMMENT COLUMN](#alter_comment-column) — Adds a text comment to the column.
- [MODIFY COLUMN](#alter_modify-column) — Changes columns type, default expression and TTL.
@ -78,6 +79,22 @@ Example:
ALTER TABLE visits DROP COLUMN browser
```
## RENAME COLUMN {#alter_rename-column}
``` sql
RENAME COLUMN [IF EXISTS] name to new_name
```
Renames the column `name` to `new_name`. If the `IF EXISTS` clause is specified, the query wont return an error if the column doesnt exist. Since renaming does not involve the underlying data, the query is completed almost instantly.
**NOTE**: Columns specified in the key expression of the table (either with `ORDER BY` or `PRIMARY KEY`) cannot be renamed. Trying to change these columns will produce `SQL Error [524]`.
Example:
``` sql
ALTER TABLE visits RENAME COLUMN webBrowser TO browser
```
## CLEAR COLUMN {#alter_clear-column}
``` sql

View File

@ -13,7 +13,7 @@ Supports all data types that can be stored in table except `LowCardinality` and
generateRandom('name TypeName[, name TypeName]...', [, 'random_seed'[, 'max_string_length'[, 'max_array_length']]]);
```
**Parameters**
**Arguments**
- `name` — Name of corresponding column.
- `TypeName` — Type of corresponding column.

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