Merge branch 'master' into fix/datetime

This commit is contained in:
zhang2014 2020-09-09 23:37:54 +08:00
commit 094897fe69
556 changed files with 8399 additions and 3406 deletions

View File

@ -22,14 +22,14 @@
* Add setting `allow_non_metadata_alters` which restricts to execute `ALTER` queries which modify data on disk. Disabled be default. Closes [#11547](https://github.com/ClickHouse/ClickHouse/issues/11547). [#12635](https://github.com/ClickHouse/ClickHouse/pull/12635) ([alesapin](https://github.com/alesapin)).
* A function `formatRow` is added to support turning arbitrary expressions into a string via given format. It's useful for manipulating SQL outputs and is quite versatile combined with the `columns` function. [#12574](https://github.com/ClickHouse/ClickHouse/pull/12574) ([Amos Bird](https://github.com/amosbird)).
* Add `FROM_UNIXTIME` function for compatibility with MySQL, related to [12149](https://github.com/ClickHouse/ClickHouse/issues/12149). [#12484](https://github.com/ClickHouse/ClickHouse/pull/12484) ([flynn](https://github.com/ucasFL)).
* Allow Nullable types as keys in MergeTree tables if `allow_nullable_key` table setting is enabled. https://github.com/ClickHouse/ClickHouse/issues/5319. [#12433](https://github.com/ClickHouse/ClickHouse/pull/12433) ([Amos Bird](https://github.com/amosbird)).
* Allow Nullable types as keys in MergeTree tables if `allow_nullable_key` table setting is enabled. Closes [#5319](https://github.com/ClickHouse/ClickHouse/issues/5319). [#12433](https://github.com/ClickHouse/ClickHouse/pull/12433) ([Amos Bird](https://github.com/amosbird)).
* Integration with [COS](https://intl.cloud.tencent.com/product/cos). [#12386](https://github.com/ClickHouse/ClickHouse/pull/12386) ([fastio](https://github.com/fastio)).
* Add mapAdd and mapSubtract functions for adding/subtracting key-mapped values. [#11735](https://github.com/ClickHouse/ClickHouse/pull/11735) ([Ildus Kurbangaliev](https://github.com/ildus)).
#### Bug Fix
* Fix premature `ON CLUSTER` timeouts for queries that must be executed on a single replica. Fixes [#6704](https://github.com/ClickHouse/ClickHouse/issues/6704), [#7228](https://github.com/ClickHouse/ClickHouse/issues/7228), [#13361](https://github.com/ClickHouse/ClickHouse/issues/13361), [#11884](https://github.com/ClickHouse/ClickHouse/issues/11884). [#13450](https://github.com/ClickHouse/ClickHouse/pull/13450) ([alesapin](https://github.com/alesapin)).
* Fix crash in mark inclusion search introduced in https://github.com/ClickHouse/ClickHouse/pull/12277. [#14225](https://github.com/ClickHouse/ClickHouse/pull/14225) ([Amos Bird](https://github.com/amosbird)).
* Fix crash in mark inclusion search introduced in [#12277](https://github.com/ClickHouse/ClickHouse/pull/12277). [#14225](https://github.com/ClickHouse/ClickHouse/pull/14225) ([Amos Bird](https://github.com/amosbird)).
* Fix race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)).
* Fix visible data clobbering by progress bar in client in interactive mode. This fixes [#12562](https://github.com/ClickHouse/ClickHouse/issues/12562) and [#13369](https://github.com/ClickHouse/ClickHouse/issues/13369) and [#13584](https://github.com/ClickHouse/ClickHouse/issues/13584) and fixes [#12964](https://github.com/ClickHouse/ClickHouse/issues/12964). [#13691](https://github.com/ClickHouse/ClickHouse/pull/13691) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fixed incorrect sorting order for `LowCardinality` columns when ORDER BY multiple columns is used. This fixes [#13958](https://github.com/ClickHouse/ClickHouse/issues/13958). [#14223](https://github.com/ClickHouse/ClickHouse/pull/14223) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
@ -71,7 +71,7 @@
* Fix function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fix assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fix DateTime64 conversion functions with constant argument. [#13205](https://github.com/ClickHouse/ClickHouse/pull/13205) ([Azat Khuzhin](https://github.com/azat)).
* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes https://github.com/ClickHouse/ClickHouse/issues/5779, https://github.com/ClickHouse/ClickHouse/issues/12527. [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix access to `redis` dictionary after connection was dropped once. It may happen with `cache` and `direct` dictionary layouts. [#13082](https://github.com/ClickHouse/ClickHouse/pull/13082) ([Anton Popov](https://github.com/CurtizJ)).
* Fix wrong index analysis with functions. It could lead to some data parts being skipped when reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)).
* Fix error `Cannot convert column because it is constant but values of constants are different in source and result` for remote queries which use deterministic functions in scope of query, but not deterministic between queries, like `now()`, `now64()`, `randConstant()`. Fixes [#11327](https://github.com/ClickHouse/ClickHouse/issues/11327). [#13075](https://github.com/ClickHouse/ClickHouse/pull/13075) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
@ -89,7 +89,7 @@
* Fixed [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) fix bloom filter index with const expression. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)).
* Fix SIGSEGV in StorageKafka when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)).
* Add support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* CREATE USER IF NOT EXISTS now doesn't throw exception if the user exists. This fixes https://github.com/ClickHouse/ClickHouse/issues/12507. [#12646](https://github.com/ClickHouse/ClickHouse/pull/12646) ([Vitaly Baranov](https://github.com/vitlibar)).
* CREATE USER IF NOT EXISTS now doesn't throw exception if the user exists. This fixes [#12507](https://github.com/ClickHouse/ClickHouse/issues/12507). [#12646](https://github.com/ClickHouse/ClickHouse/pull/12646) ([Vitaly Baranov](https://github.com/vitlibar)).
* Exception `There is no supertype...` can be thrown during `ALTER ... UPDATE` in unexpected cases (e.g. when subtracting from UInt64 column). This fixes [#7306](https://github.com/ClickHouse/ClickHouse/issues/7306). This fixes [#4165](https://github.com/ClickHouse/ClickHouse/issues/4165). [#12633](https://github.com/ClickHouse/ClickHouse/pull/12633) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fix possible `Pipeline stuck` error for queries with external sorting. Fixes [#12617](https://github.com/ClickHouse/ClickHouse/issues/12617). [#12618](https://github.com/ClickHouse/ClickHouse/pull/12618) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix error `Output of TreeExecutor is not sorted` for `OPTIMIZE DEDUPLICATE`. Fixes [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572). [#12613](https://github.com/ClickHouse/ClickHouse/pull/12613) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
@ -123,7 +123,7 @@
* Fix assert in `parseDateTimeBestEffort`. This fixes [#12649](https://github.com/ClickHouse/ClickHouse/issues/12649). [#13227](https://github.com/ClickHouse/ClickHouse/pull/13227) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Minor optimization in Processors/PipelineExecutor: breaking out of a loop because it makes sense to do so. [#13058](https://github.com/ClickHouse/ClickHouse/pull/13058) ([Mark Papadakis](https://github.com/markpapadakis)).
* Support TRUNCATE table without TABLE keyword. [#12653](https://github.com/ClickHouse/ClickHouse/pull/12653) ([Winter Zhang](https://github.com/zhang2014)).
* Fix explain query format overwrite by default, issue https://github.com/ClickHouse/ClickHouse/issues/12432. [#12541](https://github.com/ClickHouse/ClickHouse/pull/12541) ([BohuTANG](https://github.com/BohuTANG)).
* Fix explain query format overwrite by default. This fixes [#12541](https://github.com/ClickHouse/ClickHouse/issues/12432). [#12541](https://github.com/ClickHouse/ClickHouse/pull/12541) ([BohuTANG](https://github.com/BohuTANG)).
* Allow to set JOIN kind and type in more standad way: `LEFT SEMI JOIN` instead of `SEMI LEFT JOIN`. For now both are correct. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)).
* Changes default value for `multiple_joins_rewriter_version` to 2. It enables new multiple joins rewriter that knows about column names. [#12469](https://github.com/ClickHouse/ClickHouse/pull/12469) ([Artem Zuikov](https://github.com/4ertus2)).
* Add several metrics for requests to S3 storages. [#12464](https://github.com/ClickHouse/ClickHouse/pull/12464) ([ianton-ru](https://github.com/ianton-ru)).

View File

@ -18,3 +18,4 @@ ClickHouse is an open-source column-oriented database management system that all
## Upcoming Events
* [ClickHouse Data Integration Virtual Meetup](https://www.eventbrite.com/e/clickhouse-september-virtual-meetup-data-integration-tickets-117421895049) on September 10, 2020.
* [ClickHouse talk at Ya.Subbotnik (in Russian)](https://ya.cc/t/cIBI-3yECj5JF) on September 12, 2020.

View File

@ -32,6 +32,8 @@ PEERDIR(
contrib/restricted/cityhash-1.0.2
)
CFLAGS(-g0)
SRCS(
argsToConfig.cpp
coverage.cpp

View File

@ -31,6 +31,8 @@ PEERDIR(
contrib/restricted/cityhash-1.0.2
)
CFLAGS(-g0)
SRCS(
<? find . -name '*.cpp' | grep -v -F tests/ | grep -v -F Replxx | grep -v -F Readline | sed 's/^\.\// /' | sort ?>
)

View File

@ -6,6 +6,8 @@ PEERDIR(
clickhouse/src/Common
)
CFLAGS(-g0)
SRCS(
BaseDaemon.cpp
GraphiteWriter.cpp

View File

@ -4,6 +4,8 @@ PEERDIR(
clickhouse/src/Common
)
CFLAGS(-g0)
SRCS(
ExtendedLogChannel.cpp
Loggers.cpp

View File

@ -1,9 +1,7 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <mysqlxx/Types.h>
namespace mysqlxx
{
@ -22,6 +20,11 @@ class ResultBase
public:
ResultBase(MYSQL_RES * res_, Connection * conn_, const Query * query_);
ResultBase(const ResultBase &) = delete;
ResultBase & operator=(const ResultBase &) = delete;
ResultBase(ResultBase &&) = default;
ResultBase & operator=(ResultBase &&) = default;
Connection * getConnection() { return conn; }
MYSQL_FIELDS getFields() { return fields; }
unsigned getNumFields() { return num_fields; }

View File

@ -254,7 +254,23 @@ template <> inline std::string Value::get<std::string >() cons
template <> inline LocalDate Value::get<LocalDate >() const { return getDate(); }
template <> inline LocalDateTime Value::get<LocalDateTime >() const { return getDateTime(); }
template <typename T> inline T Value::get() const { return T(*this); }
namespace details
{
// To avoid stack overflow when converting to type with no appropriate c-tor,
// resulting in endless recursive calls from `Value::get<T>()` to `Value::operator T()` to `Value::get<T>()` to ...
template <typename T, typename std::enable_if_t<std::is_constructible_v<T, Value>>>
inline T contructFromValue(const Value & val)
{
return T(val);
}
}
template <typename T>
inline T Value::get() const
{
return details::contructFromValue<T>(*this);
}
inline std::ostream & operator<< (std::ostream & ostr, const Value & x)

View File

@ -1,5 +1,7 @@
LIBRARY()
CFLAGS(-g0)
SRCS(
readpassphrase.c
)

View File

@ -2,6 +2,8 @@ LIBRARY()
ADDINCL(GLOBAL clickhouse/base/widechar_width)
CFLAGS(-g0)
SRCS(
widechar_width.cpp
)

View File

@ -1,9 +1,9 @@
# This strings autochanged from release_lib.sh:
SET(VERSION_REVISION 54439)
SET(VERSION_REVISION 54440)
SET(VERSION_MAJOR 20)
SET(VERSION_MINOR 9)
SET(VERSION_MINOR 10)
SET(VERSION_PATCH 1)
SET(VERSION_GITHASH 0586f0d555f7481b394afc55bbb29738cd573a1c)
SET(VERSION_DESCRIBE v20.9.1.1-prestable)
SET(VERSION_STRING 20.9.1.1)
SET(VERSION_GITHASH 11a247d2f42010c1a17bf678c3e00a4bc89b23f8)
SET(VERSION_DESCRIBE v20.10.1.1-prestable)
SET(VERSION_STRING 20.10.1.1)
# end of autochange

View File

@ -6,6 +6,11 @@ endif()
if ((ENABLE_CCACHE OR NOT DEFINED ENABLE_CCACHE) AND NOT COMPILER_MATCHES_CCACHE)
find_program (CCACHE_FOUND ccache)
if (CCACHE_FOUND)
set(ENABLE_CCACHE_BY_DEFAULT 1)
else()
set(ENABLE_CCACHE_BY_DEFAULT 0)
endif()
endif()
if (NOT CCACHE_FOUND AND NOT DEFINED ENABLE_CCACHE AND NOT COMPILER_MATCHES_CCACHE)
@ -13,7 +18,7 @@ if (NOT CCACHE_FOUND AND NOT DEFINED ENABLE_CCACHE AND NOT COMPILER_MATCHES_CCAC
"Setting it up will significantly reduce compilation time for 2nd and consequent builds")
endif()
option(ENABLE_CCACHE "Speedup re-compilations using ccache" ${CCACHE_FOUND})
option(ENABLE_CCACHE "Speedup re-compilations using ccache" ${ENABLE_CCACHE_BY_DEFAULT})
if (NOT ENABLE_CCACHE)
return()
@ -24,7 +29,7 @@ if (CCACHE_FOUND AND NOT COMPILER_MATCHES_CCACHE)
string(REGEX REPLACE "ccache version ([0-9\\.]+).*" "\\1" CCACHE_VERSION ${CCACHE_VERSION})
if (CCACHE_VERSION VERSION_GREATER "3.2.0" OR NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
#message(STATUS "Using ${CCACHE_FOUND} ${CCACHE_VERSION}")
message(STATUS "Using ${CCACHE_FOUND} ${CCACHE_VERSION}")
set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE ${CCACHE_FOUND})
set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK ${CCACHE_FOUND})
else ()

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (20.9.1.1) unstable; urgency=low
clickhouse (20.10.1.1) unstable; urgency=low
* Modified source code
-- clickhouse-release <clickhouse-release@yandex-team.ru> Mon, 31 Aug 2020 23:07:38 +0300
-- clickhouse-release <clickhouse-release@yandex-team.ru> Tue, 08 Sep 2020 17:04:39 +0300

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=20.9.1.*
ARG version=20.10.1.*
RUN apt-get update \
&& apt-get install --yes --no-install-recommends \

View File

@ -1,5 +1,5 @@
# docker build -t yandex/clickhouse-binary-builder .
FROM ubuntu:19.10
FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=10
@ -32,6 +32,8 @@ RUN apt-get update \
curl \
gcc-9 \
g++-9 \
gcc-10 \
g++-10 \
llvm-${LLVM_VERSION} \
clang-${LLVM_VERSION} \
lld-${LLVM_VERSION} \

View File

@ -18,7 +18,7 @@ ccache --zero-stats ||:
ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||:
rm -f CMakeCache.txt
cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=$BUILD_TYPE -DSANITIZE=$SANITIZER $CMAKE_FLAGS ..
ninja $NINJA_FLAGS clickhouse-bundle
ninja -j $(($(nproc) / 2)) $NINJA_FLAGS clickhouse-bundle
mv ./programs/clickhouse* /output
mv ./src/unit_tests_dbms /output
find . -name '*.so' -print -exec mv '{}' /output \;

View File

@ -1,7 +1,7 @@
FROM ubuntu:20.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=20.9.1.*
ARG version=20.10.1.*
ARG gosu_ver=1.10
RUN apt-get update \

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=20.9.1.*
ARG version=20.10.1.*
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -5,4 +5,4 @@ services:
restart: always
ports:
- 6380:6379
command: redis-server --requirepass "clickhouse"
command: redis-server --requirepass "clickhouse" --databases 32

View File

@ -2,6 +2,7 @@
# -*- coding: utf-8 -*-
import os
import sys
import time
import tarfile
import logging
import argparse
@ -16,6 +17,8 @@ AVAILABLE_DATASETS = {
'visits': 'visits_v1.tar',
}
RETRIES_COUNT = 5
def _get_temp_file_name():
return os.path.join(tempfile._get_default_tempdir(), next(tempfile._get_candidate_names()))
@ -24,25 +27,37 @@ def build_url(base_url, dataset):
def dowload_with_progress(url, path):
logging.info("Downloading from %s to temp path %s", url, path)
with open(path, 'w') as f:
response = requests.get(url, stream=True)
response.raise_for_status()
total_length = response.headers.get('content-length')
if total_length is None or int(total_length) == 0:
logging.info("No content-length, will download file without progress")
f.write(response.content)
else:
dl = 0
total_length = int(total_length)
logging.info("Content length is %ld bytes", total_length)
for data in response.iter_content(chunk_size=4096):
dl += len(data)
f.write(data)
if sys.stdout.isatty():
done = int(50 * dl / total_length)
percent = int(100 * float(dl) / total_length)
sys.stdout.write("\r[{}{}] {}%".format('=' * done, ' ' * (50-done), percent))
sys.stdout.flush()
for i in range(RETRIES_COUNT):
try:
with open(path, 'w') as f:
response = requests.get(url, stream=True)
response.raise_for_status()
total_length = response.headers.get('content-length')
if total_length is None or int(total_length) == 0:
logging.info("No content-length, will download file without progress")
f.write(response.content)
else:
dl = 0
total_length = int(total_length)
logging.info("Content length is %ld bytes", total_length)
for data in response.iter_content(chunk_size=4096):
dl += len(data)
f.write(data)
if sys.stdout.isatty():
done = int(50 * dl / total_length)
percent = int(100 * float(dl) / total_length)
sys.stdout.write("\r[{}{}] {}%".format('=' * done, ' ' * (50-done), percent))
sys.stdout.flush()
break
except Exception as ex:
sys.stdout.write("\n")
time.sleep(3)
logging.info("Exception while downloading %s, retry %s", ex, i + 1)
if os.path.exists(path):
os.remove(path)
else:
raise Exception("Cannot download dataset from {}, all retries exceeded".format(url))
sys.stdout.write("\n")
logging.info("Downloading finished")

View File

@ -2,6 +2,7 @@
# -*- coding: utf-8 -*-
import os
import sys
import time
import tarfile
import logging
import argparse
@ -16,6 +17,8 @@ AVAILABLE_DATASETS = {
'visits': 'visits_v1.tar',
}
RETRIES_COUNT = 5
def _get_temp_file_name():
return os.path.join(tempfile._get_default_tempdir(), next(tempfile._get_candidate_names()))
@ -24,25 +27,37 @@ def build_url(base_url, dataset):
def dowload_with_progress(url, path):
logging.info("Downloading from %s to temp path %s", url, path)
with open(path, 'w') as f:
response = requests.get(url, stream=True)
response.raise_for_status()
total_length = response.headers.get('content-length')
if total_length is None or int(total_length) == 0:
logging.info("No content-length, will download file without progress")
f.write(response.content)
else:
dl = 0
total_length = int(total_length)
logging.info("Content length is %ld bytes", total_length)
for data in response.iter_content(chunk_size=4096):
dl += len(data)
f.write(data)
if sys.stdout.isatty():
done = int(50 * dl / total_length)
percent = int(100 * float(dl) / total_length)
sys.stdout.write("\r[{}{}] {}%".format('=' * done, ' ' * (50-done), percent))
sys.stdout.flush()
for i in range(RETRIES_COUNT):
try:
with open(path, 'w') as f:
response = requests.get(url, stream=True)
response.raise_for_status()
total_length = response.headers.get('content-length')
if total_length is None or int(total_length) == 0:
logging.info("No content-length, will download file without progress")
f.write(response.content)
else:
dl = 0
total_length = int(total_length)
logging.info("Content length is %ld bytes", total_length)
for data in response.iter_content(chunk_size=4096):
dl += len(data)
f.write(data)
if sys.stdout.isatty():
done = int(50 * dl / total_length)
percent = int(100 * float(dl) / total_length)
sys.stdout.write("\r[{}{}] {}%".format('=' * done, ' ' * (50-done), percent))
sys.stdout.flush()
break
except Exception as ex:
sys.stdout.write("\n")
time.sleep(3)
logging.info("Exception while downloading %s, retry %s", ex, i + 1)
if os.path.exists(path):
os.remove(path)
else:
raise Exception("Cannot download dataset from {}, all retries exceeded".format(url))
sys.stdout.write("\n")
logging.info("Downloading finished")

View File

@ -27,9 +27,15 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
[rabbitmq_exchange_type = 'exchange_type',]
[rabbitmq_routing_key_list = 'key1,key2,...',]
[rabbitmq_row_delimiter = 'delimiter_symbol',]
[rabbitmq_schema = '',]
[rabbitmq_num_consumers = N,]
[rabbitmq_num_queues = N,]
[rabbitmq_transactional_channel = 0]
[rabbitmq_queue_base = 'queue',]
[rabbitmq_deadletter_exchange = 'dl-exchange',]
[rabbitmq_persistent = 0,]
[rabbitmq_skip_broken_messages = N,]
[rabbitmq_max_block_size = N,]
[rabbitmq_flush_interval_ms = N]
```
Required parameters:
@ -40,12 +46,18 @@ Required parameters:
Optional parameters:
- `rabbitmq_exchange_type` The type of RabbitMQ exchange: `direct`, `fanout`, `topic`, `headers`, `consistent-hash`. Default: `fanout`.
- `rabbitmq_exchange_type` The type of RabbitMQ exchange: `direct`, `fanout`, `topic`, `headers`, `consistent_hash`. Default: `fanout`.
- `rabbitmq_routing_key_list` A comma-separated list of routing keys.
- `rabbitmq_row_delimiter` Delimiter character, which ends the message.
- `rabbitmq_schema` Parameter that must be used if the format requires a schema definition. For example, [Capn Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object.
- `rabbitmq_num_consumers` The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient.
- `rabbitmq_num_queues` The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient. A single queue can contain up to 50K messages at the same time.
- `rabbitmq_transactional_channel` Wrap `INSERT` queries in transactions. Default: `0`.
- `rabbitmq_num_queues` The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient.
- `rabbitmq_queue_base` - Specify a base name for queues that will be declared. By default, queues are declared unique to tables based on db and table names.
- `rabbitmq_deadletter_exchange` - Specify name for a [dead letter exchange](https://www.rabbitmq.com/dlx.html). You can create another table with this exchange name and collect messages in cases when they are republished to dead letter exchange. By default dead letter exchange is not specified.
- `persistent` - If set to 1 (true), in insert query delivery mode will be set to 2 (marks messages as 'persistent'). Default: `0`.
- `rabbitmq_skip_broken_messages` RabbitMQ message parser tolerance to schema-incompatible messages per block. Default: `0`. If `rabbitmq_skip_broken_messages = N` then the engine skips *N* RabbitMQ messages that cannot be parsed (a message equals a row of data).
- `rabbitmq_max_block_size`
- `rabbitmq_flush_interval_ms`
Required configuration:
@ -92,13 +104,22 @@ Exchange type options:
- `headers` - Routing is based on `key=value` matches with a setting `x-match=all` or `x-match=any`. Example table key list: `x-match=all,format=logs,type=report,year=2020`.
- `consistent-hash` - Data is evenly distributed between all bound tables (where the exchange name is the same). Note that this exchange type must be enabled with RabbitMQ plugin: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`.
If exchange type is not specified, then default is `fanout` and routing keys for data publishing must be randomized in range `[1, num_consumers]` for every message/batch (or in range `[1, num_consumers * num_queues]` if `rabbitmq_num_queues` is set). This table configuration works quicker than any other, especially when `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` parameters are set.
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 be able 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 be able 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. Note: it makes sence only if messages are sent with delivery mode 2 - marked 'persistent', durable.
- to reuse queues as they are declared durable and not auto-deleted.
If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` parameters are specified along with `rabbitmq_exchange_type`, then:
To improve performance, received messages are grouped into blocks the size of [max\_insert\_block\_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). If the block wasnt formed within [stream\_flush\_interval\_ms](../../../operations/server-configuration-parameters/settings.md) milliseconds, the data will be flushed to the table regardless of the completeness of the block.
If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings are specified along with `rabbitmq_exchange_type`, then:
- `rabbitmq-consistent-hash-exchange` plugin must be enabled.
- `message_id` property of the published messages must be specified (unique for each message/batch).
For insert query there is message metadata, which is added for each published message: `messageID` and `republished` flag (true, if published more than once) - can be accessed via message headers.
Do not use the same table for inserts and materialized views.
Example:
``` sql
@ -113,10 +134,18 @@ Example:
rabbitmq_num_consumers = 5;
CREATE TABLE daily (key UInt64, value UInt64)
ENGINE = MergeTree();
ENGINE = MergeTree() ORDER BY key;
CREATE MATERIALIZED VIEW consumer TO daily
AS SELECT key, value FROM queue;
SELECT key, value FROM daily ORDER BY key;
```
## Virtual Columns {#virtual-columns}
- `_exchange_name` - RabbitMQ exchange name.
- `_channel_id` - ChannelID, on which consumer, who received the message, was declared.
- `_delivery_tag` - DeliveryTag of the received message. Scoped per channel.
- `_redelivered` - `redelivered` flag of the message.
- `_message_id` - MessageID of the received message; non-empty if was set, when message was published.

View File

@ -62,6 +62,7 @@ Management queries:
- [ALTER USER](../sql-reference/statements/alter/user.md#alter-user-statement)
- [DROP USER](../sql-reference/statements/drop.md)
- [SHOW CREATE USER](../sql-reference/statements/show.md#show-create-user-statement)
- [SHOW USERS](../sql-reference/statements/show.md#show-users-statement)
### Settings Applying {#access-control-settings-applying}
@ -90,6 +91,7 @@ Management queries:
- [SET ROLE](../sql-reference/statements/set-role.md)
- [SET DEFAULT ROLE](../sql-reference/statements/set-role.md#set-default-role-statement)
- [SHOW CREATE ROLE](../sql-reference/statements/show.md#show-create-role-statement)
- [SHOW ROLES](../sql-reference/statements/show.md#show-roles-statement)
Privileges can be granted to a role by the [GRANT](../sql-reference/statements/grant.md) query. To revoke privileges from a role ClickHouse provides the [REVOKE](../sql-reference/statements/revoke.md) query.
@ -103,6 +105,7 @@ Management queries:
- [ALTER ROW POLICY](../sql-reference/statements/alter/row-policy.md#alter-row-policy-statement)
- [DROP ROW POLICY](../sql-reference/statements/drop.md#drop-row-policy-statement)
- [SHOW CREATE ROW POLICY](../sql-reference/statements/show.md#show-create-row-policy-statement)
- [SHOW POLICIES](../sql-reference/statements/show.md#show-policies-statement)
## Settings Profile {#settings-profiles-management}
@ -114,6 +117,7 @@ Management queries:
- [ALTER SETTINGS PROFILE](../sql-reference/statements/alter/settings-profile.md#alter-settings-profile-statement)
- [DROP SETTINGS PROFILE](../sql-reference/statements/drop.md#drop-settings-profile-statement)
- [SHOW CREATE SETTINGS PROFILE](../sql-reference/statements/show.md#show-create-settings-profile-statement)
- [SHOW PROFILES](../sql-reference/statements/show.md#show-profiles-statement)
## Quota {#quotas-management}
@ -127,6 +131,8 @@ Management queries:
- [ALTER QUOTA](../sql-reference/statements/alter/quota.md#alter-quota-statement)
- [DROP QUOTA](../sql-reference/statements/drop.md#drop-quota-statement)
- [SHOW CREATE QUOTA](../sql-reference/statements/show.md#show-create-quota-statement)
- [SHOW QUOTA](../sql-reference/statements/show.md#show-quota-statement)
- [SHOW QUOTAS](../sql-reference/statements/show.md#show-quotas-statement)
## Enabling SQL-driven Access Control and Account Management {#enabling-access-control}

View File

@ -34,6 +34,7 @@ Columns:
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Query starting date.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query starting time.
- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution.
- `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution in milliseconds.
- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Total number or rows read from all tables and table functions participated in query. It includes usual subqueries, subqueries for `IN` and `JOIN`. For distributed queries `read_rows` includes the total number of rows read at all replicas. Each replica sends its `read_rows` value, and the server-initiator of the query summarize all received and local values. The cache volumes doesnt affect this value.
- `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Total number or bytes read from all tables and table functions participated in query. It includes usual subqueries, subqueries for `IN` and `JOIN`. For distributed queries `read_bytes` includes the total number of rows read at all replicas. Each replica sends its `read_bytes` value, and the server-initiator of the query summarize all received and local values. The cache volumes doesnt affect this value.

View File

@ -16,6 +16,7 @@ Columns:
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the thread has finished execution of the query.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query.
- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution.
- `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution.
- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of read rows.
- `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of read bytes.

View File

@ -23,4 +23,8 @@ Columns:
- `execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — The total query execution time, in seconds (wall time).
- `max_execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — Maximum of query execution time.
## See Also {#see-also}
- [SHOW QUOTA](../../sql-reference/statements/show.md#show-quota-statement)
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/quota_usage) <!--hide-->

View File

@ -20,5 +20,9 @@ Columns:
- `apply_to_list` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — List of user names/[roles](../../operations/access-rights.md#role-management) that the quota should be applied to.
- `apply_to_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — List of user names/roles that the quota should not apply to.
## See Also {#see-also}
- [SHOW QUOTAS](../../sql-reference/statements/show.md#show-quotas-statement)
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/quotas) <!--hide-->

View File

@ -24,4 +24,8 @@ Columns:
- `execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — The total query execution time, in seconds (wall time).
- `max_execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — Maximum of query execution time.
## See Also {#see-also}
- [SHOW QUOTA](../../sql-reference/statements/show.md#show-quota-statement)
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/quotas_usage) <!--hide-->

View File

@ -5,11 +5,15 @@ Contains the role grants for users and roles. To add entries to this table, use
Columns:
- `user_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — User name.
- `role_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — Role name.
- `granted_role_name` ([String](../../sql-reference/data-types/string.md)) — Name of role granted to the `role_name` role. To grant one role to another one use `GRANT role1 TO role2`.
- `granted_role_is_default` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Flag that shows whether `granted_role` is a default role. Possible values:
- 1 — `granted_role` is a default role.
- 0 — `granted_role` is not a default role.
- `with_admin_option` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Flag that shows whether `granted_role` is a role with [ADMIN OPTION](../../sql-reference/statements/grant.md#admin-option-privilege) privilege. Possible values:
- 1 — The role has `ADMIN OPTION` privilege.
- 0 — The role without `ADMIN OPTION` privilege.

View File

@ -8,4 +8,8 @@ Columns:
- `id` ([UUID](../../sql-reference/data-types/uuid.md)) — Role ID.
- `storage` ([String](../../sql-reference/data-types/string.md)) — Path to the storage of roles. Configured in the `access_control_path` parameter.
## See Also {#see-also}
- [SHOW ROLES](../../sql-reference/statements/show.md#show-roles-statement)
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/roles) <!--hide-->

View File

@ -27,4 +27,8 @@ Columns:
- `apply_to_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — The row policies is applied to all roles and/or users excepting of the listed ones.
## See Also {#see-also}
- [SHOW POLICIES](../../sql-reference/statements/show.md#show-policies-statement)
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/row_policies) <!--hide-->

View File

@ -17,4 +17,8 @@ Columns:
- `apply_to_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — The setting profile is applied to all roles and/or users excepting of the listed ones.
## See Also {#see-also}
- [SHOW PROFILES](../../sql-reference/statements/show.md#show-profiles-statement)
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/settings_profiles) <!--hide-->

View File

@ -82,8 +82,8 @@ res: /lib/x86_64-linux-gnu/libc-2.27.so
- [Introspection Functions](../../sql-reference/functions/introspection.md) — Which introspection functions are available and how to use them.
- [system.trace_log](../system-tables/trace_log.md) — Contains stack traces collected by the sampling query profiler.
- [arrayMap](../../sql-reference/functions/higher-order-functions.md#higher_order_functions-array-map) — Description and usage example of the `arrayMap` function.
- [arrayFilter](../../sql-reference/functions/higher-order-functions.md#higher_order_functions-array-filter) — Description and usage example of the `arrayFilter` function.
- [arrayMap](../../sql-reference/functions/array-functions.md#array-map) — Description and usage example of the `arrayMap` function.
- [arrayFilter](../../sql-reference/functions/array-functions.md#array-filter) — Description and usage example of the `arrayFilter` function.
[Original article](https://clickhouse.tech/docs/en/operations/system-tables/stack_trace) <!--hide-->

View File

@ -27,4 +27,8 @@ Columns:
- `default_roles_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — All the granted roles set as default excepting of the listed ones.
## See Also {#see-also}
- [SHOW USERS](../../sql-reference/statements/show.md#show-users-statement)
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/users) <!--hide-->

View File

@ -7,7 +7,7 @@ toc_title: Tuple(T1, T2, ...)
A tuple of elements, each having an individual [type](../../sql-reference/data-types/index.md#data_types).
Tuples are used for temporary column grouping. Columns can be grouped when an IN expression is used in a query, and for specifying certain formal parameters of lambda functions. For more information, see the sections [IN operators](../../sql-reference/operators/in.md) and [Higher order functions](../../sql-reference/functions/higher-order-functions.md).
Tuples are used for temporary column grouping. Columns can be grouped when an IN expression is used in a query, and for specifying certain formal parameters of lambda functions. For more information, see the sections [IN operators](../../sql-reference/operators/in.md) and [Higher order functions](../../sql-reference/functions/index.md#higher-order-functions).
Tuples can be the result of a query. In this case, for text formats other than JSON, values are comma-separated in brackets. In JSON formats, tuples are output as arrays (in square brackets).

View File

@ -1,5 +1,5 @@
---
toc_priority: 35
toc_priority: 34
toc_title: Arithmetic
---

View File

@ -1,9 +1,9 @@
---
toc_priority: 46
toc_priority: 35
toc_title: Arrays
---
# Functions for Working with Arrays {#functions-for-working-with-arrays}
# Array Functions {#functions-for-working-with-arrays}
## empty {#function-empty}
@ -241,6 +241,12 @@ SELECT indexOf([1, 3, NULL, NULL], NULL)
Elements set to `NULL` are handled as normal values.
## arrayCount(\[func,\] arr1, …) {#array-count}
Returns the number of elements in the arr array for which func returns something other than 0. If func is not specified, it returns the number of non-zero elements in the array.
Note that the `arrayCount` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument.
## countEqual(arr, x) {#countequalarr-x}
Returns the number of elements in the array equal to x. Equivalent to arrayCount (elem -\> elem = x, arr).
@ -568,7 +574,7 @@ SELECT arraySort([1, nan, 2, NULL, 3, nan, -4, NULL, inf, -inf]);
- `NaN` values are right before `NULL`.
- `Inf` values are right before `NaN`.
Note that `arraySort` is a [higher-order function](../../sql-reference/functions/higher-order-functions.md). You can pass a lambda function to it as the first argument. In this case, sorting order is determined by the result of the lambda function applied to the elements of the array.
Note that `arraySort` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. In this case, sorting order is determined by the result of the lambda function applied to the elements of the array.
Lets consider the following example:
@ -668,7 +674,7 @@ SELECT arrayReverseSort([1, nan, 2, NULL, 3, nan, -4, NULL, inf, -inf]) as res;
- `NaN` values are right before `NULL`.
- `-Inf` values are right before `NaN`.
Note that the `arrayReverseSort` is a [higher-order function](../../sql-reference/functions/higher-order-functions.md). You can pass a lambda function to it as the first argument. Example is shown below.
Note that the `arrayReverseSort` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. Example is shown below.
``` sql
SELECT arrayReverseSort((x) -> -x, [1, 2, 3]) as res;
@ -1120,7 +1126,205 @@ Result:
``` text
┌─arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐
│ 0.75 │
└────────────────────────────────────────---──┘
└───────────────────────────────────────────────┘
```
## arrayMap(func, arr1, …) {#array-map}
Returns an array obtained from the original application of the `func` function to each element in the `arr` array.
Examples:
``` sql
SELECT arrayMap(x -> (x + 2), [1, 2, 3]) as res;
```
``` text
┌─res─────┐
│ [3,4,5] │
└─────────┘
```
The following example shows how to create a tuple of elements from different arrays:
``` sql
SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) AS res
```
``` text
┌─res─────────────────┐
│ [(1,4),(2,5),(3,6)] │
└─────────────────────┘
```
Note that the `arrayMap` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it cant be omitted.
## arrayFilter(func, arr1, …) {#array-filter}
Returns an array containing only the elements in `arr1` for which `func` returns something other than 0.
Examples:
``` sql
SELECT arrayFilter(x -> x LIKE '%World%', ['Hello', 'abc World']) AS res
```
``` text
┌─res───────────┐
│ ['abc World'] │
└───────────────┘
```
``` sql
SELECT
arrayFilter(
(i, x) -> x LIKE '%World%',
arrayEnumerate(arr),
['Hello', 'abc World'] AS arr)
AS res
```
``` text
┌─res─┐
│ [2] │
└─────┘
```
Note that the `arrayFilter` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it cant be omitted.
## arrayFill(func, arr1, …) {#array-fill}
Scan through `arr1` from the first element to the last element and replace `arr1[i]` by `arr1[i - 1]` if `func` returns 0. The first element of `arr1` will not be replaced.
Examples:
``` sql
SELECT arrayFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 6, 14, null, null]) AS res
```
``` text
┌─res──────────────────────────────┐
│ [1,1,3,11,12,12,12,5,6,14,14,14] │
└──────────────────────────────────┘
```
Note that the `arrayFill` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it cant be omitted.
## arrayReverseFill(func, arr1, …) {#array-reverse-fill}
Scan through `arr1` from the last element to the first element and replace `arr1[i]` by `arr1[i + 1]` if `func` returns 0. The last element of `arr1` will not be replaced.
Examples:
``` sql
SELECT arrayReverseFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 6, 14, null, null]) AS res
```
``` text
┌─res────────────────────────────────┐
│ [1,3,3,11,12,5,5,5,6,14,NULL,NULL] │
└────────────────────────────────────┘
```
Note that the `arrayReverseFilter` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it cant be omitted.
## arraySplit(func, arr1, …) {#array-split}
Split `arr1` into multiple arrays. When `func` returns something other than 0, the array will be split on the left hand side of the element. The array will not be split before the first element.
Examples:
``` sql
SELECT arraySplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res
```
``` text
┌─res─────────────┐
│ [[1,2,3],[4,5]] │
└─────────────────┘
```
Note that the `arraySplit` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it cant be omitted.
## arrayReverseSplit(func, arr1, …) {#array-reverse-split}
Split `arr1` into multiple arrays. When `func` returns something other than 0, the array will be split on the right hand side of the element. The array will not be split after the last element.
Examples:
``` sql
SELECT arrayReverseSplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res
```
``` text
┌─res───────────────┐
│ [[1],[2,3,4],[5]] │
└───────────────────┘
```
Note that the `arrayReverseSplit` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it cant be omitted.
## arrayExists(\[func,\] arr1, …) {#arrayexistsfunc-arr1}
Returns 1 if there is at least one element in `arr` for which `func` returns something other than 0. Otherwise, it returns 0.
Note that the `arrayExists` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument.
## arrayAll(\[func,\] arr1, …) {#arrayallfunc-arr1}
Returns 1 if `func` returns something other than 0 for all the elements in `arr`. Otherwise, it returns 0.
Note that the `arrayAll` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument.
## arrayFirst(func, arr1, …) {#array-first}
Returns the first element in the `arr1` array for which `func` returns something other than 0.
Note that the `arrayFirst` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it cant be omitted.
## arrayFirstIndex(func, arr1, …) {#array-first-index}
Returns the index of the first element in the `arr1` array for which `func` returns something other than 0.
Note that the `arrayFirstIndex` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it cant be omitted.
## arraySum(\[func,\] arr1, …) {#array-sum}
Returns the sum of the `func` values. If the function is omitted, it just returns the sum of the array elements.
Note that the `arraySum` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument.
## arrayCumSum(\[func,\] arr1, …) {#arraycumsumfunc-arr1}
Returns an array of partial sums of elements in the source array (a running sum). If the `func` function is specified, then the values of the array elements are converted by this function before summing.
Example:
``` sql
SELECT arrayCumSum([1, 1, 1, 1]) AS res
```
``` text
┌─res──────────┐
│ [1, 2, 3, 4] │
└──────────────┘
```
Note that the `arrayCumSum` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument.
## arrayCumSumNonNegative(arr) {#arraycumsumnonnegativearr}
Same as `arrayCumSum`, returns an array of partial sums of elements in the source array (a running sum). Different `arrayCumSum`, when then returned value contains a value less than zero, the value is replace with zero and the subsequent calculation is performed with zero parameters. For example:
``` sql
SELECT arrayCumSumNonNegative([1, 1, -4, 1]) AS res
```
``` text
┌─res───────┐
│ [1,2,0,1] │
└───────────┘
```
Note that the `arraySumNonNegative` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument.
[Original article](https://clickhouse.tech/docs/en/query_language/functions/array_functions/) <!--hide-->

View File

@ -1,262 +0,0 @@
---
toc_priority: 57
toc_title: Higher-Order
---
# Higher-order Functions {#higher-order-functions}
## `->` operator, lambda(params, expr) function {#operator-lambdaparams-expr-function}
Allows describing a lambda function for passing to a higher-order function. The left side of the arrow has a formal parameter, which is any ID, or multiple formal parameters any IDs in a tuple. The right side of the arrow has an expression that can use these formal parameters, as well as any table columns.
Examples: `x -> 2 * x, str -> str != Referer.`
Higher-order functions can only accept lambda functions as their functional argument.
A lambda function that accepts multiple arguments can be passed to a higher-order function. In this case, the higher-order function is passed several arrays of identical length that these arguments will correspond to.
For some functions, such as [arrayCount](#higher_order_functions-array-count) or [arraySum](#higher_order_functions-array-count), the first argument (the lambda function) can be omitted. In this case, identical mapping is assumed.
A lambda function cant be omitted for the following functions:
- [arrayMap](#higher_order_functions-array-map)
- [arrayFilter](#higher_order_functions-array-filter)
- [arrayFill](#higher_order_functions-array-fill)
- [arrayReverseFill](#higher_order_functions-array-reverse-fill)
- [arraySplit](#higher_order_functions-array-split)
- [arrayReverseSplit](#higher_order_functions-array-reverse-split)
- [arrayFirst](#higher_order_functions-array-first)
- [arrayFirstIndex](#higher_order_functions-array-first-index)
### arrayMap(func, arr1, …) {#higher_order_functions-array-map}
Returns an array obtained from the original application of the `func` function to each element in the `arr` array.
Examples:
``` sql
SELECT arrayMap(x -> (x + 2), [1, 2, 3]) as res;
```
``` text
┌─res─────┐
│ [3,4,5] │
└─────────┘
```
The following example shows how to create a tuple of elements from different arrays:
``` sql
SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) AS res
```
``` text
┌─res─────────────────┐
│ [(1,4),(2,5),(3,6)] │
└─────────────────────┘
```
Note that the first argument (lambda function) cant be omitted in the `arrayMap` function.
### arrayFilter(func, arr1, …) {#higher_order_functions-array-filter}
Returns an array containing only the elements in `arr1` for which `func` returns something other than 0.
Examples:
``` sql
SELECT arrayFilter(x -> x LIKE '%World%', ['Hello', 'abc World']) AS res
```
``` text
┌─res───────────┐
│ ['abc World'] │
└───────────────┘
```
``` sql
SELECT
arrayFilter(
(i, x) -> x LIKE '%World%',
arrayEnumerate(arr),
['Hello', 'abc World'] AS arr)
AS res
```
``` text
┌─res─┐
│ [2] │
└─────┘
```
Note that the first argument (lambda function) cant be omitted in the `arrayFilter` function.
### arrayFill(func, arr1, …) {#higher_order_functions-array-fill}
Scan through `arr1` from the first element to the last element and replace `arr1[i]` by `arr1[i - 1]` if `func` returns 0. The first element of `arr1` will not be replaced.
Examples:
``` sql
SELECT arrayFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 6, 14, null, null]) AS res
```
``` text
┌─res──────────────────────────────┐
│ [1,1,3,11,12,12,12,5,6,14,14,14] │
└──────────────────────────────────┘
```
Note that the first argument (lambda function) cant be omitted in the `arrayFill` function.
### arrayReverseFill(func, arr1, …) {#higher_order_functions-array-reverse-fill}
Scan through `arr1` from the last element to the first element and replace `arr1[i]` by `arr1[i + 1]` if `func` returns 0. The last element of `arr1` will not be replaced.
Examples:
``` sql
SELECT arrayReverseFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 6, 14, null, null]) AS res
```
``` text
┌─res────────────────────────────────┐
│ [1,3,3,11,12,5,5,5,6,14,NULL,NULL] │
└────────────────────────────────────┘
```
Note that the first argument (lambda function) cant be omitted in the `arrayReverseFill` function.
### arraySplit(func, arr1, …) {#higher_order_functions-array-split}
Split `arr1` into multiple arrays. When `func` returns something other than 0, the array will be split on the left hand side of the element. The array will not be split before the first element.
Examples:
``` sql
SELECT arraySplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res
```
``` text
┌─res─────────────┐
│ [[1,2,3],[4,5]] │
└─────────────────┘
```
Note that the first argument (lambda function) cant be omitted in the `arraySplit` function.
### arrayReverseSplit(func, arr1, …) {#higher_order_functions-array-reverse-split}
Split `arr1` into multiple arrays. When `func` returns something other than 0, the array will be split on the right hand side of the element. The array will not be split after the last element.
Examples:
``` sql
SELECT arrayReverseSplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res
```
``` text
┌─res───────────────┐
│ [[1],[2,3,4],[5]] │
└───────────────────┘
```
Note that the first argument (lambda function) cant be omitted in the `arraySplit` function.
### arrayCount(\[func,\] arr1, …) {#higher_order_functions-array-count}
Returns the number of elements in the arr array for which func returns something other than 0. If func is not specified, it returns the number of non-zero elements in the array.
### arrayExists(\[func,\] arr1, …) {#arrayexistsfunc-arr1}
Returns 1 if there is at least one element in arr for which func returns something other than 0. Otherwise, it returns 0.
### arrayAll(\[func,\] arr1, …) {#arrayallfunc-arr1}
Returns 1 if func returns something other than 0 for all the elements in arr. Otherwise, it returns 0.
### arraySum(\[func,\] arr1, …) {#higher-order-functions-array-sum}
Returns the sum of the func values. If the function is omitted, it just returns the sum of the array elements.
### arrayFirst(func, arr1, …) {#higher_order_functions-array-first}
Returns the first element in the arr1 array for which func returns something other than 0.
Note that the first argument (lambda function) cant be omitted in the `arrayFirst` function.
### arrayFirstIndex(func, arr1, …) {#higher_order_functions-array-first-index}
Returns the index of the first element in the arr1 array for which func returns something other than 0.
Note that the first argument (lambda function) cant be omitted in the `arrayFirstIndex` function.
### arrayCumSum(\[func,\] arr1, …) {#arraycumsumfunc-arr1}
Returns an array of partial sums of elements in the source array (a running sum). If the `func` function is specified, then the values of the array elements are converted by this function before summing.
Example:
``` sql
SELECT arrayCumSum([1, 1, 1, 1]) AS res
```
``` text
┌─res──────────┐
│ [1, 2, 3, 4] │
└──────────────┘
```
### arrayCumSumNonNegative(arr) {#arraycumsumnonnegativearr}
Same as `arrayCumSum`, returns an array of partial sums of elements in the source array (a running sum). Different `arrayCumSum`, when then returned value contains a value less than zero, the value is replace with zero and the subsequent calculation is performed with zero parameters. For example:
``` sql
SELECT arrayCumSumNonNegative([1, 1, -4, 1]) AS res
```
``` text
┌─res───────┐
│ [1,2,0,1] │
└───────────┘
```
### arraySort(\[func,\] arr1, …) {#arraysortfunc-arr1}
Returns an array as result of sorting the elements of `arr1` in ascending order. If the `func` function is specified, sorting order is determined by the result of the function `func` applied to the elements of array (arrays)
The [Schwartzian transform](https://en.wikipedia.org/wiki/Schwartzian_transform) is used to improve sorting efficiency.
Example:
``` sql
SELECT arraySort((x, y) -> y, ['hello', 'world'], [2, 1]);
```
``` text
┌─res────────────────┐
│ ['world', 'hello'] │
└────────────────────┘
```
For more information about the `arraySort` method, see the [Functions for Working With Arrays](../../sql-reference/functions/array-functions.md#array_functions-sort) section.
### arrayReverseSort(\[func,\] arr1, …) {#arrayreversesortfunc-arr1}
Returns an array as result of sorting the elements of `arr1` in descending order. If the `func` function is specified, sorting order is determined by the result of the function `func` applied to the elements of array (arrays).
Example:
``` sql
SELECT arrayReverseSort((x, y) -> y, ['hello', 'world'], [2, 1]) as res;
```
``` text
┌─res───────────────┐
│ ['hello','world'] │
└───────────────────┘
```
For more information about the `arrayReverseSort` method, see the [Functions for Working With Arrays](../../sql-reference/functions/array-functions.md#array_functions-reverse-sort) section.
[Original article](https://clickhouse.tech/docs/en/query_language/functions/higher_order_functions/) <!--hide-->

View File

@ -44,6 +44,21 @@ Functions have the following behaviors:
Functions cant change the values of their arguments any changes are returned as the result. Thus, the result of calculating separate functions does not depend on the order in which the functions are written in the query.
## Higher-order functions, `->` operator and lambda(params, expr) function {#higher-order-functions}
Higher-order functions can only accept lambda functions as their functional argument. To pass a lambda function to a higher-order function use `->` operator. The left side of the arrow has a formal parameter, which is any ID, or multiple formal parameters any IDs in a tuple. The right side of the arrow has an expression that can use these formal parameters, as well as any table columns.
Examples:
```
x -> 2 * x
str -> str != Referer
```
A lambda function that accepts multiple arguments can also be passed to a higher-order function. In this case, the higher-order function is passed several arrays of identical length that these arguments will correspond to.
For some functions the first argument (the lambda function) can be omitted. In this case, identical mapping is assumed.
## Error Handling {#error-handling}
Some functions might throw an exception if the data is invalid. In this case, the query is canceled and an error text is returned to the client. For distributed processing, when an exception occurs on one of the servers, the other servers also attempt to abort the query.

View File

@ -98,7 +98,7 @@ LIMIT 1
\G
```
The [arrayMap](../../sql-reference/functions/higher-order-functions.md#higher_order_functions-array-map) function allows to process each individual element of the `trace` array by the `addressToLine` function. The result of this processing you see in the `trace_source_code_lines` column of output.
The [arrayMap](../../sql-reference/functions/array-functions.md#array-map) function allows to process each individual element of the `trace` array by the `addressToLine` function. The result of this processing you see in the `trace_source_code_lines` column of output.
``` text
Row 1:
@ -184,7 +184,7 @@ LIMIT 1
\G
```
The [arrayMap](../../sql-reference/functions/higher-order-functions.md#higher_order_functions-array-map) function allows to process each individual element of the `trace` array by the `addressToSymbols` function. The result of this processing you see in the `trace_symbols` column of output.
The [arrayMap](../../sql-reference/functions/array-functions.md#array-map) function allows to process each individual element of the `trace` array by the `addressToSymbols` function. The result of this processing you see in the `trace_symbols` column of output.
``` text
Row 1:
@ -281,7 +281,7 @@ LIMIT 1
\G
```
The [arrayMap](../../sql-reference/functions/higher-order-functions.md#higher_order_functions-array-map) function allows to process each individual element of the `trace` array by the `demangle` function. The result of this processing you see in the `trace_functions` column of output.
The [arrayMap](../../sql-reference/functions/array-functions.md#array-map) function allows to process each individual element of the `trace` array by the `demangle` function. The result of this processing you see in the `trace_functions` column of output.
``` text
Row 1:

View File

@ -10,7 +10,7 @@ Changes settings profiles.
Syntax:
``` sql
ALTER SETTINGS PROFILE [IF EXISTS] name [ON CLUSTER cluster_name]
ALTER SETTINGS PROFILE [IF EXISTS] TO name [ON CLUSTER cluster_name]
[RENAME TO new_name]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | INHERIT 'profile_name'] [,...]
```

View File

@ -10,7 +10,7 @@ Creates a [settings profile](../../../operations/access-rights.md#settings-profi
Syntax:
``` sql
CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name]
CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] TO name [ON CLUSTER cluster_name]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | INHERIT 'profile_name'] [,...]
```

View File

@ -148,7 +148,7 @@ SHOW CREATE [ROW] POLICY name ON [database.]table
Shows parameters that were used at a [quota creation](../../sql-reference/statements/create/quota.md).
### Syntax {#show-create-row-policy-syntax}
### Syntax {#show-create-quota-syntax}
``` sql
SHOW CREATE QUOTA [name | CURRENT]
@ -158,10 +158,70 @@ SHOW CREATE QUOTA [name | CURRENT]
Shows parameters that were used at a [settings profile creation](../../sql-reference/statements/create/settings-profile.md).
### Syntax {#show-create-row-policy-syntax}
### Syntax {#show-create-settings-profile-syntax}
``` sql
SHOW CREATE [SETTINGS] PROFILE name
```
## SHOW USERS {#show-users-statement}
Returns a list of [user account](../../operations/access-rights.md#user-account-management) names. To view user accounts parameters, see the system table [system.users](../../operations/system-tables/users.md#system_tables-users).
### Syntax {#show-users-syntax}
``` sql
SHOW USERS
```
## SHOW ROLES {#show-roles-statement}
Returns a list of [roles](../../operations/access-rights.md#role-management). To view another parameters, see system tables [system.roles](../../operations/system-tables/roles.md#system_tables-roles) and [system.role-grants](../../operations/system-tables/role-grants.md#system_tables-role_grants).
### Syntax {#show-roles-syntax}
``` sql
SHOW [CURRENT|ENABLED] ROLES
```
## SHOW PROFILES {#show-profiles-statement}
Returns a list of [setting profiles](../../operations/access-rights.md#settings-profiles-management). To view user accounts parameters, see the system table [settings_profiles](../../operations/system-tables/settings_profiles.md#system_tables-settings_profiles).
### Syntax {#show-profiles-syntax}
``` sql
SHOW [SETTINGS] PROFILES
```
## SHOW POLICIES {#show-policies-statement}
Returns a list of [row policies](../../operations/access-rights.md#row-policy-management) for the specified table. To view user accounts parameters, see the system table [system.row_policies](../../operations/system-tables/row_policies.md#system_tables-row_policies).
### Syntax {#show-policies-syntax}
``` sql
SHOW [ROW] POLICIES [ON [db.]table]
```
## SHOW QUOTAS {#show-quotas-statement}
Returns a list of [quotas](../../operations/access-rights.md#quotas-management). To view quotas parameters, see the system table [system.quotas](../../operations/system-tables/quotas.md#system_tables-quotas).
### Syntax {#show-quotas-syntax}
``` sql
SHOW QUOTAS
```
## SHOW QUOTA {#show-quota-statement}
Returns a [quota](../../operations/quotas.md) consumption for all users or for current user. To view another parameters, see system tables [system.quotas_usage](../../operations/system-tables/quotas_usage.md#system_tables-quotas_usage) and [system.quota_usage](../../operations/system-tables/quota_usage.md#system_tables-quota_usage).
### Syntax {#show-quota-syntax}
``` sql
SHOW [CURRENT] QUOTA
```
[Original article](https://clickhouse.tech/docs/en/query_language/show/) <!--hide-->

View File

@ -1,3 +1,8 @@
---
toc_priority: 30
toc_title: MergeTree
---
# MergeTree {#table_engines-mergetree}
Движок `MergeTree`, а также другие движки этого семейства (`*MergeTree`) — это наиболее функциональные движки таблиц ClickHouse.
@ -28,8 +33,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1,
INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2
) ENGINE = MergeTree()
ORDER BY expr
[PARTITION BY expr]
[ORDER BY expr]
[PRIMARY KEY expr]
[SAMPLE BY expr]
[TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'], ...]
@ -38,27 +43,42 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
Описание параметров смотрите в [описании запроса CREATE](../../../engines/table-engines/mergetree-family/mergetree.md).
!!! note "Note"
!!! note "Примечание"
`INDEX` — экспериментальная возможность, смотрите [Индексы пропуска данных](#table_engine-mergetree-data_skipping-indexes).
### Секции запроса {#mergetree-query-clauses}
- `ENGINE` — имя и параметры движка. `ENGINE = MergeTree()`. `MergeTree` не имеет параметров.
- `PARTITION BY` — [ключ партиционирования](custom-partitioning-key.md). Для партиционирования по месяцам используйте выражение `toYYYYMM(date_column)`, где `date_column` — столбец с датой типа [Date](../../../engines/table-engines/mergetree-family/mergetree.md). В этом случае имена партиций имеют формат `"YYYYMM"`.
- `ORDER BY` — ключ сортировки.
- `ORDER BY` — ключ сортировки. Кортеж столбцов или произвольных выражений. Пример: `ORDER BY (CounterID, EventDate)`.
Кортеж столбцов или произвольных выражений. Пример: `ORDER BY (CounterID, EventDate)`.
- `PRIMARY KEY` — первичный ключ, если он [отличается от ключа сортировки](#pervichnyi-kliuch-otlichnyi-ot-kliucha-sortirovki). По умолчанию первичный ключ совпадает с ключом сортировки (который задаётся секцией `ORDER BY`.) Поэтому в большинстве случаев секцию `PRIMARY KEY` отдельно указывать не нужно.
ClickHouse использует ключ сортировки в качестве первичного ключа, если первичный ключ не задан в секции `PRIMARY KEY`.
- `SAMPLE BY` — выражение для сэмплирования. Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Пример: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`.
Чтобы отключить сортировку, используйте синтаксис `ORDER BY tuple()`. Смотрите [выбор первичного ключа](#vybor-pervichnogo-kliucha).
- `TTL` — список правил, определяющих длительности хранения строк, а также задающих правила перемещения частей на определённые тома или диски. Выражение должно возвращать столбец `Date` или `DateTime`. Пример: `TTL date + INTERVAL 1 DAY`.
- Тип правила `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'` указывает действие, которое будет выполнено с частью, удаление строк (прореживание), перемещение (при выполнении условия для всех строк части) на определённый диск (`TO DISK 'xxx'`) или том (`TO VOLUME 'xxx'`).
- Поведение по умолчанию соответствует удалению строк (`DELETE`). В списке правил может быть указано только одно выражение с поведением `DELETE`.
- Дополнительные сведения смотрите в разделе [TTL для столбцов и таблиц](#table_engine-mergetree-ttl)
- `PARTITION BY` — [ключ партиционирования](custom-partitioning-key.md). Необязательный параметр.
- `SETTINGS` — дополнительные параметры, регулирующие поведение `MergeTree`:
Для партиционирования по месяцам используйте выражение `toYYYYMM(date_column)`, где `date_column` — столбец с датой типа [Date](../../../engines/table-engines/mergetree-family/mergetree.md). В этом случае имена партиций имеют формат `"YYYYMM"`.
- `PRIMARY KEY` — первичный ключ, если он [отличается от ключа сортировки](#pervichnyi-kliuch-otlichnyi-ot-kliucha-sortirovki). Необязательный параметр.
По умолчанию первичный ключ совпадает с ключом сортировки (который задаётся секцией `ORDER BY`.) Поэтому в большинстве случаев секцию `PRIMARY KEY` отдельно указывать не нужно.
- `SAMPLE BY` — выражение для сэмплирования. Необязательный параметр.
Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Пример: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`.
- `TTL` — список правил, определяющих длительности хранения строк, а также задающих правила перемещения частей на определённые тома или диски. Необязательный параметр.
Выражение должно возвращать столбец `Date` или `DateTime`. Пример: `TTL date + INTERVAL 1 DAY`.
Тип правила `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'` указывает действие, которое будет выполнено с частью, удаление строк (прореживание), перемещение (при выполнении условия для всех строк части) на определённый диск (`TO DISK 'xxx'`) или том (`TO VOLUME 'xxx'`). Поведение по умолчанию соответствует удалению строк (`DELETE`). В списке правил может быть указано только одно выражение с поведением `DELETE`.
Дополнительные сведения смотрите в разделе [TTL для столбцов и таблиц](#table_engine-mergetree-ttl)
- `SETTINGS` — дополнительные параметры, регулирующие поведение `MergeTree` (необязательные):
- `index_granularity` — максимальное количество строк данных между засечками индекса. По умолчанию — 8192. Смотрите [Хранение данных](#mergetree-data-storage).
- `index_granularity_bytes` — максимальный размер гранул данных в байтах. По умолчанию — 10Mb. Чтобы ограничить размер гранул только количеством строк, установите значение 0 (не рекомендовано). Смотрите [Хранение данных](#mergetree-data-storage).
@ -180,6 +200,14 @@ ClickHouse не требует уникального первичного кл
Длинный первичный ключ будет негативно влиять на производительность вставки и потребление памяти, однако на производительность ClickHouse при запросах `SELECT` лишние столбцы в первичном ключе не влияют.
Вы можете создать таблицу без первичного ключа, используя синтаксис `ORDER BY tuple()`. В этом случае ClickHouse хранит данные в порядке вставки. Если вы хотите сохранить порядок данных при вставке данных с помощью запросов `INSERT ... SELECT`, установите [max\_insert\_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads).
Чтобы выбрать данные в первоначальном порядке, используйте
[однопоточные](../../../operations/settings/settings.md#settings-max_threads) запросы `SELECT.
### Первичный ключ, отличный от ключа сортировки {#pervichnyi-kliuch-otlichnyi-ot-kliucha-sortirovki}
Существует возможность задать первичный ключ (выражение, значения которого будут записаны в индексный файл для

View File

@ -93,6 +93,10 @@
[cickhouse-plantuml](https://pypi.org/project/clickhouse-plantuml/) — скрипт, генерирующий [PlantUML](https://plantuml.com/) диаграммы схем таблиц.
### xeus-clickhouse {#xeus-clickhouse}
[xeus-clickhouse](https://github.com/wangfenjin/xeus-clickhouse) — это ядро Jupyter для ClickHouse, которое поддерживает запрос ClickHouse-данных с использованием SQL в Jupyter.
## Коммерческие {#kommercheskie}
### DataGrip {#datagrip}

View File

@ -7,10 +7,38 @@ toc_title: Системные таблицы
## Введение {#system-tables-introduction}
Системные таблицы используются для реализации части функциональности системы, а также предоставляют доступ к информации о работе системы.
Вы не можете удалить системную таблицу (хотя можете сделать DETACH).
Для системных таблиц нет файлов с данными на диске и файлов с метаданными. Сервер создаёт все системные таблицы при старте.
В системные таблицы нельзя записывать данные - можно только читать.
Системные таблицы расположены в базе данных system.
Системные таблицы содержат информацию о:
- Состоянии сервера, процессов и окружении.
- Внутренних процессах сервера.
Системные таблицы:
- Находятся в базе данных `system`.
- Доступны только для чтения данных.
- Не могут быть удалены или изменены, но их можно отсоединить.
Системные таблицы `metric_log`, `query_log`, `query_thread_log`, `trace_log` системные таблицы хранят данные в файловой системе. Остальные системные таблицы хранят свои данные в оперативной памяти. Сервер ClickHouse создает такие системные таблицы при запуске.
### Источники системных показателей
Для сбора системных показателей сервер ClickHouse использует:
- Возможности `CAP_NET_ADMIN`.
- [procfs](https://ru.wikipedia.org/wiki/Procfs) (только Linux).
**procfs**
Если для сервера ClickHouse не включено `CAP_NET_ADMIN`, он пытается обратиться к `ProcfsMetricsProvider`. `ProcfsMetricsProvider` позволяет собирать системные показатели для каждого запроса (для CPU и I/O).
Если procfs поддерживается и включена в системе, то сервер ClickHouse собирает следующие системные показатели:
- `OSCPUVirtualTimeMicroseconds`
- `OSCPUWaitMicroseconds`
- `OSIOWaitMicroseconds`
- `OSReadChars`
- `OSWriteChars`
- `OSReadBytes`
- `OSWriteBytes`
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system-tables/) <!--hide-->

View File

@ -24,4 +24,8 @@
- `execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — Общее время выполнения запроса, в секундах.
- `max_execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — Максимальное время выполнения запроса.
## Смотрите также {#see-also}
- [SHOW QUOTA](../../sql-reference/statements/show.md#show-quota-statement)
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/quota_usage) <!--hide-->

View File

@ -21,5 +21,9 @@
- `apply_to_list` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Список имен пользователей/[ролей](../../operations/access-rights.md#role-management) к которым применяется квота.
- `apply_to_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Список имен пользователей/ролей к которым квота применяться не должна.
## Смотрите также {#see-also}
- [SHOW QUOTAS](../../sql-reference/statements/show.md#show-quotas-statement)
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/quotas) <!--hide-->

View File

@ -25,4 +25,8 @@
- `execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — Общее время выполнения запроса, в секундах.
- `max_execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — Максимальное время выполнения запроса.
## Смотрите также {#see-also}
- [SHOW QUOTA](../../sql-reference/statements/show.md#show-quota-statement)
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/quotas_usage) <!--hide-->

View File

@ -5,7 +5,13 @@
Столбцы:
- `name` ([String](../../sql-reference/data-types/string.md)) — Имя роли.
- `id` ([UUID](../../sql-reference/data-types/uuid.md)) — ID роли.
- `storage` ([String](../../sql-reference/data-types/string.md)) — Путь к хранилищу ролей. Настраивается в параметре `access_control_path`.
## Смотрите также {#see-also}
- [SHOW ROLES](../../sql-reference/statements/show.md#show-roles-statement)
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/roles) <!--hide-->

View File

@ -27,4 +27,8 @@
- `apply_to_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Политики строк применяются ко всем ролям и/или пользователям, за исключением перечисленных.
## Смотрите также {#see-also}
- [SHOW POLICIES](../../sql-reference/statements/show.md#show-policies-statement)
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/row_policies) <!--hide-->

View File

@ -17,4 +17,8 @@
- `apply_to_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Профиль настроек применяется ко всем ролям и/или пользователям, за исключением перечисленных.
## Смотрите также {#see-also}
- [SHOW PROFILES](../../sql-reference/statements/show.md#show-profiles-statement)
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/settings_profiles) <!--hide-->

View File

@ -82,7 +82,7 @@ res: /lib/x86_64-linux-gnu/libc-2.27.so
- [Функции интроспекции](../../sql-reference/functions/introspection.md) — Что такое функции интроспекции и как их использовать.
- [system.trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) — Содержит трассировки стека, собранные профилировщиком выборочных запросов.
- [arrayMap](../../sql-reference/functions/higher-order-functions.md#higher_order_functions-array-map) — Описание и пример использования функции `arrayMap`.
- [arrayFilter](../../sql-reference/functions/higher-order-functions.md#higher_order_functions-array-filter) — Описание и пример использования функции `arrayFilter`.
- [arrayMap](../../sql-reference/functions/array-functions.md#array-map) — Описание и пример использования функции `arrayMap`.
- [arrayFilter](../../sql-reference/functions/array-functions.md#array-filter) — Описание и пример использования функции `arrayFilter`.
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/stack_trace) <!--hide-->

View File

@ -27,4 +27,8 @@
- `default_roles_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Все предоставленные роли задаются по умолчанию, за исключением перечисленных.
## Смотрите также {#see-also}
- [SHOW USERS](../../sql-reference/statements/show.md#show-users-statement)
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/users) <!--hide-->

View File

@ -9,6 +9,7 @@ The following aggregate functions are supported:
- [`min`](../../sql-reference/aggregate-functions/reference/min.md#agg_function-min)
- [`max`](../../sql-reference/aggregate-functions/reference/max.md#agg_function-max)
- [`sum`](../../sql-reference/aggregate-functions/reference/sum.md#agg_function-sum)
- [`sumWithOverflow`](../../sql-reference/aggregate-functions/reference/sumwithoverflow.md#sumwithoverflowx)
- [`groupBitAnd`](../../sql-reference/aggregate-functions/reference/groupbitand.md#groupbitand)
- [`groupBitOr`](../../sql-reference/aggregate-functions/reference/groupbitor.md#groupbitor)
- [`groupBitXor`](../../sql-reference/aggregate-functions/reference/groupbitxor.md#groupbitxor)

View File

@ -2,7 +2,7 @@
Кортеж из элементов любого [типа](index.md#data_types). Элементы кортежа могут быть одного или разных типов.
Кортежи используются для временной группировки столбцов. Столбцы могут группироваться при использовании выражения IN в запросе, а также для указания нескольких формальных параметров лямбда-функций. Подробнее смотрите разделы [Операторы IN](../../sql-reference/data-types/tuple.md), [Функции высшего порядка](../../sql-reference/functions/higher-order-functions.md#higher-order-functions).
Кортежи используются для временной группировки столбцов. Столбцы могут группироваться при использовании выражения IN в запросе, а также для указания нескольких формальных параметров лямбда-функций. Подробнее смотрите разделы [Операторы IN](../../sql-reference/data-types/tuple.md), [Функции высшего порядка](../../sql-reference/functions/index.md#higher-order-functions).
Кортежи могут быть результатом запроса. В этом случае, в текстовых форматах кроме JSON, значения выводятся в круглых скобках через запятую. В форматах JSON, кортежи выводятся в виде массивов (в квадратных скобках).

View File

@ -1,4 +1,4 @@
# Функции по работе с массивами {#funktsii-po-rabote-s-massivami}
# Массивы {#functions-for-working-with-arrays}
## empty {#function-empty}
@ -186,6 +186,13 @@ SELECT indexOf([1, 3, NULL, NULL], NULL)
Элементы, равные `NULL`, обрабатываются как обычные значения.
## arrayCount(\[func,\] arr1, …) {#array-count}
Возвращает количество элементов массива `arr`, для которых функция `func` возвращает не 0. Если `func` не указана - возвращает количество ненулевых элементов массива.
Функция `arrayCount` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей можно передать лямбда-функцию.
## countEqual(arr, x) {#countequalarr-x}
Возвращает количество элементов массива, равных x. Эквивалентно arrayCount(elem -\> elem = x, arr).
@ -513,7 +520,7 @@ SELECT arraySort([1, nan, 2, NULL, 3, nan, -4, NULL, inf, -inf]);
- Значения `NaN` идут перед `NULL`.
- Значения `Inf` идут перед `NaN`.
Функция `arraySort` является [функцией высшего порядка](higher-order-functions.md) — в качестве первого аргумента ей можно передать лямбда-функцию. В этом случае порядок сортировки определяется результатом применения лямбда-функции на элементы массива.
Функция `arraySort` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей можно передать лямбда-функцию. В этом случае порядок сортировки определяется результатом применения лямбда-функции на элементы массива.
Рассмотрим пример:
@ -613,7 +620,7 @@ SELECT arrayReverseSort([1, nan, 2, NULL, 3, nan, -4, NULL, inf, -inf]) as res;
- Значения `NaN` идут перед `NULL`.
- Значения `-Inf` идут перед `NaN`.
Функция `arrayReverseSort` является [функцией высшего порядка](higher-order-functions.md). Вы можете передать ей в качестве первого аргумента лямбда-функцию. Например:
Функция `arrayReverseSort` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей можно передать лямбда-функцию. Например:
``` sql
SELECT arrayReverseSort((x) -> -x, [1, 2, 3]) as res;
@ -1036,6 +1043,116 @@ SELECT arrayZip(['a', 'b', 'c'], [5, 2, 1])
└──────────────────────────────────────┘
```
## arrayMap(func, arr1, …) {#array-map}
Возвращает массив, полученный на основе результатов применения функции `func` к каждому элементу массива `arr`.
Примеры:
``` sql
SELECT arrayMap(x -> (x + 2), [1, 2, 3]) as res;
```
``` text
┌─res─────┐
│ [3,4,5] │
└─────────┘
```
Следующий пример показывает, как создать кортежи из элементов разных массивов:
``` sql
SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) AS res
```
``` text
┌─res─────────────────┐
│ [(1,4),(2,5),(3,6)] │
└─────────────────────┘
```
Функция `arrayMap` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен.
## arrayFilter(func, arr1, …) {#array-filter}
Возвращает массив, содержащий только те элементы массива `arr1`, для которых функция `func` возвращает не 0.
Примеры:
``` sql
SELECT arrayFilter(x -> x LIKE '%World%', ['Hello', 'abc World']) AS res
```
``` text
┌─res───────────┐
│ ['abc World'] │
└───────────────┘
```
``` sql
SELECT
arrayFilter(
(i, x) -> x LIKE '%World%',
arrayEnumerate(arr),
['Hello', 'abc World'] AS arr)
AS res
```
``` text
┌─res─┐
│ [2] │
└─────┘
```
Функция `arrayFilter` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен.
## arrayExists(\[func,\] arr1, …) {#arrayexistsfunc-arr1}
Возвращает 1, если существует хотя бы один элемент массива `arr`, для которого функция func возвращает не 0. Иначе возвращает 0.
Функция `arrayExists` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) - в качестве первого аргумента ей можно передать лямбда-функцию.
## arrayAll(\[func,\] arr1, …) {#arrayallfunc-arr1}
Возвращает 1, если для всех элементов массива `arr`, функция `func` возвращает не 0. Иначе возвращает 0.
Функция `arrayAll` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) - в качестве первого аргумента ей можно передать лямбда-функцию.
## arrayFirst(func, arr1, …) {#array-first}
Возвращает первый элемент массива `arr1`, для которого функция func возвращает не 0.
Функция `arrayFirst` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен.
## arrayFirstIndex(func, arr1, …) {#array-first-index}
Возвращает индекс первого элемента массива `arr1`, для которого функция func возвращает не 0.
Функция `arrayFirstIndex` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен.
## arraySum(\[func,\] arr1, …) {#array-sum}
Возвращает сумму значений функции `func`. Если функция не указана - просто возвращает сумму элементов массива.
Функция `arraySum` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) - в качестве первого аргумента ей можно передать лямбда-функцию.
## arrayCumSum(\[func,\] arr1, …) {#arraycumsumfunc-arr1}
Возвращает массив из частичных сумм элементов исходного массива (сумма с накоплением). Если указана функция `func`, то значения элементов массива преобразуются этой функцией перед суммированием.
Функция `arrayCumSum` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) - в качестве первого аргумента ей можно передать лямбда-функцию.
Пример:
``` sql
SELECT arrayCumSum([1, 1, 1, 1]) AS res
```
``` text
┌─res──────────┐
│ [1, 2, 3, 4] │
└──────────────┘
## arrayAUC {#arrayauc}
Вычисляет площадь под кривой.

View File

@ -1,167 +0,0 @@
# Функции высшего порядка {#higher-order-functions}
## Оператор `->`, функция lambda(params, expr) {#operator-funktsiia-lambdaparams-expr}
Позволяет описать лямбда-функцию для передачи в функцию высшего порядка. Слева от стрелочки стоит формальный параметр - произвольный идентификатор, или несколько формальных параметров - произвольные идентификаторы в кортеже. Справа от стрелочки стоит выражение, в котором могут использоваться эти формальные параметры, а также любые столбцы таблицы.
Примеры: `x -> 2 * x, str -> str != Referer.`
Функции высшего порядка, в качестве своего функционального аргумента могут принимать только лямбда-функции.
В функции высшего порядка может быть передана лямбда-функция, принимающая несколько аргументов. В этом случае, в функцию высшего порядка передаётся несколько массивов одинаковых длин, которым эти аргументы будут соответствовать.
Для некоторых функций, например [arrayCount](#higher_order_functions-array-count) или [arraySum](#higher_order_functions-array-sum), первый аргумент (лямбда-функция) может отсутствовать. В этом случае, подразумевается тождественное отображение.
Для функций, перечисленных ниже, лямбда-функцию должна быть указана всегда:
- [arrayMap](#higher_order_functions-array-map)
- [arrayFilter](#higher_order_functions-array-filter)
- [arrayFirst](#higher_order_functions-array-first)
- [arrayFirstIndex](#higher_order_functions-array-first-index)
### arrayMap(func, arr1, …) {#higher_order_functions-array-map}
Вернуть массив, полученный на основе результатов применения функции `func` к каждому элементу массива `arr`.
Примеры:
``` sql
SELECT arrayMap(x -> (x + 2), [1, 2, 3]) as res;
```
``` text
┌─res─────┐
│ [3,4,5] │
└─────────┘
```
Следующий пример показывает, как создать кортежи из элементов разных массивов:
``` sql
SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) AS res
```
``` text
┌─res─────────────────┐
│ [(1,4),(2,5),(3,6)] │
└─────────────────────┘
```
Обратите внимание, что у функции `arrayMap` первый аргумент (лямбда-функция) не может быть опущен.
### arrayFilter(func, arr1, …) {#higher_order_functions-array-filter}
Вернуть массив, содержащий только те элементы массива `arr1`, для которых функция `func` возвращает не 0.
Примеры:
``` sql
SELECT arrayFilter(x -> x LIKE '%World%', ['Hello', 'abc World']) AS res
```
``` text
┌─res───────────┐
│ ['abc World'] │
└───────────────┘
```
``` sql
SELECT
arrayFilter(
(i, x) -> x LIKE '%World%',
arrayEnumerate(arr),
['Hello', 'abc World'] AS arr)
AS res
```
``` text
┌─res─┐
│ [2] │
└─────┘
```
Обратите внимание, что у функции `arrayFilter` первый аргумент (лямбда-функция) не может быть опущен.
### arrayCount(\[func,\] arr1, …) {#higher_order_functions-array-count}
Вернуть количество элементов массива `arr`, для которых функция func возвращает не 0. Если func не указана - вернуть количество ненулевых элементов массива.
### arrayExists(\[func,\] arr1, …) {#arrayexistsfunc-arr1}
Вернуть 1, если существует хотя бы один элемент массива `arr`, для которого функция func возвращает не 0. Иначе вернуть 0.
### arrayAll(\[func,\] arr1, …) {#arrayallfunc-arr1}
Вернуть 1, если для всех элементов массива `arr`, функция `func` возвращает не 0. Иначе вернуть 0.
### arraySum(\[func,\] arr1, …) {#higher_order_functions-array-sum}
Вернуть сумму значений функции `func`. Если функция не указана - просто вернуть сумму элементов массива.
### arrayFirst(func, arr1, …) {#higher_order_functions-array-first}
Вернуть первый элемент массива `arr1`, для которого функция func возвращает не 0.
Обратите внимание, что у функции `arrayFirst` первый аргумент (лямбда-функция) не может быть опущен.
### arrayFirstIndex(func, arr1, …) {#higher_order_functions-array-first-index}
Вернуть индекс первого элемента массива `arr1`, для которого функция func возвращает не 0.
Обратите внимание, что у функции `arrayFirstFilter` первый аргумент (лямбда-функция) не может быть опущен.
### arrayCumSum(\[func,\] arr1, …) {#arraycumsumfunc-arr1}
Возвращает массив из частичных сумм элементов исходного массива (сумма с накоплением). Если указана функция `func`, то значения элементов массива преобразуются этой функцией перед суммированием.
Пример:
``` sql
SELECT arrayCumSum([1, 1, 1, 1]) AS res
```
``` text
┌─res──────────┐
│ [1, 2, 3, 4] │
└──────────────┘
```
### arraySort(\[func,\] arr1, …) {#arraysortfunc-arr1}
Возвращает отсортированный в восходящем порядке массив `arr1`. Если задана функция `func`, то порядок сортировки определяется результатом применения функции `func` на элементы массива (массивов).
Для улучшения эффективности сортировки применяется [Преобразование Шварца](https://ru.wikipedia.org/wiki/%D0%9F%D1%80%D0%B5%D0%BE%D0%B1%D1%80%D0%B0%D0%B7%D0%BE%D0%B2%D0%B0%D0%BD%D0%B8%D0%B5_%D0%A8%D0%B2%D0%B0%D1%80%D1%86%D0%B0).
Пример:
``` sql
SELECT arraySort((x, y) -> y, ['hello', 'world'], [2, 1]);
```
``` text
┌─res────────────────┐
│ ['world', 'hello'] │
└────────────────────┘
```
Подробная информация о методе `arraySort` приведена в разделе [Функции по работе с массивами](array-functions.md#array_functions-sort).
### arrayReverseSort(\[func,\] arr1, …) {#arrayreversesortfunc-arr1}
Возвращает отсортированный в нисходящем порядке массив `arr1`. Если задана функция `func`, то порядок сортировки определяется результатом применения функции `func` на элементы массива (массивов).
Пример:
``` sql
SELECT arrayReverseSort((x, y) -> y, ['hello', 'world'], [2, 1]) as res;
```
``` text
┌─res───────────────┐
│ ['hello','world'] │
└───────────────────┘
```
Подробная информация о методе `arrayReverseSort` приведена в разделе [Функции по работе с массивами](array-functions.md#array_functions-reverse-sort).
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/higher_order_functions/) <!--hide-->

View File

@ -38,6 +38,20 @@
Функции не могут поменять значения своих аргументов - любые изменения возвращаются в качестве результата. Соответственно, от порядка записи функций в запросе, результат вычислений отдельных функций не зависит.
## Функции высшего порядка, оператор `->` и функция lambda(params, expr) {#higher-order-functions}
Функции высшего порядка, в качестве своего функционального аргумента могут принимать только лямбда-функции. Чтобы передать лямбда-функцию в функцию высшего порядка, используйте оператор `->`. Слева от стрелочки стоит формальный параметр — произвольный идентификатор, или несколько формальных параметров — произвольные идентификаторы в кортеже. Справа от стрелочки стоит выражение, в котором могут использоваться эти формальные параметры, а также любые столбцы таблицы.
Примеры:
```
x -> 2 * x
str -> str != Referer
```
В функции высшего порядка может быть передана лямбда-функция, принимающая несколько аргументов. В этом случае в функцию высшего порядка передаётся несколько массивов одинаковой длины, которым эти аргументы будут соответствовать.
Для некоторых функций первый аргумент (лямбда-функция) может отсутствовать. В этом случае подразумевается тождественное отображение.
## Обработка ошибок {#obrabotka-oshibok}
Некоторые функции могут кидать исключения в случае ошибочных данных. В этом случае, выполнение запроса прерывается, и текст ошибки выводится клиенту. При распределённой обработке запроса, при возникновении исключения на одном из серверов, на другие серверы пытается отправиться просьба тоже прервать выполнение запроса.

View File

@ -93,7 +93,7 @@ LIMIT 1
\G
```
Функция [arrayMap](higher-order-functions.md#higher_order_functions-array-map) позволяет обрабатывать каждый отдельный элемент массива `trace` с помощью функции `addressToLine`. Результат этой обработки вы видите в виде `trace_source_code_lines` колонки выходных данных.
Функция [arrayMap](../../sql-reference/functions/array-functions.md#array-map) позволяет обрабатывать каждый отдельный элемент массива `trace` с помощью функции `addressToLine`. Результат этой обработки вы видите в виде `trace_source_code_lines` колонки выходных данных.
``` text
Row 1:
@ -179,7 +179,7 @@ LIMIT 1
\G
```
То [arrayMap](higher-order-functions.md#higher_order_functions-array-map) функция позволяет обрабатывать каждый отдельный элемент системы. `trace` массив по типу `addressToSymbols` функция. Результат этой обработки вы видите в виде `trace_symbols` колонка выходных данных.
То [arrayMap](../../sql-reference/functions/array-functions.md#array-map) функция позволяет обрабатывать каждый отдельный элемент системы. `trace` массив по типу `addressToSymbols` функция. Результат этой обработки вы видите в виде `trace_symbols` колонка выходных данных.
``` text
Row 1:
@ -276,7 +276,7 @@ LIMIT 1
\G
```
Функция [arrayMap](higher-order-functions.md#higher_order_functions-array-map) позволяет обрабатывать каждый отдельный элемент массива `trace` с помощью функции `demangle`.
Функция [arrayMap](../../sql-reference/functions/array-functions.md#array-map) позволяет обрабатывать каждый отдельный элемент массива `trace` с помощью функции `demangle`.
``` text
Row 1:

View File

@ -169,4 +169,65 @@ SHOW CREATE QUOTA [name | CURRENT]
SHOW CREATE [SETTINGS] PROFILE name
```
## SHOW USERS {#show-users-statement}
Выводит список [пользовательских аккаунтов](../../operations/access-rights.md#user-account-management). Для просмотра параметров пользовательских аккаунтов, см. системную таблицу [system.users](../../operations/system-tables/users.md#system_tables-users).
### Синтаксис {#show-users-syntax}
``` sql
SHOW USERS
```
## SHOW ROLES {#show-roles-statement}
Выводит список [ролей](../../operations/access-rights.md#role-management). Для просмотра параметров ролей, см. системные таблицы [system.roles](../../operations/system-tables/roles.md#system_tables-roles) и [system.role-grants](../../operations/system-tables/role-grants.md#system_tables-role_grants).
### Синтаксис {#show-roles-syntax}
``` sql
SHOW [CURRENT|ENABLED] ROLES
```
## SHOW PROFILES {#show-profiles-statement}
Выводит список [профилей настроек](../../operations/access-rights.md#settings-profiles-management). Для просмотра других параметров профилей настроек, см. системную таблицу [settings_profiles](../../operations/system-tables/settings_profiles.md#system_tables-settings_profiles).
### Синтаксис {#show-profiles-syntax}
``` sql
SHOW [SETTINGS] PROFILES
```
## SHOW POLICIES {#show-policies-statement}
Выводит список [политик доступа к строкам](../../operations/access-rights.md#row-policy-management) для указанной таблицы. Для просмотра других параметров, см. системную таблицу [system.row_policies](../../operations/system-tables/row_policies.md#system_tables-row_policies).
### Синтаксис {#show-policies-syntax}
``` sql
SHOW [ROW] POLICIES [ON [db.]table]
```
## SHOW QUOTAS {#show-quotas-statement}
Выводит список [квот](../../operations/access-rights.md#quotas-management). Для просмотра параметров квот, см. системную таблицу [system.quotas](../../operations/system-tables/quotas.md#system_tables-quotas).
### Синтаксис {#show-quotas-syntax}
``` sql
SHOW QUOTAS
```
## SHOW QUOTA {#show-quota-statement}
Выводит потребление [квоты](../../operations/quotas.md) для всех пользователей или только для текущего пользователя. Для просмотра других параметров, см. системные таблицы [system.quotas_usage](../../operations/system-tables/quotas_usage.md#system_tables-quotas_usage) и [system.quota_usage](../../operations/system-tables/quota_usage.md#system_tables-quota_usage).
### Синтаксис {#show-quota-syntax}
``` sql
SHOW [CURRENT] QUOTA
```
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/show/) <!--hide-->

View File

@ -1,6 +1,4 @@
---
machine_translated: true
machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd
toc_priority: 12
toc_title: "\u6559\u7A0B"
---
@ -9,27 +7,27 @@ toc_title: "\u6559\u7A0B"
## 从本教程中可以期待什么? {#what-to-expect-from-this-tutorial}
通过本教程您将学习如何设置一个简单的ClickHouse集群。 它会很小,但容错和可扩展。 然后,我们将使用其中一个示例数据集来填充数据并执行一些演示查询。
通过本教程您将学习如何设置一个简单的ClickHouse集群。 它会很小,但却是容错和可扩展。 然后,我们将使用其中一个示例数据集来填充数据并执行一些演示查询。
## 单节点设置 {#single-node-setup}
为了推迟分布式环境的复杂性我们将首先在单个服务器或虚拟机上部署ClickHouse。 ClickHouse通常是从安装 [黛布](install.md#install-from-deb-packages) 或 [rpm](install.md#from-rpm-packages) 包,但也有 [替代办法](install.md#from-docker-image) 对于不支持它们的操作系统
为了推迟分布式环境的复杂性我们将首先在单个服务器或虚拟机上部署ClickHouse。 ClickHouse通常是从[deb](install.md#install-from-deb-packages) 或 [rpm](install.md#from-rpm-packages) 包安装,但对于不支持它们的操作系统也有 [替代方法](install.md#from-docker-image)
例如,您选择了 `deb` 包和执行:
例如,您选择了`deb` 包安装,执行:
``` bash
{% include 'install/deb.sh' %}
```
我们在安装的软件包中有什么:
在我们安装的软件中包含这些包:
- `clickhouse-client`包含 [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 应用程序交互式ClickHouse控制台客户端。
- `clickhouse-common` 包包含一个ClickHouse可执行文件。
- `clickhouse-server`包含要作为服务器运行ClickHouse的配置文件。
- `clickhouse-client`,包含 [clickhouse-client](../interfaces/cli.md) 应用程序,它是交互式ClickHouse控制台客户端。
- `clickhouse-common`包含一个ClickHouse可执行文件。
- `clickhouse-server`包含要作为服务端运行的ClickHouse配置文件。
服务器配置文件位于 `/etc/clickhouse-server/`. 在进一步讨论之前,请注意 `<path>` 元素in `config.xml`. Path确定数据存储的位置因此应该位于磁盘容量较大的卷上默认值为 `/var/lib/clickhouse/`. 如果你想调整配置,直接编辑并不方便 `config.xml` 文件,考虑到它可能会在未来的软件包更新中被重写。 复盖配置元素的推荐方法是创建 [在配置文件。d目录](../operations/configuration-files.md) 它作为 “patches” 要配置。xml
服务端配置文件位于 `/etc/clickhouse-server/`。 在进一步讨论之前,请注意 `config.xml`文件中的`<path>` 元素. Path决定了数据存储的位置因此该位置应该位于磁盘容量较大的卷上默认值为 `/var/lib/clickhouse/`。 如果你想调整配置,考虑到它可能会在未来的软件包更新中被重写,直接编辑`config.xml` 文件并不方便。 推荐的方法是在[配置文件](../operations/configuration-files.md)目录创建文件作为config.xml文件的“补丁”用以复写配置元素。
你可能已经注意到了, `clickhouse-server` 安装后不会自动启动。 它也不会在更新后自动重新启动。 您启动服务器的方式取决于您的init系统通常情况下它是:
你可能已经注意到了, `clickhouse-server` 安装后不会自动启动。 它也不会在更新后自动重新启动。 您启动服务端的方式取决于您的初始系统,通常情况下是这样:
``` bash
sudo service clickhouse-server start
@ -41,13 +39,13 @@ sudo service clickhouse-server start
sudo /etc/init.d/clickhouse-server start
```
服务器日志的默认位置是 `/var/log/clickhouse-server/`. 服务器已准备好处理客户端连接一旦它记录 `Ready for connections` 消息
服务端日志的默认位置是 `/var/log/clickhouse-server/`。当服务端在日志中记录 `Ready for connections` 消息,即表示服务端已准备好处理客户端连接。
一旦 `clickhouse-server` 正在运行我们可以利用 `clickhouse-client` 连接到服务器并运行一些测试查询,如 `SELECT "Hello, world!";`.
一旦 `clickhouse-server` 启动并运行,我们可以利用 `clickhouse-client` 连接到服务端,并运行一些测试查询,如 `SELECT "Hello, world!";`.
<details markdown="1">
<summary>Clickhouse-客户端的快速提示</summary>
<summary>Clickhouse-client的快速提示</summary>
交互模式:

View File

@ -15,7 +15,7 @@ toc_title: "\u5E94\u7528CatBoost\u6A21\u578B"
1. [创建表](#create-table).
2. [将数据插入到表中](#insert-data-to-table).
3. [碌莽禄into拢Integrate010-68520682\<url\>](#integrate-catboost-into-clickhouse) (可选步骤)。
3. [将CatBoost集成到ClickHouse中](#integrate-catboost-into-clickhouse) (可选步骤)。
4. [从SQL运行模型推理](#run-model-inference).
有关训练CatBoost模型的详细信息请参阅 [培训和应用模型](https://catboost.ai/docs/features/training.html#training).
@ -119,12 +119,12 @@ FROM amazon_train
+-------+
```
## 3. 碌莽禄into拢Integrate010-68520682\<url\> {#integrate-catboost-into-clickhouse}
## 3. 将CatBoost集成到ClickHouse中 {#integrate-catboost-into-clickhouse}
!!! note "注"
**可选步骤。** Docker映像包含运行CatBoost和ClickHouse所需的所有内容。
碌莽禄to拢integrate010-68520682\<url\>:
CatBoost集成到ClickHouse步骤:
**1.** 构建评估库。

View File

@ -15,6 +15,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
extern const int UNKNOWN_TYPE;
}
@ -86,6 +87,8 @@ namespace
case ValueType::vtUUID:
assert_cast<ColumnUInt128 &>(column).insert(parse<UUID>(value.convert<std::string>()));
break;
default:
throw Exception("Unsupported value type", ErrorCodes::UNKNOWN_TYPE);
}
}

View File

@ -13,6 +13,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_TYPE;
}
namespace
{
using ValueType = ExternalResultDescription::ValueType;
@ -79,6 +84,9 @@ namespace
return Poco::Dynamic::Var(std::to_string(LocalDateTime(time_t(field.get<UInt64>())))).convert<String>();
case ValueType::vtUUID:
return Poco::Dynamic::Var(UUID(field.get<UInt128>()).toUnderType().toHexString()).convert<std::string>();
default:
throw Exception("Unsupported value type", ErrorCodes::UNKNOWN_TYPE);
}
__builtin_unreachable();
}

View File

@ -8,6 +8,8 @@ PEERDIR(
contrib/libs/poco/NetSSL_OpenSSL
)
CFLAGS(-g0)
SRCS(
clickhouse-server.cpp

View File

@ -12,6 +12,8 @@ PEERDIR(
clickhouse/src
)
CFLAGS(-g0)
SRCS(
main.cpp

View File

@ -5,6 +5,8 @@ PEERDIR(
clickhouse/src/Common
)
CFLAGS(-g0)
SRCS(
AccessControlManager.cpp
AccessRights.cpp

View File

@ -4,6 +4,8 @@ PEERDIR(
clickhouse/src/Common
)
CFLAGS(-g0)
SRCS(
<? find . -name '*.cpp' | sed 's/^\.\// /' | sort ?>
)

View File

@ -12,6 +12,9 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
class AggregateFunctionCombinatorArray final : public IAggregateFunctionCombinator
{
public:
@ -45,6 +48,8 @@ public:
}
};
}
void registerAggregateFunctionCombinatorArray(AggregateFunctionCombinatorFactory & factory)
{
factory.registerCombinator(std::make_shared<AggregateFunctionCombinatorArray>());

View File

@ -6,12 +6,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
class AggregateFunctionCombinatorDistinct final : public IAggregateFunctionCombinator
{
public:
@ -56,6 +58,8 @@ public:
}
};
}
void registerAggregateFunctionCombinatorDistinct(AggregateFunctionCombinatorFactory & factory)
{
factory.registerCombinator(std::make_shared<AggregateFunctionCombinatorDistinct>());

View File

@ -12,6 +12,9 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
class AggregateFunctionCombinatorForEach final : public IAggregateFunctionCombinator
{
public:
@ -42,6 +45,8 @@ public:
}
};
}
void registerAggregateFunctionCombinatorForEach(AggregateFunctionCombinatorFactory & factory)
{
factory.registerCombinator(std::make_shared<AggregateFunctionCombinatorForEach>());

View File

@ -13,6 +13,9 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
class AggregateFunctionCombinatorMerge final : public IAggregateFunctionCombinator
{
public:
@ -55,6 +58,8 @@ public:
}
};
}
void registerAggregateFunctionCombinatorMerge(AggregateFunctionCombinatorFactory & factory)
{
factory.registerCombinator(std::make_shared<AggregateFunctionCombinatorMerge>());

View File

@ -15,6 +15,9 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
class AggregateFunctionCombinatorNull final : public IAggregateFunctionCombinator
{
public:
@ -119,6 +122,8 @@ public:
}
};
}
void registerAggregateFunctionCombinatorNull(AggregateFunctionCombinatorFactory & factory)
{
factory.registerCombinator(std::make_shared<AggregateFunctionCombinatorNull>());

View File

@ -6,6 +6,8 @@
namespace DB
{
namespace
{
template <bool UseNull>
class AggregateFunctionCombinatorOrFill final : public IAggregateFunctionCombinator
@ -32,6 +34,8 @@ public:
}
};
}
void registerAggregateFunctionCombinatorOrFill(AggregateFunctionCombinatorFactory & factory)
{
factory.registerCombinator(std::make_shared<AggregateFunctionCombinatorOrFill<false>>());

View File

@ -13,6 +13,9 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
class AggregateFunctionCombinatorResample final : public IAggregateFunctionCombinator
{
public:
@ -93,6 +96,8 @@ public:
}
};
}
void registerAggregateFunctionCombinatorResample(AggregateFunctionCombinatorFactory & factory)
{
factory.registerCombinator(std::make_shared<AggregateFunctionCombinatorResample>());

View File

@ -4,6 +4,7 @@
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypeArray.h>
#include <Common/assert_cast.h>
#include <common/arithmeticOverflow.h>
namespace DB
@ -60,7 +61,18 @@ public:
if (end < begin)
total = 0;
else
total = (end - begin + step - 1) / step;
{
Key dif;
size_t sum;
if (common::subOverflow(end, begin, dif)
|| common::addOverflow(static_cast<size_t>(dif), step, sum))
{
throw Exception("Overflow in internal computations in function " + getName()
+ ". Too large arguments", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
total = (sum - 1) / step; // total = (end - begin + step - 1) / step
}
if (total > MAX_ELEMENTS)
throw Exception("The range given in function "

View File

@ -13,6 +13,9 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
namespace
{
class AggregateFunctionCombinatorState final : public IAggregateFunctionCombinator
{
public:
@ -33,6 +36,8 @@ public:
}
};
}
void registerAggregateFunctionCombinatorState(AggregateFunctionCombinatorFactory & factory)
{
factory.registerCombinator(std::make_shared<AggregateFunctionCombinatorState>());

View File

@ -5,6 +5,8 @@ PEERDIR(
clickhouse/src/Common
)
CFLAGS(-g0)
SRCS(
AggregateFunctionAggThrow.cpp
AggregateFunctionArray.cpp

View File

@ -4,6 +4,8 @@ PEERDIR(
clickhouse/src/Common
)
CFLAGS(-g0)
SRCS(
<? find . -name '*.cpp' | grep -v -F tests | grep -v -F GroupBitmap | sed 's/^\.\// /' | sort ?>
)

View File

@ -6,6 +6,8 @@ PEERDIR(
contrib/libs/poco/NetSSL_OpenSSL
)
CFLAGS(-g0)
SRCS(
Connection.cpp
ConnectionPoolWithFailover.cpp

View File

@ -5,6 +5,8 @@ PEERDIR(
contrib/libs/poco/NetSSL_OpenSSL
)
CFLAGS(-g0)
SRCS(
<? find . -name '*.cpp' | grep -v -F tests | sed 's/^\.\// /' | sort ?>
)

View File

@ -170,7 +170,6 @@ public:
size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); }
bool isNumeric() const override { return getDictionary().isNumeric(); }
bool lowCardinality() const override { return true; }
bool isNullable() const override { return isColumnNullable(*dictionary.getColumnUniquePtr()); }
const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); }
const ColumnPtr & getDictionaryPtr() const { return dictionary.getColumnUniquePtr(); }

View File

@ -13,6 +13,8 @@ PEERDIR(
contrib/libs/pdqsort
)
CFLAGS(-g0)
SRCS(
Collator.cpp
ColumnAggregateFunction.cpp

View File

@ -21,6 +21,8 @@
#define PREPROCESSED_SUFFIX "-preprocessed"
namespace fs = std::filesystem;
using namespace Poco::XML;
namespace DB
@ -451,7 +453,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
XMLDocumentPtr config;
LOG_DEBUG(log, "Processing configuration file '{}'.", path);
if (std::filesystem::exists(path))
if (fs::exists(path))
{
config = dom_parser.parse(path);
}
@ -610,6 +612,7 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
{
if (preprocessed_path.empty())
{
fs::path preprocessed_configs_path("preprocessed_configs/");
auto new_path = loaded_config.config_path;
if (new_path.substr(0, main_config_path.size()) == main_config_path)
new_path.replace(0, main_config_path.size(), "");
@ -628,15 +631,17 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
}
else
{
preprocessed_dir = loaded_config.configuration->getString("path") + "/preprocessed_configs/";
fs::path loaded_config_path(loaded_config.configuration->getString("path"));
preprocessed_dir = loaded_config_path / preprocessed_configs_path;
}
}
else
{
preprocessed_dir += "/preprocessed_configs/";
fs::path preprocessed_dir_path(preprocessed_dir);
preprocessed_dir = (preprocessed_dir_path / preprocessed_configs_path).string();
}
preprocessed_path = preprocessed_dir + new_path;
preprocessed_path = (fs::path(preprocessed_dir) / fs::path(new_path)).string();
auto preprocessed_path_parent = Poco::Path(preprocessed_path).makeParent();
if (!preprocessed_path_parent.toString().empty())
Poco::File(preprocessed_path_parent).createDirectories();

View File

@ -281,7 +281,7 @@ namespace ErrorCodes
extern const int DICTIONARY_IS_EMPTY = 281;
extern const int INCORRECT_INDEX = 282;
extern const int UNKNOWN_DISTRIBUTED_PRODUCT_MODE = 283;
extern const int UNKNOWN_GLOBAL_SUBQUERIES_METHOD = 284;
extern const int WRONG_GLOBAL_SUBQUERY = 284;
extern const int TOO_FEW_LIVE_REPLICAS = 285;
extern const int UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE = 286;
extern const int UNKNOWN_FORMAT_VERSION = 287;
@ -503,6 +503,11 @@ namespace ErrorCodes
extern const int CANNOT_RESTORE_FROM_FIELD_DUMP = 536;
extern const int ILLEGAL_MYSQL_VARIABLE = 537;
extern const int MYSQL_SYNTAX_ERROR = 538;
extern const int CANNOT_BIND_RABBITMQ_EXCHANGE = 539;
extern const int CANNOT_DECLARE_RABBITMQ_EXCHANGE = 540;
extern const int CANNOT_CREATE_RABBITMQ_QUEUE_BINDING = 541;
extern const int CANNOT_REMOVE_RABBITMQ_EXCHANGE = 542;
extern const int UNKNOWN_MYSQL_DATATYPES_SUPPORT_LEVEL = 543;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -82,6 +82,12 @@ inline UInt64 getCurrentTimeNanoseconds(clockid_t clock_type = CLOCK_MONOTONIC)
return ts.tv_sec * 1000000000ULL + ts.tv_nsec;
}
inline UInt64 getCurrentTimeMicroseconds()
{
struct timeval tv;
gettimeofday(&tv, nullptr);
return (tv.tv_sec) * 1000000U + (tv.tv_usec);
}
struct RUsageCounters
{

View File

@ -190,6 +190,7 @@ protected:
bool performance_counters_finalized = false;
UInt64 query_start_time_nanoseconds = 0;
UInt64 query_start_time_microseconds = 0;
time_t query_start_time = 0;
size_t queries_started = 0;

View File

@ -21,6 +21,8 @@ PEERDIR(
INCLUDE(${ARCADIA_ROOT}/clickhouse/cmake/yandex/ya.make.versions.inc)
CFLAGS(-g0)
SRCS(
ActionLock.cpp
AlignedBuffer.cpp

View File

@ -20,6 +20,8 @@ PEERDIR(
INCLUDE(${ARCADIA_ROOT}/clickhouse/cmake/yandex/ya.make.versions.inc)
CFLAGS(-g0)
SRCS(
<? find . -name '*.cpp' | grep -v -F tests | sed 's/^\.\// /' | sort ?>
)

View File

@ -72,9 +72,10 @@ bool CachedCompressedReadBuffer::nextImpl()
}
CachedCompressedReadBuffer::CachedCompressedReadBuffer(
const std::string & path_, std::function<std::unique_ptr<ReadBufferFromFileBase>()> file_in_creator_, UncompressedCache * cache_)
const std::string & path_, std::function<std::unique_ptr<ReadBufferFromFileBase>()> file_in_creator_, UncompressedCache * cache_, bool allow_different_codecs_)
: ReadBuffer(nullptr, 0), file_in_creator(std::move(file_in_creator_)), cache(cache_), path(path_), file_pos(0)
{
allow_different_codecs = allow_different_codecs_;
}
void CachedCompressedReadBuffer::seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block)

View File

@ -38,7 +38,7 @@ private:
clockid_t clock_type {};
public:
CachedCompressedReadBuffer(const std::string & path, std::function<std::unique_ptr<ReadBufferFromFileBase>()> file_in_creator, UncompressedCache * cache_);
CachedCompressedReadBuffer(const std::string & path, std::function<std::unique_ptr<ReadBufferFromFileBase>()> file_in_creator, UncompressedCache * cache_, bool allow_different_codecs_ = false);
void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block);

View File

@ -105,13 +105,24 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed,
uint8_t method = ICompressionCodec::readMethod(own_compressed_buffer.data());
if (!codec)
{
codec = CompressionCodecFactory::instance().get(method);
}
else if (method != codec->getMethodByte())
throw Exception("Data compressed with different methods, given method byte 0x"
+ getHexUIntLowercase(method)
+ ", previous method byte 0x"
+ getHexUIntLowercase(codec->getMethodByte()),
ErrorCodes::CANNOT_DECOMPRESS);
{
if (allow_different_codecs)
{
codec = CompressionCodecFactory::instance().get(method);
}
else
{
throw Exception("Data compressed with different methods, given method byte 0x"
+ getHexUIntLowercase(method)
+ ", previous method byte 0x"
+ getHexUIntLowercase(codec->getMethodByte()),
ErrorCodes::CANNOT_DECOMPRESS);
}
}
size_compressed_without_checksum = ICompressionCodec::readCompressedBlockSize(own_compressed_buffer.data());
size_decompressed = ICompressionCodec::readDecompressedBlockSize(own_compressed_buffer.data());
@ -163,21 +174,32 @@ void CompressedReadBufferBase::decompress(char * to, size_t size_decompressed, s
uint8_t method = ICompressionCodec::readMethod(compressed_buffer);
if (!codec)
{
codec = CompressionCodecFactory::instance().get(method);
}
else if (codec->getMethodByte() != method)
throw Exception("Data compressed with different methods, given method byte "
+ getHexUIntLowercase(method)
+ ", previous method byte "
+ getHexUIntLowercase(codec->getMethodByte()),
ErrorCodes::CANNOT_DECOMPRESS);
{
if (allow_different_codecs)
{
codec = CompressionCodecFactory::instance().get(method);
}
else
{
throw Exception("Data compressed with different methods, given method byte "
+ getHexUIntLowercase(method)
+ ", previous method byte "
+ getHexUIntLowercase(codec->getMethodByte()),
ErrorCodes::CANNOT_DECOMPRESS);
}
}
codec->decompress(compressed_buffer, size_compressed_without_checksum, to);
}
/// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'.
CompressedReadBufferBase::CompressedReadBufferBase(ReadBuffer * in)
: compressed_in(in), own_compressed_buffer(0)
CompressedReadBufferBase::CompressedReadBufferBase(ReadBuffer * in, bool allow_different_codecs_)
: compressed_in(in), own_compressed_buffer(0), allow_different_codecs(allow_different_codecs_)
{
}

View File

@ -26,6 +26,9 @@ protected:
/// Don't checksum on decompressing.
bool disable_checksum = false;
/// Allow reading data, compressed by different codecs from one file.
bool allow_different_codecs;
/// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need.
/// Returns number of compressed bytes read.
size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum);
@ -34,7 +37,7 @@ protected:
public:
/// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'.
CompressedReadBufferBase(ReadBuffer * in = nullptr);
CompressedReadBufferBase(ReadBuffer * in = nullptr, bool allow_different_codecs_ = false);
~CompressedReadBufferBase();
/** Disable checksums.

View File

@ -36,20 +36,22 @@ bool CompressedReadBufferFromFile::nextImpl()
return true;
}
CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr<ReadBufferFromFileBase> buf)
CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr<ReadBufferFromFileBase> buf, bool allow_different_codecs_)
: BufferWithOwnMemory<ReadBuffer>(0), p_file_in(std::move(buf)), file_in(*p_file_in)
{
compressed_in = &file_in;
allow_different_codecs = allow_different_codecs_;
}
CompressedReadBufferFromFile::CompressedReadBufferFromFile(
const std::string & path, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, size_t buf_size)
const std::string & path, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, size_t buf_size, bool allow_different_codecs_)
: BufferWithOwnMemory<ReadBuffer>(0)
, p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, mmap_threshold, buf_size))
, file_in(*p_file_in)
{
compressed_in = &file_in;
allow_different_codecs = allow_different_codecs_;
}

View File

@ -28,10 +28,11 @@ private:
bool nextImpl() override;
public:
CompressedReadBufferFromFile(std::unique_ptr<ReadBufferFromFileBase> buf);
CompressedReadBufferFromFile(std::unique_ptr<ReadBufferFromFileBase> buf, bool allow_different_codecs_ = false);
CompressedReadBufferFromFile(
const std::string & path, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE);
const std::string & path, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, bool allow_different_codecs_ = false);
void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block);

View File

@ -36,6 +36,11 @@ ASTPtr CompressionCodecDelta::getCodecDesc() const
return makeASTFunction("Delta", literal);
}
void CompressionCodecDelta::updateHash(SipHash & hash) const
{
getCodecDesc()->updateTreeHash(hash);
}
namespace
{

View File

@ -14,7 +14,10 @@ public:
ASTPtr getCodecDesc() const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;

View File

@ -339,6 +339,12 @@ ASTPtr CompressionCodecDoubleDelta::getCodecDesc() const
return std::make_shared<ASTIdentifier>("DoubleDelta");
}
void CompressionCodecDoubleDelta::updateHash(SipHash & hash) const
{
getCodecDesc()->updateTreeHash(hash);
hash.update(data_bytes_size);
}
UInt32 CompressionCodecDoubleDelta::getMaxCompressedDataSize(UInt32 uncompressed_size) const
{
const auto result = 2 // common header

View File

@ -100,7 +100,10 @@ public:
ASTPtr getCodecDesc() const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;

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