mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge branch 'master' into fix_force_drop_materialized_view
This commit is contained in:
commit
103d569885
6
.gitignore
vendored
6
.gitignore
vendored
@ -137,3 +137,9 @@ website/package-lock.json
|
||||
/prof
|
||||
|
||||
*.iml
|
||||
|
||||
# data store
|
||||
/programs/server/data
|
||||
/programs/server/metadata
|
||||
/programs/server/store
|
||||
|
||||
|
@ -416,7 +416,9 @@ static void sanitizerDeathCallback()
|
||||
else
|
||||
log_message = "Terminate called without an active exception";
|
||||
|
||||
static const size_t buf_size = 1024;
|
||||
/// POSIX.1 says that write(2)s of less than PIPE_BUF bytes must be atomic - man 7 pipe
|
||||
/// And the buffer should not be too small because our exception messages can be large.
|
||||
static constexpr size_t buf_size = PIPE_BUF;
|
||||
|
||||
if (log_message.size() > buf_size - 16)
|
||||
log_message.resize(buf_size - 16);
|
||||
|
@ -83,7 +83,7 @@ public:
|
||||
template <class T>
|
||||
void writeToGraphite(const std::string & key, const T & value, const std::string & config_name = DEFAULT_GRAPHITE_CONFIG_NAME, time_t timestamp = 0, const std::string & custom_root_path = "")
|
||||
{
|
||||
auto writer = getGraphiteWriter(config_name);
|
||||
auto *writer = getGraphiteWriter(config_name);
|
||||
if (writer)
|
||||
writer->write(key, value, timestamp, custom_root_path);
|
||||
}
|
||||
@ -91,7 +91,7 @@ public:
|
||||
template <class T>
|
||||
void writeToGraphite(const GraphiteWriter::KeyValueVector<T> & key_vals, const std::string & config_name = DEFAULT_GRAPHITE_CONFIG_NAME, time_t timestamp = 0, const std::string & custom_root_path = "")
|
||||
{
|
||||
auto writer = getGraphiteWriter(config_name);
|
||||
auto *writer = getGraphiteWriter(config_name);
|
||||
if (writer)
|
||||
writer->write(key_vals, timestamp, custom_root_path);
|
||||
}
|
||||
@ -99,7 +99,7 @@ public:
|
||||
template <class T>
|
||||
void writeToGraphite(const GraphiteWriter::KeyValueVector<T> & key_vals, const std::chrono::system_clock::time_point & current_time, const std::string & custom_root_path)
|
||||
{
|
||||
auto writer = getGraphiteWriter();
|
||||
auto *writer = getGraphiteWriter();
|
||||
if (writer)
|
||||
writer->write(key_vals, std::chrono::system_clock::to_time_t(current_time), custom_root_path);
|
||||
}
|
||||
|
@ -37,15 +37,13 @@ if (CCACHE_FOUND AND NOT COMPILER_MATCHES_CCACHE)
|
||||
#
|
||||
# - 4.0+ ccache always includes this environment variable into the hash
|
||||
# of the manifest, which do not allow to use previous cache,
|
||||
# - 4.2+ ccache ignores SOURCE_DATE_EPOCH under time_macros sloppiness.
|
||||
# - 4.2+ ccache ignores SOURCE_DATE_EPOCH for every file w/o __DATE__/__TIME__
|
||||
#
|
||||
# So for:
|
||||
# - 4.2+ time_macros sloppiness is used,
|
||||
# - 4.2+ does not require any sloppiness
|
||||
# - 4.0+ will ignore SOURCE_DATE_EPOCH environment variable.
|
||||
if (CCACHE_VERSION VERSION_GREATER_EQUAL "4.2")
|
||||
message(STATUS "Use time_macros sloppiness for ccache")
|
||||
set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE "${CCACHE_FOUND} --set-config=sloppiness=time_macros")
|
||||
set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK "${CCACHE_FOUND} --set-config=sloppiness=time_macros")
|
||||
message(STATUS "ccache is 4.2+ no quirks for SOURCE_DATE_EPOCH required")
|
||||
elseif (CCACHE_VERSION VERSION_GREATER_EQUAL "4.0")
|
||||
message(STATUS "Ignore SOURCE_DATE_EPOCH for ccache")
|
||||
set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE "env -u SOURCE_DATE_EPOCH ${CCACHE_FOUND}")
|
||||
|
@ -47,6 +47,7 @@ RUN apt-get update \
|
||||
expect \
|
||||
fakeroot \
|
||||
git \
|
||||
gdb \
|
||||
gperf \
|
||||
lld-${LLVM_VERSION} \
|
||||
llvm-${LLVM_VERSION} \
|
||||
|
@ -107,6 +107,18 @@ function start_server
|
||||
fi
|
||||
|
||||
echo "ClickHouse server pid '$server_pid' started and responded"
|
||||
|
||||
echo "
|
||||
handle all noprint
|
||||
handle SIGSEGV stop print
|
||||
handle SIGBUS stop print
|
||||
handle SIGABRT stop print
|
||||
continue
|
||||
thread apply all backtrace
|
||||
continue
|
||||
" > script.gdb
|
||||
|
||||
gdb -batch -command script.gdb -p "$server_pid" &
|
||||
}
|
||||
|
||||
function clone_root
|
||||
@ -259,6 +271,7 @@ function run_tests
|
||||
00929_multi_match_edit_distance
|
||||
01681_hyperscan_debug_assertion
|
||||
|
||||
01176_mysql_client_interactive # requires mysql client
|
||||
01031_mutations_interpreter_and_context
|
||||
01053_ssd_dictionary # this test mistakenly requires acces to /var/lib/clickhouse -- can't run this locally, disabled
|
||||
01083_expressions_in_engine_arguments
|
||||
@ -326,7 +339,7 @@ function run_tests
|
||||
# Look at DistributedFilesToInsert, so cannot run in parallel.
|
||||
01460_DistributedFilesToInsert
|
||||
|
||||
01541_max_memory_usage_for_user
|
||||
01541_max_memory_usage_for_user_long
|
||||
|
||||
# Require python libraries like scipy, pandas and numpy
|
||||
01322_ttest_scipy
|
||||
@ -342,9 +355,10 @@ function run_tests
|
||||
|
||||
# JSON functions
|
||||
01666_blns
|
||||
01674_htm_xml_coarse_parse
|
||||
)
|
||||
|
||||
time clickhouse-test --hung-check -j 8 --order=random --use-skip-list --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" -- "$FASTTEST_FOCUS" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt"
|
||||
(time clickhouse-test --hung-check -j 8 --order=random --use-skip-list --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" -- "$FASTTEST_FOCUS" 2>&1 ||:) | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt"
|
||||
|
||||
# substr is to remove semicolon after test name
|
||||
readarray -t FAILED_TESTS < <(awk '/\[ FAIL|TIMEOUT|ERROR \]/ { print substr($3, 1, length($3)-1) }' "$FASTTEST_OUTPUT/test_log.txt" | tee "$FASTTEST_OUTPUT/failed-parallel-tests.txt")
|
||||
|
@ -58,7 +58,7 @@ RUN dockerd --version; docker --version
|
||||
|
||||
RUN python3 -m pip install \
|
||||
PyMySQL \
|
||||
aerospike \
|
||||
aerospike==4.0.0 \
|
||||
avro \
|
||||
cassandra-driver \
|
||||
confluent-kafka==1.5.0 \
|
||||
|
@ -60,4 +60,8 @@ fi
|
||||
# more idiologically correct.
|
||||
read -ra ADDITIONAL_OPTIONS <<< "${ADDITIONAL_OPTIONS:-}"
|
||||
|
||||
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
|
||||
ADDITIONAL_OPTIONS+=('--replicated-database')
|
||||
fi
|
||||
|
||||
clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
|
||||
|
@ -3,6 +3,9 @@ FROM yandex/clickhouse-test-base
|
||||
|
||||
ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/download/v1.1.4.20200302/clickhouse-odbc-1.1.4-Linux.tar.gz"
|
||||
|
||||
RUN echo "deb [trusted=yes] http://repo.mysql.com/apt/ubuntu/ bionic mysql-5.7" >> /etc/apt/sources.list \
|
||||
&& apt-key adv --keyserver keyserver.ubuntu.com --recv-keys 8C718D3B5072E1F5
|
||||
|
||||
RUN apt-get update -y \
|
||||
&& env DEBIAN_FRONTEND=noninteractive \
|
||||
apt-get install --yes --no-install-recommends \
|
||||
@ -13,6 +16,7 @@ RUN apt-get update -y \
|
||||
ncdu \
|
||||
netcat-openbsd \
|
||||
openssl \
|
||||
protobuf-compiler \
|
||||
python3 \
|
||||
python3-lxml \
|
||||
python3-requests \
|
||||
@ -23,7 +27,8 @@ RUN apt-get update -y \
|
||||
telnet \
|
||||
tree \
|
||||
unixodbc \
|
||||
wget
|
||||
wget \
|
||||
mysql-client=5.7*
|
||||
|
||||
RUN pip3 install numpy scipy pandas
|
||||
|
||||
|
@ -57,6 +57,10 @@ function run_tests()
|
||||
ADDITIONAL_OPTIONS+=('4')
|
||||
fi
|
||||
|
||||
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
|
||||
ADDITIONAL_OPTIONS+=('--replicated-database')
|
||||
fi
|
||||
|
||||
clickhouse-test --testname --shard --zookeeper --hung-check --print-time \
|
||||
--test-runs "$NUM_TRIES" \
|
||||
"$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
|
||||
|
@ -8,16 +8,23 @@ dpkg -i package_folder/clickhouse-server_*.deb
|
||||
dpkg -i package_folder/clickhouse-client_*.deb
|
||||
dpkg -i package_folder/clickhouse-test_*.deb
|
||||
|
||||
function configure()
|
||||
{
|
||||
# install test configs
|
||||
/usr/share/clickhouse-test/config/install.sh
|
||||
|
||||
# for clickhouse-server (via service)
|
||||
echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment
|
||||
# for clickhouse-client
|
||||
export ASAN_OPTIONS='malloc_context_size=10 allocator_release_to_os_interval_ms=10000'
|
||||
|
||||
# since we run clickhouse from root
|
||||
sudo chown root: /var/lib/clickhouse
|
||||
}
|
||||
|
||||
function stop()
|
||||
{
|
||||
timeout 120 service clickhouse-server stop
|
||||
|
||||
# Wait for process to disappear from processlist and also try to kill zombies.
|
||||
while kill -9 "$(pidof clickhouse-server)"
|
||||
do
|
||||
echo "Killed clickhouse-server"
|
||||
sleep 0.5
|
||||
done
|
||||
clickhouse stop
|
||||
}
|
||||
|
||||
function start()
|
||||
@ -33,19 +40,26 @@ function start()
|
||||
tail -n1000 /var/log/clickhouse-server/clickhouse-server.log
|
||||
break
|
||||
fi
|
||||
timeout 120 service clickhouse-server start
|
||||
# use root to match with current uid
|
||||
clickhouse start --user root >/var/log/clickhouse-server/stdout.log 2>/var/log/clickhouse-server/stderr.log
|
||||
sleep 0.5
|
||||
counter=$((counter + 1))
|
||||
done
|
||||
|
||||
echo "
|
||||
handle all noprint
|
||||
handle SIGSEGV stop print
|
||||
handle SIGBUS stop print
|
||||
handle SIGABRT stop print
|
||||
continue
|
||||
thread apply all backtrace
|
||||
continue
|
||||
" > script.gdb
|
||||
|
||||
gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" &
|
||||
}
|
||||
|
||||
# install test configs
|
||||
/usr/share/clickhouse-test/config/install.sh
|
||||
|
||||
# for clickhouse-server (via service)
|
||||
echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment
|
||||
# for clickhouse-client
|
||||
export ASAN_OPTIONS='malloc_context_size=10 allocator_release_to_os_interval_ms=10000'
|
||||
configure
|
||||
|
||||
start
|
||||
|
||||
@ -64,7 +78,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
|
||||
|
@ -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
|
||||
@ -22,12 +23,15 @@ def get_options(i):
|
||||
if 0 < i:
|
||||
options += " --order=random"
|
||||
|
||||
if i % 2 == 1:
|
||||
if i % 3 == 1:
|
||||
options += " --db-engine=Ordinary"
|
||||
|
||||
if i % 3 == 2:
|
||||
options += ''' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i)
|
||||
|
||||
# If database name is not specified, new database is created for each functional test.
|
||||
# Run some threads with one database for all tests.
|
||||
if i % 3 == 1:
|
||||
if i % 2 == 1:
|
||||
options += " --database=test_{}".format(i)
|
||||
|
||||
if i == 13:
|
||||
@ -64,7 +68,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 +86,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")
|
||||
|
@ -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.
|
||||
- Returned values list.
|
||||
|
||||
Type: [Type](relative/path/to/type/dscr.md#type).
|
||||
Type: [Type name](relative/path/to/type/dscr.md#type).
|
||||
|
||||
**Example**
|
||||
|
||||
|
@ -8,10 +8,14 @@ Columns:
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT * FROM system.table_name
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
Some output. It shouldn't be too long.
|
||||
```
|
||||
|
@ -59,10 +59,26 @@ Optional parameters:
|
||||
- `rabbitmq_max_block_size`
|
||||
- `rabbitmq_flush_interval_ms`
|
||||
|
||||
Required configuration:
|
||||
Also format settings can be added along with rabbitmq-related settings.
|
||||
|
||||
Example:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE queue (
|
||||
key UInt64,
|
||||
value UInt64,
|
||||
date DateTime
|
||||
) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'localhost:5672',
|
||||
rabbitmq_exchange_name = 'exchange1',
|
||||
rabbitmq_format = 'JSONEachRow',
|
||||
rabbitmq_num_consumers = 5,
|
||||
date_time_input_format = 'best_effort';
|
||||
```
|
||||
|
||||
The RabbitMQ server configuration should be added using the ClickHouse config file.
|
||||
|
||||
Required configuration:
|
||||
|
||||
``` xml
|
||||
<rabbitmq>
|
||||
<username>root</username>
|
||||
@ -70,16 +86,12 @@ The RabbitMQ server configuration should be added using the ClickHouse config fi
|
||||
</rabbitmq>
|
||||
```
|
||||
|
||||
Example:
|
||||
Additional configuration:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE queue (
|
||||
key UInt64,
|
||||
value UInt64
|
||||
) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'localhost:5672',
|
||||
rabbitmq_exchange_name = 'exchange1',
|
||||
rabbitmq_format = 'JSONEachRow',
|
||||
rabbitmq_num_consumers = 5;
|
||||
``` xml
|
||||
<rabbitmq>
|
||||
<vhost>clickhouse</vhost>
|
||||
</rabbitmq>
|
||||
```
|
||||
|
||||
## Description {#description}
|
||||
@ -105,6 +117,7 @@ Exchange type options:
|
||||
- `consistent_hash` - Data is evenly distributed between all bound tables (where the exchange name is the same). Note that this exchange type must be enabled with RabbitMQ plugin: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`.
|
||||
|
||||
Setting `rabbitmq_queue_base` may be used for the following cases:
|
||||
|
||||
- to let different tables share queues, so that multiple consumers could be registered for the same queues, which makes a better performance. If using `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings, the exact match of queues is achieved in case these parameters are the same.
|
||||
- to be able to restore reading from certain durable queues when not all messages were successfully consumed. To resume consumption from one specific queue - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. By default, queue names will be unique to tables.
|
||||
- to reuse queues as they are declared durable and not auto-deleted. (Can be deleted via any of RabbitMQ CLI tools.)
|
||||
|
@ -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}'
|
||||
)
|
||||
|
13
docs/en/operations/external-authenticators/index.md
Normal file
13
docs/en/operations/external-authenticators/index.md
Normal file
@ -0,0 +1,13 @@
|
||||
---
|
||||
toc_folder_title: External User Authenticators and Directories
|
||||
toc_priority: 48
|
||||
toc_title: Introduction
|
||||
---
|
||||
|
||||
# External User Authenticators and Directories {#external-authenticators}
|
||||
|
||||
ClickHouse supports authenticating and managing users using external services.
|
||||
|
||||
The following external authenticators and directories are supported:
|
||||
|
||||
- [LDAP](./ldap.md#external-authenticators-ldap) [Authenticator](./ldap.md#ldap-external-authenticator) and [Directory](./ldap.md#ldap-external-user-directory)
|
156
docs/en/operations/external-authenticators/ldap.md
Normal file
156
docs/en/operations/external-authenticators/ldap.md
Normal file
@ -0,0 +1,156 @@
|
||||
# LDAP {#external-authenticators-ldap}
|
||||
|
||||
LDAP server can be used to authenticate ClickHouse users. There are two different approaches for doing this:
|
||||
|
||||
- use LDAP as an external authenticator for existing users, which are defined in `users.xml` or in local access control paths
|
||||
- use LDAP as an external user directory and allow locally undefined users to be authenticated if they exist on the LDAP server
|
||||
|
||||
For both of these approaches, an internally named LDAP server must be defined in the ClickHouse config so that other parts of config are able to refer to it.
|
||||
|
||||
## LDAP Server Definition {#ldap-server-definition}
|
||||
|
||||
To define LDAP server you must add `ldap_servers` section to the `config.xml`. For example,
|
||||
|
||||
```xml
|
||||
<yandex>
|
||||
<!- ... -->
|
||||
<ldap_servers>
|
||||
<my_ldap_server>
|
||||
<host>localhost</host>
|
||||
<port>636</port>
|
||||
<bind_dn>uid={user_name},ou=users,dc=example,dc=com</bind_dn>
|
||||
<verification_cooldown>300</verification_cooldown>
|
||||
<enable_tls>yes</enable_tls>
|
||||
<tls_minimum_protocol_version>tls1.2</tls_minimum_protocol_version>
|
||||
<tls_require_cert>demand</tls_require_cert>
|
||||
<tls_cert_file>/path/to/tls_cert_file</tls_cert_file>
|
||||
<tls_key_file>/path/to/tls_key_file</tls_key_file>
|
||||
<tls_ca_cert_file>/path/to/tls_ca_cert_file</tls_ca_cert_file>
|
||||
<tls_ca_cert_dir>/path/to/tls_ca_cert_dir</tls_ca_cert_dir>
|
||||
<tls_cipher_suite>ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-RSA-AES256-GCM-SHA384:AES256-GCM-SHA384</tls_cipher_suite>
|
||||
</my_ldap_server>
|
||||
</ldap_servers>
|
||||
</yandex>
|
||||
```
|
||||
|
||||
Note, that you can define multiple LDAP servers inside the `ldap_servers` section using distinct names.
|
||||
|
||||
Parameters:
|
||||
|
||||
- `host` - LDAP server hostname or IP, this parameter is mandatory and cannot be empty.
|
||||
- `port` - LDAP server port, default is `636` if `enable_tls` is set to `true`, `389` otherwise.
|
||||
- `bind_dn` - template used to construct the DN to bind to.
|
||||
- The resulting DN will be constructed by replacing all `{user_name}` substrings of the
|
||||
template with the actual user name during each authentication attempt.
|
||||
- `verification_cooldown` - a period of time, in seconds, after a successful bind attempt,
|
||||
during which the user will be assumed to be successfully authenticated for all consecutive
|
||||
requests without contacting the LDAP server.
|
||||
- Specify `0` (the default) to disable caching and force contacting the LDAP server for each authentication request.
|
||||
- `enable_tls` - flag to trigger use of secure connection to the LDAP server.
|
||||
- Specify `no` for plain text `ldap://` protocol (not recommended).
|
||||
- Specify `yes` for LDAP over SSL/TLS `ldaps://` protocol (recommended, the default).
|
||||
- Specify `starttls` for legacy StartTLS protocol (plain text `ldap://` protocol, upgraded to TLS).
|
||||
- `tls_minimum_protocol_version` - the minimum protocol version of SSL/TLS.
|
||||
- Accepted values are: `ssl2`, `ssl3`, `tls1.0`, `tls1.1`, `tls1.2` (the default).
|
||||
- `tls_require_cert` - SSL/TLS peer certificate verification behavior.
|
||||
- Accepted values are: `never`, `allow`, `try`, `demand` (the default).
|
||||
- `tls_cert_file` - path to certificate file.
|
||||
- `tls_key_file` - path to certificate key file.
|
||||
- `tls_ca_cert_file` - path to CA certificate file.
|
||||
- `tls_ca_cert_dir` - path to the directory containing CA certificates.
|
||||
- `tls_cipher_suite` - allowed cipher suite (in OpenSSL notation).
|
||||
|
||||
## LDAP External Authenticator {#ldap-external-authenticator}
|
||||
|
||||
A remote LDAP server can be used as a method for verifying passwords for locally defined users (users defined in `users.xml` or in local access control paths). In order to achieve this, specify previously defined LDAP server name instead of `password` or similar sections in the user definition.
|
||||
|
||||
At each login attempt, ClickHouse will try to "bind" to the specified DN defined by the `bind_dn` parameter in the [LDAP server definition](#ldap-server-definition) using the provided credentials, and if successful, the user will be considered authenticated. This is often called a "simple bind" method.
|
||||
|
||||
For example,
|
||||
|
||||
```xml
|
||||
<yandex>
|
||||
<!- ... -->
|
||||
<users>
|
||||
<!- ... -->
|
||||
<my_user>
|
||||
<!- ... -->
|
||||
<ldap>
|
||||
<server>my_ldap_server</server>
|
||||
</ldap>
|
||||
</my_user>
|
||||
</users>
|
||||
</yandex>
|
||||
```
|
||||
|
||||
Note, that user `my_user` refers to `my_ldap_server`. This LDAP server must be configured in the main `config.xml` file as described previously.
|
||||
|
||||
When SQL-driven [Access Control and Account Management](../access-rights.md#access-control) is enabled in ClickHouse, users that are authenticated by LDAP servers can also be created using the [CRATE USER](../../sql-reference/statements/create/user.md#create-user-statement) statement.
|
||||
|
||||
|
||||
```sql
|
||||
CREATE USER my_user IDENTIFIED WITH ldap_server BY 'my_ldap_server'
|
||||
```
|
||||
|
||||
## LDAP Exernal User Directory {#ldap-external-user-directory}
|
||||
|
||||
In addition to the locally defined users, a remote LDAP server can be used as a source of user definitions. In order to achieve this, specify previously defined LDAP server name (see [LDAP Server Definition](#ldap-server-definition)) in the `ldap` section inside the `users_directories` section of the `config.xml` file.
|
||||
|
||||
At each login attempt, ClickHouse will try to find the user definition locally and authenticate it as usual, but if the user is not defined, ClickHouse will assume it exists in the external LDAP directory, and will try to "bind" to the specified DN at the LDAP server using the provided credentials. If successful, the user will be considered existing and authenticated. The user will be assigned roles from the list specified in the `roles` section. Additionally, LDAP "search" can be performed and results can be transformed and treated as role names and then be assigned to the user if the `role_mapping` section is also configured. All this implies that the SQL-driven [Access Control and Account Management](../access-rights.md#access-control) is enabled and roles are created using the [CREATE ROLE](../../sql-reference/statements/create/role.md#create-role-statement) statement.
|
||||
|
||||
Example (goes into `config.xml`):
|
||||
|
||||
```xml
|
||||
<yandex>
|
||||
<!- ... -->
|
||||
<user_directories>
|
||||
<!- ... -->
|
||||
<ldap>
|
||||
<server>my_ldap_server</server>
|
||||
<roles>
|
||||
<my_local_role1 />
|
||||
<my_local_role2 />
|
||||
</roles>
|
||||
<role_mapping>
|
||||
<base_dn>ou=groups,dc=example,dc=com</base_dn>
|
||||
<scope>subtree</scope>
|
||||
<search_filter>(&(objectClass=groupOfNames)(member={bind_dn}))</search_filter>
|
||||
<attribute>cn</attribute>
|
||||
<prefix>clickhouse_</prefix>
|
||||
</role_mapping>
|
||||
</ldap>
|
||||
</user_directories>
|
||||
</yandex>
|
||||
```
|
||||
|
||||
Note that `my_ldap_server` referred in the `ldap` section inside the `user_directories` section must be a previously
|
||||
defined LDAP server that is configured in the `config.xml` (see [LDAP Server Definition](#ldap-server-definition)).
|
||||
|
||||
Parameters:
|
||||
|
||||
- `server` - one of LDAP server names defined in the `ldap_servers` config section above.
|
||||
This parameter is mandatory and cannot be empty.
|
||||
- `roles` - section with a list of locally defined roles that will be assigned to each user retrieved from the LDAP server.
|
||||
- If no roles are specified here or assigned during role mapping (below), user will not be able
|
||||
to perform any actions after authentication.
|
||||
- `role_mapping` - section with LDAP search parameters and mapping rules.
|
||||
- When a user authenticates, while still bound to LDAP, an LDAP search is performed using `search_filter`
|
||||
and the name of the logged in user. For each entry found during that search, the value of the specified
|
||||
attribute is extracted. For each attribute value that has the specified prefix, the prefix is removed,
|
||||
and the rest of the value becomes the name of a local role defined in ClickHouse,
|
||||
which is expected to be created beforehand by the [CREATE ROLE](../../sql-reference/statements/create/role.md#create-role-statement) statement.
|
||||
- There can be multiple `role_mapping` sections defined inside the same `ldap` section. All of them will be applied.
|
||||
- `base_dn` - template used to construct the base DN for the LDAP search.
|
||||
- The resulting DN will be constructed by replacing all `{user_name}` and `{bind_dn}`
|
||||
substrings of the template with the actual user name and bind DN during each LDAP search.
|
||||
- `scope` - scope of the LDAP search.
|
||||
- Accepted values are: `base`, `one_level`, `children`, `subtree` (the default).
|
||||
- `search_filter` - template used to construct the search filter for the LDAP search.
|
||||
- The resulting filter will be constructed by replacing all `{user_name}`, `{bind_dn}`, and `{base_dn}`
|
||||
substrings of the template with the actual user name, bind DN, and base DN during each LDAP search.
|
||||
- Note, that the special characters must be escaped properly in XML.
|
||||
- `attribute` - attribute name whose values will be returned by the LDAP search.
|
||||
- `prefix` - prefix, that will be expected to be in front of each string in the original
|
||||
list of strings returned by the LDAP search. Prefix will be removed from the original
|
||||
strings and resulting strings will be treated as local role names. Empty, by default.
|
||||
|
@ -72,7 +72,7 @@ If an aggregate function doesn’t 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 doesn’t include the `stop` value `[start, stop)`.
|
||||
|
@ -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 isn’t 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 isn’t 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, it’s value can’t 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, it’s value can’t 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).
|
||||
|
||||
|
@ -20,7 +20,7 @@ or
|
||||
argMax(tuple(arg, val))
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
- `arg` — Argument.
|
||||
- `val` — Value.
|
||||
|
@ -20,7 +20,7 @@ or
|
||||
argMin(tuple(arg, val))
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
- `arg` — Argument.
|
||||
- `val` — Value.
|
||||
|
@ -12,7 +12,7 @@ Calculates the arithmetic mean.
|
||||
avgWeighted(x)
|
||||
```
|
||||
|
||||
**Parameter**
|
||||
**Arguments**
|
||||
|
||||
- `x` — Values.
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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:
|
||||
|
||||
|
@ -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).
|
||||
|
@ -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.
|
||||
|
@ -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.
|
||||
|
@ -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`.
|
||||
|
@ -10,7 +10,7 @@ Applies bitwise `AND` for series of numbers.
|
||||
groupBitAnd(expr)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
`expr` – An expression that results in `UInt*` type.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -10,7 +10,7 @@ Applies bitwise `OR` for series of numbers.
|
||||
groupBitOr(expr)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
`expr` – An expression that results in `UInt*` type.
|
||||
|
||||
|
@ -10,7 +10,7 @@ Applies bitwise `XOR` for series of numbers.
|
||||
groupBitXor(expr)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
`expr` – An expression that results in `UInt*` type.
|
||||
|
||||
|
@ -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).
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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,13 +28,11 @@ 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**
|
||||
|
||||
[Tuple](../../../sql-reference/data-types/tuple.md) with two elements:
|
||||
|
||||
- calculated U-statistic. [Float64](../../../sql-reference/data-types/float.md).
|
||||
- calculated p-value. [Float64](../../../sql-reference/data-types/float.md).
|
||||
|
||||
|
@ -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).
|
||||
|
@ -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).
|
||||
|
@ -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).
|
||||
|
@ -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).
|
||||
|
@ -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).
|
||||
|
@ -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).
|
||||
|
@ -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).
|
||||
|
||||
|
@ -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).
|
||||
|
||||
|
@ -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).
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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).
|
||||
@ -24,6 +24,7 @@ The null hypothesis is that means of populations are equal. Normal distribution
|
||||
**Returned values**
|
||||
|
||||
[Tuple](../../../sql-reference/data-types/tuple.md) with two elements:
|
||||
|
||||
- calculated t-statistic. [Float64](../../../sql-reference/data-types/float.md).
|
||||
- calculated p-value. [Float64](../../../sql-reference/data-types/float.md).
|
||||
|
||||
|
@ -16,7 +16,7 @@ This function doesn’t 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.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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).
|
||||
@ -24,6 +24,7 @@ The null hypothesis is that means of populations are equal. Normal distribution
|
||||
**Returned values**
|
||||
|
||||
[Tuple](../../../sql-reference/data-types/tuple.md) with two elements:
|
||||
|
||||
- calculated t-statistic. [Float64](../../../sql-reference/data-types/float.md).
|
||||
- calculated p-value. [Float64](../../../sql-reference/data-types/float.md).
|
||||
|
||||
|
@ -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).
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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).
|
||||
|
||||
|
@ -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).
|
||||
|
||||
|
@ -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).
|
||||
|
@ -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).
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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).
|
||||
|
@ -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).
|
||||
|
@ -9,7 +9,7 @@ Hash functions can be used for the deterministic pseudo-random shuffling of elem
|
||||
|
||||
## halfMD5 {#hash-functions-halfmd5}
|
||||
|
||||
[Interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the [MD5](https://en.wikipedia.org/wiki/MD5) hash value for each of them. Then combines hashes, takes the first 8 bytes of the hash of the resulting string, and interprets them as `UInt64` in big-endian byte order.
|
||||
[Interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-reinterpretAsString) all the input parameters as strings and calculates the [MD5](https://en.wikipedia.org/wiki/MD5) hash value for each of them. Then combines hashes, takes the first 8 bytes of the hash of the resulting string, and interprets them as `UInt64` in big-endian byte order.
|
||||
|
||||
``` sql
|
||||
halfMD5(par1, ...)
|
||||
@ -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**
|
||||
|
||||
@ -54,16 +54,16 @@ sipHash64(par1,...)
|
||||
|
||||
This is a cryptographic hash function. It works at least three times faster than the [MD5](#hash_functions-md5) function.
|
||||
|
||||
Function [interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. Then combines hashes by the following algorithm:
|
||||
Function [interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. Then combines hashes by the following algorithm:
|
||||
|
||||
1. After hashing all the input parameters, the function gets the array of hashes.
|
||||
2. Function takes the first and the second elements and calculates a hash for the array of them.
|
||||
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.
|
||||
|
||||
|
@ -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).
|
||||
|
||||
|
@ -275,7 +275,7 @@ Determines whether the input string is an IPv4 address or not. If `string` is IP
|
||||
isIPv4String(string)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
- `string` — IP address. [String](../../sql-reference/data-types/string.md).
|
||||
|
||||
@ -313,7 +313,7 @@ Determines whether the input string is an IPv6 address or not. If `string` is IP
|
||||
isIPv6String(string)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
- `string` — IP address. [String](../../sql-reference/data-types/string.md).
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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:
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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).
|
||||
@ -924,7 +924,7 @@ The result of the function depends on the order of data in the block. It assumes
|
||||
runningConcurrency(begin, end)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**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).
|
||||
@ -989,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`.
|
||||
|
||||
@ -1018,7 +1018,7 @@ Returns size on disk (without taking into account compression).
|
||||
blockSerializedSize(value[, value[, ...]])
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
- `value` — Any value.
|
||||
|
||||
@ -1050,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.
|
||||
|
||||
@ -1090,7 +1090,7 @@ Outputs a detailed description of data structures in RAM
|
||||
dumpColumnStructure(value)
|
||||
```
|
||||
|
||||
**Parameters:**
|
||||
**Arguments:**
|
||||
|
||||
- `value` — Any type of value.
|
||||
|
||||
@ -1120,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.
|
||||
|
||||
@ -1162,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.
|
||||
|
||||
@ -1204,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.
|
||||
@ -1337,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).
|
||||
|
||||
@ -1441,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.
|
||||
@ -1547,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.
|
||||
@ -1651,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.
|
||||
|
||||
@ -1687,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.
|
||||
|
||||
@ -1735,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).
|
||||
|
||||
@ -1773,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).
|
||||
|
||||
@ -1845,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).
|
||||
@ -1882,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.
|
||||
|
||||
@ -1941,7 +1941,7 @@ Returns [native interface](../../interfaces/tcp.md) TCP port number listened by
|
||||
tcpPort()
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
- None.
|
||||
|
||||
|
@ -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`
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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).
|
||||
|
@ -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).
|
||||
|
||||
@ -600,4 +600,46 @@ Hello, "world"!
|
||||
'foo'
|
||||
```
|
||||
|
||||
## decodeXMLComponent {#decode-xml-component}
|
||||
|
||||
Replaces XML predefined entities with characters. Predefined entities are `"` `&` `'` `>` `<`
|
||||
This function also replaces numeric character references with Unicode characters. Both decimal (like `✓`) and hexadecimal (`✓`) forms are supported.
|
||||
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
decodeXMLComponent(x)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
- `x` — A sequence of characters. [String](../../sql-reference/data-types/string.md).
|
||||
|
||||
**Returned value**
|
||||
|
||||
- The sequence of characters after replacement.
|
||||
|
||||
Type: [String](../../sql-reference/data-types/string.md).
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT decodeXMLComponent(''foo'');
|
||||
SELECT decodeXMLComponent('< Σ >');
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
'foo'
|
||||
< Σ >
|
||||
```
|
||||
|
||||
**See Also**
|
||||
|
||||
- [List of XML and HTML character entity references](https://en.wikipedia.org/wiki/List_of_XML_and_HTML_character_entity_references)
|
||||
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/query_language/functions/string_functions/) <!--hide-->
|
||||
|
@ -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).
|
||||
|
@ -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).
|
||||
|
||||
|
@ -15,7 +15,7 @@ Arranges `key:value` pairs into [Map(key, value)](../../sql-reference/data-types
|
||||
map(key1, value1[, key2, value2, ...])
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**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).
|
||||
@ -77,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.
|
||||
|
||||
@ -111,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.
|
||||
|
||||
@ -149,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)).
|
||||
|
@ -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.
|
||||
@ -303,7 +303,7 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut
|
||||
└────────────┴───────┘
|
||||
```
|
||||
|
||||
## reinterpretAs(x, T) {#type_conversion_function-cast}
|
||||
## reinterpret(x, T) {#type_conversion_function-reinterpret}
|
||||
|
||||
Performs byte reinterpretation of ‘x’ as ‘t’ data type.
|
||||
|
||||
@ -313,9 +313,9 @@ Following reinterpretations are allowed:
|
||||
3. FixedString, String, types that can be interpreted as numeric (Integers, Float, Date, DateTime, UUID) into types that can be interpreted as numeric (Integers, Float, Date, DateTime, UUID) into FixedString,
|
||||
|
||||
``` sql
|
||||
SELECT reinterpretAs(toInt8(-1), 'UInt8') as int_to_uint,
|
||||
reinterpretAs(toInt8(1), 'Float32') as int_to_float,
|
||||
reinterpretAs('1', 'UInt32') as string_to_int;
|
||||
SELECT reinterpret(toInt8(-1), 'UInt8') as int_to_uint,
|
||||
reinterpret(toInt8(1), 'Float32') as int_to_float,
|
||||
reinterpret('1', 'UInt32') as string_to_int;
|
||||
```
|
||||
|
||||
``` text
|
||||
@ -324,23 +324,27 @@ SELECT reinterpretAs(toInt8(-1), 'UInt8') as int_to_uint,
|
||||
└─────────────┴──────────────┴───────────────┘
|
||||
```
|
||||
|
||||
## reinterpretAsUInt(8\|16\|32\|64\|256) {#reinterpretasuint8163264256}
|
||||
## reinterpretAsUInt(8\|16\|32\|64\|256) {#reinterpretAsUInt8163264256}
|
||||
|
||||
## reinterpretAsInt(8\|16\|32\|64\|128\|256) {#reinterpretasint8163264128256}
|
||||
## reinterpretAsInt(8\|16\|32\|64\|128\|256) {#reinterpretAsInt8163264128256}
|
||||
|
||||
## reinterpretAsFloat(32\|64) {#reinterpretasfloat3264}
|
||||
## reinterpretAsDecimal(32\|64\|128\|256) {#reinterpretAsDecimal3264128256}
|
||||
|
||||
## reinterpretAsDate {#reinterpretasdate}
|
||||
## reinterpretAsFloat(32\|64) {#type_conversion_function-reinterpretAsFloat}
|
||||
|
||||
## reinterpretAsDateTime {#reinterpretasdatetime}
|
||||
## reinterpretAsDate {#type_conversion_function-reinterpretAsDate}
|
||||
|
||||
## reinterpretAsString {#type_conversion_functions-reinterpretAsString}
|
||||
## reinterpretAsDateTime {#type_conversion_function-reinterpretAsDateTime}
|
||||
|
||||
## reinterpretAsFixedString {#reinterpretasfixedstring}
|
||||
## reinterpretAsDateTime64 {#type_conversion_function-reinterpretAsDateTime64}
|
||||
|
||||
## reinterpretAsUUID {#reinterpretasuuid}
|
||||
## reinterpretAsString {#type_conversion_function-reinterpretAsString}
|
||||
|
||||
These functions are aliases for `reinterpretAs`function.
|
||||
## reinterpretAsFixedString {#type_conversion_function-reinterpretAsFixedString}
|
||||
|
||||
## reinterpretAsUUID {#type_conversion_function-reinterpretAsUUID}
|
||||
|
||||
These functions are aliases for `reinterpret` function.
|
||||
|
||||
## CAST(x, T) {#type_conversion_function-cast}
|
||||
|
||||
@ -401,7 +405,7 @@ bounds of type T.
|
||||
|
||||
Example
|
||||
``` sql
|
||||
SELECT cast(-1, 'UInt8') as uint8;
|
||||
SELECT cast(-1, 'UInt8') as uint8;
|
||||
```
|
||||
|
||||
|
||||
@ -422,7 +426,7 @@ Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Value in c
|
||||
|
||||
## accurateCastOrNull(x, T) {#type_conversion_function-accurate-cast_or_null}
|
||||
|
||||
Converts ‘x’ to the ‘t’ data type. Always returns nullable type and returns NULL
|
||||
Converts ‘x’ to the ‘t’ data type. Always returns nullable type and returns NULL
|
||||
if the casted value is not representable in the target type.
|
||||
|
||||
Example:
|
||||
@ -467,7 +471,7 @@ toIntervalQuarter(number)
|
||||
toIntervalYear(number)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
- `number` — Duration of interval. Positive integer number.
|
||||
|
||||
@ -505,7 +509,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 +621,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 +705,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 +745,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 +797,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.
|
||||
@ -817,15 +821,15 @@ SELECT fromUnixTimestamp64Milli(i64, 'UTC')
|
||||
|
||||
## formatRow {#formatrow}
|
||||
|
||||
Converts arbitrary expressions into a string via given format.
|
||||
Converts arbitrary expressions into a string via given format.
|
||||
|
||||
**Syntax**
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
formatRow(format, x, y, ...)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
- `format` — Text format. For example, [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated).
|
||||
- `x`,`y`, ... — Expressions.
|
||||
@ -860,13 +864,13 @@ Result:
|
||||
|
||||
Converts arbitrary expressions into a string via given format. The function trims the last `\n` if any.
|
||||
|
||||
**Syntax**
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
formatRowNoNewline(format, x, y, ...)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
- `format` — Text format. For example, [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated).
|
||||
- `x`,`y`, ... — Expressions.
|
||||
|
@ -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).
|
||||
|
||||
@ -370,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).
|
||||
|
||||
|
@ -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.
|
||||
|
@ -25,6 +25,7 @@ The following actions are supported:
|
||||
- [COMMENT COLUMN](#alter_comment-column) — Adds a text comment to the column.
|
||||
- [MODIFY COLUMN](#alter_modify-column) — Changes column’s type, default expression and TTL.
|
||||
- [MODIFY COLUMN REMOVE](#modify-remove) — Removes one of the column properties.
|
||||
- [RENAME COLUMN](#alter_rename-column) — Renames an existing column.
|
||||
|
||||
These actions are described in detail below.
|
||||
|
||||
@ -183,6 +184,22 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL;
|
||||
|
||||
- [REMOVE TTL](ttl.md).
|
||||
|
||||
## RENAME COLUMN {#alter_rename-column}
|
||||
|
||||
Renames an existing column.
|
||||
|
||||
Syntax:
|
||||
|
||||
```sql
|
||||
ALTER TABLE table_name RENAME COLUMN column_name TO new_column_name;
|
||||
```
|
||||
|
||||
**Example**
|
||||
|
||||
```sql
|
||||
ALTER TABLE table_with_ttl RENAME COLUMN column_ttl TO column_ttl_new;
|
||||
```
|
||||
|
||||
## Limitations {#alter-query-limitations}
|
||||
|
||||
The `ALTER` query lets you create and delete separate elements (columns) in nested data structures, but not whole nested data structures. To add a nested data structure, you can add columns with a name like `name.nested_name` and the type `Array(T)`. A nested data structure is equivalent to multiple array columns with a name that has the same prefix before the dot.
|
||||
|
@ -12,7 +12,7 @@ Syntax:
|
||||
``` sql
|
||||
CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1]
|
||||
[, name2 [ON CLUSTER cluster_name2] ...]
|
||||
[IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}]
|
||||
[IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH|LDAP_SERVER}] BY {'password'|'hash'}]
|
||||
[HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
|
||||
[DEFAULT ROLE role [,...]]
|
||||
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]
|
||||
@ -30,6 +30,7 @@ There are multiple ways of user identification:
|
||||
- `IDENTIFIED WITH sha256_hash BY 'hash'`
|
||||
- `IDENTIFIED WITH double_sha1_password BY 'qwerty'`
|
||||
- `IDENTIFIED WITH double_sha1_hash BY 'hash'`
|
||||
- `IDENTIFIED WITH ldap_server BY 'server'`
|
||||
|
||||
## User Host {#user-host}
|
||||
|
||||
|
@ -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.
|
||||
|
@ -13,7 +13,7 @@ Allows `SELECT` and `INSERT` queries to be performed on data that is stored on a
|
||||
mysql('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_duplicate_clause'])
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
- `host:port` — MySQL server address.
|
||||
|
||||
|
@ -13,7 +13,7 @@ Turns a subquery into a table. The function implements views (see [CREATE VIEW](
|
||||
view(subquery)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
**Arguments**
|
||||
|
||||
- `subquery` — `SELECT` query.
|
||||
|
||||
|
@ -52,10 +52,26 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
- `rabbitmq_max_block_size`
|
||||
- `rabbitmq_flush_interval_ms`
|
||||
|
||||
Требуемая конфигурация:
|
||||
Настройки форматов данных также могут быть добавлены в списке RabbitMQ настроек.
|
||||
|
||||
Example:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE queue (
|
||||
key UInt64,
|
||||
value UInt64,
|
||||
date DateTime
|
||||
) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'localhost:5672',
|
||||
rabbitmq_exchange_name = 'exchange1',
|
||||
rabbitmq_format = 'JSONEachRow',
|
||||
rabbitmq_num_consumers = 5,
|
||||
date_time_input_format = 'best_effort';
|
||||
```
|
||||
|
||||
Конфигурация сервера RabbitMQ добавляется с помощью конфигурационного файла ClickHouse.
|
||||
|
||||
Требуемая конфигурация:
|
||||
|
||||
``` xml
|
||||
<rabbitmq>
|
||||
<username>root</username>
|
||||
@ -63,16 +79,12 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
</rabbitmq>
|
||||
```
|
||||
|
||||
Example:
|
||||
Дополнительная конфигурация:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE queue (
|
||||
key UInt64,
|
||||
value UInt64
|
||||
) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'localhost:5672',
|
||||
rabbitmq_exchange_name = 'exchange1',
|
||||
rabbitmq_format = 'JSONEachRow',
|
||||
rabbitmq_num_consumers = 5;
|
||||
``` xml
|
||||
<rabbitmq>
|
||||
<vhost>clickhouse</vhost>
|
||||
</rabbitmq>
|
||||
```
|
||||
|
||||
## Описание {#description}
|
||||
@ -98,6 +110,7 @@ Example:
|
||||
- `consistent_hash` - данные равномерно распределяются между всеми связанными таблицами, где имя точки обмена совпадает. Обратите внимание, что этот тип обмена должен быть включен с помощью плагина RabbitMQ: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`.
|
||||
|
||||
Настройка `rabbitmq_queue_base` может быть использована в следующих случаях:
|
||||
|
||||
1. чтобы восстановить чтение из ранее созданных очередей, если оно прекратилось по какой-либо причине, но очереди остались непустыми. Для восстановления чтения из одной конкретной очереди, нужно написать ее имя в `rabbitmq_queue_base` настройку и не указывать настройки `rabbitmq_num_consumers` и `rabbitmq_num_queues`. Чтобы восстановить чтение из всех очередей, которые были созданы для конкретной таблицы, необходимо совпадение следующих настроек: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. По умолчанию, если настройка `rabbitmq_queue_base` не указана, будут использованы уникальные для каждой таблицы имена очередей.
|
||||
2. чтобы объявить одни и те же очереди для разных таблиц, что позволяет создавать несколько параллельных подписчиков на каждую из очередей. То есть обеспечивается лучшая производительность. В данном случае, для таких таблиц также необходимо совпадение настроек: `rabbitmq_num_consumers`, `rabbitmq_num_queues`.
|
||||
3. чтобы повторно использовать созданные c `durable` настройкой очереди, так как они не удаляются автоматически (но могут быть удалены с помощью любого RabbitMQ CLI).
|
||||
|
@ -31,6 +31,7 @@ mannWhitneyUTest[(alternative[, continuity_correction])](sample_data, sample_ind
|
||||
**Возвращаемые значения**
|
||||
|
||||
[Кортеж](../../../sql-reference/data-types/tuple.md) с двумя элементами:
|
||||
|
||||
- вычисленное значение критерия Манна — Уитни. [Float64](../../../sql-reference/data-types/float.md).
|
||||
- вычисленное p-значение. [Float64](../../../sql-reference/data-types/float.md).
|
||||
|
||||
|
@ -24,6 +24,7 @@ studentTTest(sample_data, sample_index)
|
||||
**Возвращаемые значения**
|
||||
|
||||
[Кортеж](../../../sql-reference/data-types/tuple.md) с двумя элементами:
|
||||
|
||||
- вычисленное значение критерия Стьюдента. [Float64](../../../sql-reference/data-types/float.md).
|
||||
- вычисленное p-значение. [Float64](../../../sql-reference/data-types/float.md).
|
||||
|
||||
|
@ -24,6 +24,7 @@ welchTTest(sample_data, sample_index)
|
||||
**Возвращаемые значения**
|
||||
|
||||
[Кортеж](../../../sql-reference/data-types/tuple.md) с двумя элементами:
|
||||
|
||||
- вычисленное значение критерия Уэлча. [Float64](../../../sql-reference/data-types/float.md).
|
||||
- вычисленное p-значение. [Float64](../../../sql-reference/data-types/float.md).
|
||||
|
||||
|
@ -597,4 +597,46 @@ Hello, "world"!
|
||||
'foo'
|
||||
```
|
||||
|
||||
|
||||
## decodeXMLComponent {#decode-xml-component}
|
||||
|
||||
Заменяет символами предопределенные мнемоники XML: `"` `&` `'` `>` `<`
|
||||
Также эта функция заменяет числовые ссылки соответствующими символами юникод. Поддерживаются десятичная (например, `✓`) и шестнадцатеричная (`✓`) формы.
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
``` sql
|
||||
decodeXMLComponent(x)
|
||||
```
|
||||
|
||||
**Параметры**
|
||||
|
||||
- `x` — последовательность символов. [String](../../sql-reference/data-types/string.md).
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- Строка с произведенными заменами.
|
||||
|
||||
Тип: [String](../../sql-reference/data-types/string.md).
|
||||
|
||||
**Пример**
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT decodeXMLComponent(''foo'');
|
||||
SELECT decodeXMLComponent('< Σ >');
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
'foo'
|
||||
< Σ >
|
||||
```
|
||||
|
||||
**Смотрите также**
|
||||
|
||||
- [Мнемоники в HTML](https://ru.wikipedia.org/wiki/%D0%9C%D0%BD%D0%B5%D0%BC%D0%BE%D0%BD%D0%B8%D0%BA%D0%B8_%D0%B2_HTML)
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/string_functions/) <!--hide-->
|
||||
|
@ -4,14 +4,14 @@
|
||||
|
||||
# include <DataTypes/DataTypeFactory.h>
|
||||
# include <DataTypes/DataTypeNullable.h>
|
||||
# include <IO/WriteBufferFromHTTPServerResponse.h>
|
||||
# include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
|
||||
# include <IO/WriteHelpers.h>
|
||||
# include <Parsers/ParserQueryWithOutput.h>
|
||||
# include <Parsers/parseQuery.h>
|
||||
# include <Poco/Data/ODBC/ODBCException.h>
|
||||
# include <Poco/Data/ODBC/SessionImpl.h>
|
||||
# include <Poco/Data/ODBC/Utility.h>
|
||||
# include <Poco/Net/HTMLForm.h>
|
||||
# include <Server/HTTP/HTMLForm.h>
|
||||
# include <Poco/Net/HTTPServerRequest.h>
|
||||
# include <Poco/Net/HTTPServerResponse.h>
|
||||
# include <Poco/NumberParser.h>
|
||||
@ -59,16 +59,16 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
|
||||
void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
|
||||
{
|
||||
Poco::Net::HTMLForm params(request, request.stream());
|
||||
HTMLForm params(request, request.getStream());
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
auto process_error = [&response, this](const std::string & message)
|
||||
{
|
||||
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR);
|
||||
if (!response.sent())
|
||||
response.send() << message << std::endl;
|
||||
*response.send() << message << std::endl;
|
||||
LOG_WARNING(log, message);
|
||||
};
|
||||
|
||||
@ -159,8 +159,16 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
|
||||
columns.emplace_back(reinterpret_cast<char *>(column_name), std::move(column_type));
|
||||
}
|
||||
|
||||
WriteBufferFromHTTPServerResponse out(request, response, keep_alive_timeout);
|
||||
writeStringBinary(columns.toString(), out);
|
||||
WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout);
|
||||
try
|
||||
{
|
||||
writeStringBinary(columns.toString(), out);
|
||||
out.finalize();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
out.finalize();
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -3,10 +3,11 @@
|
||||
#if USE_ODBC
|
||||
|
||||
# include <Interpreters/Context.h>
|
||||
# include <Poco/Logger.h>
|
||||
# include <Poco/Net/HTTPRequestHandler.h>
|
||||
# include <Server/HTTP/HTTPRequestHandler.h>
|
||||
# include <Common/config.h>
|
||||
|
||||
# include <Poco/Logger.h>
|
||||
|
||||
/** The structure of the table is taken from the query "SELECT * FROM table WHERE 1=0".
|
||||
* TODO: It would be much better to utilize ODBC methods dedicated for columns description.
|
||||
* If there is no such table, an exception is thrown.
|
||||
@ -14,7 +15,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ODBCColumnsInfoHandler : public Poco::Net::HTTPRequestHandler
|
||||
class ODBCColumnsInfoHandler : public HTTPRequestHandler
|
||||
{
|
||||
public:
|
||||
ODBCColumnsInfoHandler(size_t keep_alive_timeout_, Context & context_)
|
||||
@ -22,7 +23,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override;
|
||||
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
|
@ -7,39 +7,40 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
Poco::Net::HTTPRequestHandler * HandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request)
|
||||
|
||||
std::unique_ptr<HTTPRequestHandler> HandlerFactory::createRequestHandler(const HTTPServerRequest & request)
|
||||
{
|
||||
Poco::URI uri{request.getURI()};
|
||||
LOG_TRACE(log, "Request URI: {}", uri.toString());
|
||||
|
||||
if (uri.getPath() == "/ping" && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET)
|
||||
return new PingHandler(keep_alive_timeout);
|
||||
return std::make_unique<PingHandler>(keep_alive_timeout);
|
||||
|
||||
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST)
|
||||
{
|
||||
|
||||
if (uri.getPath() == "/columns_info")
|
||||
#if USE_ODBC
|
||||
return new ODBCColumnsInfoHandler(keep_alive_timeout, context);
|
||||
return std::make_unique<ODBCColumnsInfoHandler>(keep_alive_timeout, context);
|
||||
#else
|
||||
return nullptr;
|
||||
#endif
|
||||
else if (uri.getPath() == "/identifier_quote")
|
||||
#if USE_ODBC
|
||||
return new IdentifierQuoteHandler(keep_alive_timeout, context);
|
||||
return std::make_unique<IdentifierQuoteHandler>(keep_alive_timeout, context);
|
||||
#else
|
||||
return nullptr;
|
||||
#endif
|
||||
else if (uri.getPath() == "/schema_allowed")
|
||||
#if USE_ODBC
|
||||
return new SchemaAllowedHandler(keep_alive_timeout, context);
|
||||
return std::make_unique<SchemaAllowedHandler>(keep_alive_timeout, context);
|
||||
#else
|
||||
return nullptr;
|
||||
#endif
|
||||
else if (uri.getPath() == "/write")
|
||||
return new ODBCHandler(pool_map, keep_alive_timeout, context, "write");
|
||||
return std::make_unique<ODBCHandler>(pool_map, keep_alive_timeout, context, "write");
|
||||
else
|
||||
return new ODBCHandler(pool_map, keep_alive_timeout, context, "read");
|
||||
return std::make_unique<ODBCHandler>(pool_map, keep_alive_timeout, context, "read");
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user