Merge branch 'master' into log-engine-rollback-on-insert-error

This commit is contained in:
Alexey Milovidov 2020-07-16 05:34:02 +03:00
commit 3408b7e259
265 changed files with 4736 additions and 1600 deletions

2
.gitmodules vendored
View File

@ -49,7 +49,7 @@
url = https://github.com/ClickHouse-Extras/boost.git
[submodule "contrib/base64"]
path = contrib/base64
url = https://github.com/powturbo/Turbo-Base64.git
url = https://github.com/ClickHouse-Extras/Turbo-Base64.git
[submodule "contrib/arrow"]
path = contrib/arrow
url = https://github.com/apache/arrow

View File

@ -287,7 +287,7 @@ endif ()
include(cmake/dbms_glob_sources.cmake)
if (OS_LINUX)
if (OS_LINUX OR OS_ANDROID)
include(cmake/linux/default_libs.cmake)
elseif (OS_DARWIN)
include(cmake/darwin/default_libs.cmake)

View File

@ -9,12 +9,12 @@ ClickHouse is an open-source column-oriented database management system that all
* [Documentation](https://clickhouse.tech/docs/en/) provides more in-depth information.
* [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format.
* [Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-d2zxkf9e-XyxDa_ucfPxzuH4SJIm~Ng) and [Telegram](https://telegram.me/clickhouse_en) allow to chat with ClickHouse users in real-time.
* [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events.
* [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announcements and reports about events.
* [Yandex.Messenger channel](https://yandex.ru/chat/#/join/20e380d9-c7be-4123-ab06-e95fb946975e) shares announcements and useful links in Russian.
* [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any.
* You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person.
## Upcoming Events
* [ClickHouse for genetic data (in Russian)](https://cloud.yandex.ru/events/152) on July 14, 2020.
* [ClickHouse virtual office hours](https://www.eventbrite.com/e/clickhouse-july-virtual-meetup-tickets-111199787558) on July 15, 2020.
* [ClickHouse at ByteDance (in Chinese)](https://mp.weixin.qq.com/s/Em-HjPylO8D7WPui4RREAQ) on July 17, 2020.

View File

@ -686,12 +686,17 @@ public:
inline time_t makeDateTime(UInt16 year, UInt8 month, UInt8 day_of_month, UInt8 hour, UInt8 minute, UInt8 second) const
{
size_t index = makeDayNum(year, month, day_of_month);
time_t time_offset = hour * 3600 + minute * 60 + second;
UInt32 time_offset = hour * 3600 + minute * 60 + second;
if (time_offset >= lut[index].time_at_offset_change)
time_offset -= lut[index].amount_of_offset_change;
return lut[index].date + time_offset;
UInt32 res = lut[index].date + time_offset;
if (unlikely(res > DATE_LUT_MAX))
return 0;
return res;
}
inline const Values & getValues(DayNum d) const { return lut[d]; }

View File

@ -1,6 +1,9 @@
#include <common/getThreadId.h>
#if defined(OS_LINUX)
#if defined(OS_ANDROID)
#include <sys/types.h>
#include <unistd.h>
#elif defined(OS_LINUX)
#include <unistd.h>
#include <syscall.h>
#elif defined(OS_FREEBSD)
@ -16,7 +19,9 @@ uint64_t getThreadId()
{
if (!current_tid)
{
#if defined(OS_LINUX)
#if defined(OS_ANDROID)
current_tid = gettid();
#elif defined(OS_LINUX)
current_tid = syscall(SYS_gettid); /// This call is always successful. - man gettid
#elif defined(OS_FREEBSD)
current_tid = pthread_getthreadid_np();

View File

@ -9,7 +9,6 @@
#include <string.h>
#include <signal.h>
#include <cxxabi.h>
#include <execinfo.h>
#include <unistd.h>
#include <typeinfo>

View File

@ -1,9 +1,9 @@
# This strings autochanged from release_lib.sh:
SET(VERSION_REVISION 54436)
SET(VERSION_REVISION 54437)
SET(VERSION_MAJOR 20)
SET(VERSION_MINOR 6)
SET(VERSION_MINOR 7)
SET(VERSION_PATCH 1)
SET(VERSION_GITHASH efc57fb063b3fb4df968d916720ec4d4ced4642e)
SET(VERSION_DESCRIBE v20.6.1.1-prestable)
SET(VERSION_STRING 20.6.1.1)
SET(VERSION_GITHASH d64e51d1a78c1b53c33915ca0f75c97b2333844f)
SET(VERSION_DESCRIBE v20.7.1.1-prestable)
SET(VERSION_STRING 20.7.1.1)
# end of autochange

View File

@ -1,4 +1,4 @@
SET(ENABLE_AMQPCPP ${ENABLE_LIBRARIES})
option(ENABLE_AMQPCPP "Enalbe AMQP-CPP" ${ENABLE_LIBRARIES})
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/CMakeLists.txt")
message (WARNING "submodule contrib/AMQP-CPP is missing. to fix try run: \n git submodule update --init --recursive")
set (ENABLE_AMQPCPP 0)

View File

@ -11,7 +11,12 @@ else ()
set (BUILTINS_LIBRARY "-lgcc")
endif ()
if (OS_ANDROID)
# pthread and rt are included in libc
set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -ldl")
else ()
set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -lrt -lpthread -ldl")
endif ()
message(STATUS "Default libraries: ${DEFAULT_LIBS}")
@ -35,7 +40,11 @@ add_library(global-libs INTERFACE)
set(THREADS_PREFER_PTHREAD_FLAG ON)
find_package(Threads REQUIRED)
add_subdirectory(base/glibc-compatibility)
if (NOT OS_ANDROID)
# Our compatibility layer doesn't build under Android, many errors in musl.
add_subdirectory(base/glibc-compatibility)
endif ()
include (cmake/find/unwind.cmake)
include (cmake/find/cxx.cmake)

View File

@ -1,6 +1,11 @@
if (CMAKE_SYSTEM_NAME MATCHES "Linux")
set (OS_LINUX 1)
add_definitions(-D OS_LINUX)
elseif (CMAKE_SYSTEM_NAME MATCHES "Android")
# This is a toy configuration and not in CI, so expect it to be broken.
# Use cmake flags such as: -DCMAKE_TOOLCHAIN_FILE=~/ch2/android-ndk-r21d/build/cmake/android.toolchain.cmake -DANDROID_ABI=arm64-v8a -DANDROID_PLATFORM=28
set (OS_ANDROID 1)
add_definitions(-D OS_ANDROID)
elseif (CMAKE_SYSTEM_NAME MATCHES "FreeBSD")
set (OS_FREEBSD 1)
add_definitions(-D OS_FREEBSD)
@ -17,7 +22,7 @@ if (CMAKE_CROSSCOMPILING)
set (ENABLE_PARQUET OFF CACHE INTERNAL "")
set (ENABLE_ICU OFF CACHE INTERNAL "")
set (ENABLE_FASTOPS OFF CACHE INTERNAL "")
elseif (OS_LINUX)
elseif (OS_LINUX OR OS_ANDROID)
if (ARCH_AARCH64)
# FIXME: broken dependencies
set (ENABLE_PROTOBUF OFF CACHE INTERNAL "")

View File

@ -24,7 +24,7 @@ set (SRCS
add_library(amqp-cpp ${SRCS})
target_compile_options (amqp-cpp
PUBLIC
PRIVATE
-Wno-old-style-cast
-Wno-inconsistent-missing-destructor-override
-Wno-deprecated
@ -38,7 +38,7 @@ target_compile_options (amqp-cpp
-w
)
target_include_directories (amqp-cpp PUBLIC ${LIBRARY_DIR}/include)
target_include_directories (amqp-cpp SYSTEM PUBLIC ${LIBRARY_DIR}/include)
target_link_libraries (amqp-cpp PUBLIC ssl)

2
contrib/base64 vendored

@ -1 +1 @@
Subproject commit 95ba56a9b041f9933f5cd2bbb2ee4e083468c20a
Subproject commit af9b331f2b4f30b41c70f3a571ff904a8251c1d3

2
contrib/fmtlib vendored

@ -1 +1 @@
Subproject commit 297c3b2ed551a4989826fc8c4780bf533e964bd9
Subproject commit c108ee1d590089ccf642fc85652b845924067af2

2
contrib/libhdfs3 vendored

@ -1 +1 @@
Subproject commit e2131aa752d7e95441e08f9a18304c1445f2576a
Subproject commit 1b666578c85094306b061352078022f6350bfab8

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (20.6.1.1) unstable; urgency=low
clickhouse (20.7.1.1) unstable; urgency=low
* Modified source code
-- clickhouse-release <clickhouse-release@yandex-team.ru> Mon, 22 Jun 2020 20:40:23 +0300
-- clickhouse-release <clickhouse-release@yandex-team.ru> Mon, 13 Jul 2020 18:25:58 +0300

View File

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

View File

@ -33,6 +33,24 @@ then
rm /output/clickhouse-odbc-bridge ||:
cp -r ../docker/test/performance-comparison /output/scripts ||:
# We have to know the revision that corresponds to this binary build.
# It is not the nominal SHA from pull/*/head, but the pull/*/merge, which is
# head merged to master by github, at some point after the PR is updated.
# There are some quirks to consider:
# - apparently the real SHA is not recorded in system.build_options;
# - it can change at any time as github pleases, so we can't just record
# the SHA and use it later, it might become inaccessible;
# - CI has an immutable snapshot of repository that it uses for all checks
# for a given nominal SHA, but it is not accessible outside Yandex.
# This is why we add this repository snapshot from CI to the performance test
# package.
mkdir /output/ch
git -C /output/ch init --bare
git -C /output/ch remote add origin /build
git -C /output/ch fetch --no-tags --depth 50 origin HEAD
git -C /output/ch reset --soft FETCH_HEAD
git -C /output/ch log -5
fi
# May be set for split build or for performance test.

View File

@ -1,7 +1,7 @@
FROM ubuntu:20.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=20.6.1.*
ARG version=20.7.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.6.1.*
ARG version=20.7.1.*
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -0,0 +1,7 @@
<yandex>
<profiles>
<default>
<max_execution_time>10</max_execution_time>
</default>
</profiles>
</yandex>

View File

@ -46,6 +46,7 @@ function configure
cp -av "$repo_dir"/programs/server/config* db
cp -av "$repo_dir"/programs/server/user* db
cp -av "$repo_dir"/tests/config db/config.d
cp -av "$script_dir"/query-fuzzer-tweaks-users.xml db/users.d
}
function watchdog
@ -53,12 +54,13 @@ function watchdog
sleep 3600
echo "Fuzzing run has timed out"
killall -9 clickhouse clickhouse-server clickhouse-client
./clickhouse client --query "select elapsed, query from system.processes" ||:
killall -9 clickhouse clickhouse-server clickhouse-client ||:
}
function fuzz
{
./clickhouse server --config-file db/config.xml -- --path db 2>&1 | tail -1000000 > server.log &
./clickhouse server --config-file db/config.xml -- --path db 2>&1 | tail -100000 > server.log &
server_pid=$!
kill -0 $server_pid
while ! ./clickhouse client --query "select 1" && kill -0 $server_pid ; do echo . ; sleep 1 ; done
@ -67,14 +69,15 @@ function fuzz
echo Server started
fuzzer_exit_code=0
./clickhouse client --query-fuzzer-runs=100 \
./clickhouse client --query-fuzzer-runs=1000 \
< <(for f in $(ls ch/tests/queries/0_stateless/*.sql | sort -R); do cat "$f"; echo ';'; done) \
> >(tail -1000000 > fuzzer.log) \
> >(tail -100000 > fuzzer.log) \
2>&1 \
|| fuzzer_exit_code=$?
echo "Fuzzer exit code is $fuzzer_exit_code"
kill -9 $server_pid
./clickhouse client --query "select elapsed, query from system.processes" ||:
kill -9 $server_pid ||:
return $fuzzer_exit_code
}
@ -107,7 +110,14 @@ case "$stage" in
watchdog_pid=$!
fuzzer_exit_code=0
time fuzz || fuzzer_exit_code=$?
kill $watchdog_pid
kill $watchdog_pid ||:
# Debug
date
sleep 10
jobs
pstree -aspgT
exit $fuzzer_exit_code
;&
esac

View File

@ -11,6 +11,7 @@ services:
environment:
MINIO_ACCESS_KEY: minio
MINIO_SECRET_KEY: minio123
MINIO_PROMETHEUS_AUTH_TYPE: public
command: server --address :9001 --certs-dir /certs /data1-1
depends_on:
- proxy1

View File

@ -36,6 +36,9 @@ Action required for every item -- these are errors that must be fixed. The error
#### Slow on client
Action required for every item -- these are errors that must be fixed. This table shows queries that take significantly longer to process on the client than on the server. A possible reason might be sending too much data to the client, e.g., a forgotten `format Null`.
#### Short queries not marked as short
Action required for every item -- these are errors that must be fixed. This table shows queries that are "short" but not explicitly marked as such. "Short" queries are too fast to meaningfully compare performance, because the changes are drowned by the noise. We consider all queries that run faster than 0.02 s to be "short", and only check the performance if they became slower than this threshold. Probably this mode is not what you want, so you have to increase the query run time to be between 1 and 0.1 s, so that the performance can be compared. You do want this "short" mode for queries that complete "immediately", such as some varieties of `select count(*)`. You have to mark them as "short" explicitly by writing `<query short="1">...`. The value of "short" attribute is evaluated as a python expression, and substitutions are performed, so you can write something like `<query short="{column1} = {column2}">select count(*) from table where {column1} > {column2}</query>`, to mark only a particular combination of variables as short.
#### Partial queries
Action required for the cells marked in red. Shows the queries we are unable to run on an old server -- probably because they contain a new function. You should see this table when you add a new function and a performance test for it. Check that the run time and variance are acceptable (run time between 0.1 and 1 seconds, variance below 10%). If not, they will be highlighted in red.

View File

@ -282,6 +282,7 @@ do
sed -n "s/^report-threshold\t/$test_name\t/p" < "$test_file" >> "analyze/report-thresholds.tsv"
sed -n "s/^skipped\t/$test_name\t/p" < "$test_file" >> "analyze/skipped-tests.tsv"
sed -n "s/^display-name\t/$test_name\t/p" < "$test_file" >> "analyze/query-display-names.tsv"
sed -n "s/^short\t/$test_name\t/p" < "$test_file" >> "analyze/marked-short-queries.tsv"
sed -n "s/^partial\t/$test_name\t/p" < "$test_file" >> "analyze/partial-queries.tsv"
done
unset IFS
@ -291,6 +292,9 @@ clickhouse-local --query "
create view query_runs as select * from file('analyze/query-runs.tsv', TSV,
'test text, query_index int, query_id text, version UInt8, time float');
-- Separately process 'partial' queries which we could only run on the new server
-- because they use new functions. We can't make normal stats for them, but still
-- have to show some stats so that the PR author can tweak them.
create view partial_queries as select test, query_index
from file('analyze/partial-queries.tsv', TSV,
'test text, query_index int, servers Array(int)');
@ -303,6 +307,7 @@ create table partial_query_times engine File(TSVWithNamesAndTypes,
group by test, query_index
;
-- Process queries that were run normally, on both servers.
create view left_query_log as select *
from file('left-query-log.tsv', TSVWithNamesAndTypes,
'$(cat "left-query-log.tsv.columns")');
@ -317,7 +322,10 @@ create view query_logs as
select *, 1 version from right_query_log
;
create table query_run_metrics_full engine File(TSV, 'analyze/query-run-metrics-full.tsv')
-- This is a single source of truth on all metrics we have for query runs. The
-- metrics include ProfileEvents from system.query_log, and query run times
-- reported by the perf.py test runner.
create table query_run_metric_arrays engine File(TSV, 'analyze/query-run-metric-arrays.tsv')
as
with (
-- sumMapState with the list of all keys with '-0.' values. Negative zero is because
@ -349,18 +357,29 @@ create table query_run_metrics_full engine File(TSV, 'analyze/query-run-metrics-
where (test, query_index) not in partial_queries
;
create table query_run_metrics engine File(
-- This is just for convenience -- human-readable + easy to make plots.
create table query_run_metrics_denorm engine File(TSV, 'analyze/query-run-metrics-denorm.tsv')
as select test, query_index, metric_names, version, query_id, metric_values
from query_run_metric_arrays
array join metric_names, metric_values
order by test, query_index, metric_names, version, query_id
;
-- This is for statistical processing with eqmed.sql
create table query_run_metrics_for_stats engine File(
TSV, -- do not add header -- will parse with grep
'analyze/query-run-metrics.tsv')
'analyze/query-run-metrics-for-stats.tsv')
as select test, query_index, 0 run, version, metric_values
from query_run_metrics_full
from query_run_metric_arrays
order by test, query_index, run, version
;
-- This is the list of metric names, so that we can join them back after
-- statistical processing.
create table query_run_metric_names engine File(TSV, 'analyze/query-run-metric-names.tsv')
as select metric_names from query_run_metrics_full limit 1
as select metric_names from query_run_metric_arrays limit 1
;
"
" 2> >(tee -a analyze/errors.log 1>&2)
# This is a lateral join in bash... please forgive me.
# We don't have arrayPermute(), so I have to make random permutations with
@ -370,16 +389,16 @@ create table query_run_metric_names engine File(TSV, 'analyze/query-run-metric-n
# for each file. I do this in parallel using GNU parallel.
( set +x # do not bloat the log
IFS=$'\n'
for prefix in $(cut -f1,2 "analyze/query-run-metrics.tsv" | sort | uniq)
for prefix in $(cut -f1,2 "analyze/query-run-metrics-for-stats.tsv" | sort | uniq)
do
file="analyze/tmp/$(echo "$prefix" | sed 's/\t/_/g').tsv"
grep "^$prefix " "analyze/query-run-metrics.tsv" > "$file" &
grep "^$prefix " "analyze/query-run-metrics-for-stats.tsv" > "$file" &
printf "%s\0\n" \
"clickhouse-local \
--file \"$file\" \
--structure 'test text, query text, run int, version UInt8, metrics Array(float)' \
--query \"$(cat "$script_dir/eqmed.sql")\" \
>> \"analyze/query-reports.tsv\"" \
>> \"analyze/query-metric-stats.tsv\"" \
2>> analyze/errors.log \
>> analyze/commands.txt
done
@ -388,6 +407,33 @@ unset IFS
)
parallel --joblog analyze/parallel-log.txt --null < analyze/commands.txt 2>> analyze/errors.log
clickhouse-local --query "
-- Join the metric names back to the metric statistics we've calculated, and make
-- a denormalized table of them -- statistics for all metrics for all queries.
-- The WITH, ARRAY JOIN and CROSS JOIN do not like each other:
-- https://github.com/ClickHouse/ClickHouse/issues/11868
-- https://github.com/ClickHouse/ClickHouse/issues/11757
-- Because of this, we make a view with arrays first, and then apply all the
-- array joins.
create view query_metric_stat_arrays as
with (select * from file('analyze/query-run-metric-names.tsv',
TSV, 'n Array(String)')) as metric_name
select test, query_index, metric_name, left, right, diff, stat_threshold
from file('analyze/query-metric-stats.tsv', TSV, 'left Array(float),
right Array(float), diff Array(float), stat_threshold Array(float),
test text, query_index int') reports
order by test, query_index, metric_name
;
create table query_metric_stats_denorm engine File(TSVWithNamesAndTypes,
'analyze/query-metric-stats-denorm.tsv')
as select test, query_index, metric_name, left, right, diff, stat_threshold
from query_metric_stat_arrays
left array join metric_name, left, right, diff, stat_threshold
order by test, query_index, metric_name
;
" 2> >(tee -a analyze/errors.log 1>&2)
}
# Analyze results
@ -403,58 +449,46 @@ build_log_column_definitions
cat analyze/errors.log >> report/errors.log ||:
cat profile-errors.log >> report/errors.log ||:
short_query_threshold="0.02"
clickhouse-local --query "
create view query_display_names as select * from
file('analyze/query-display-names.tsv', TSV,
'test text, query_index int, query_display_name text')
;
create view partial_query_times as select * from
file('analyze/partial-query-times.tsv', TSVWithNamesAndTypes,
'test text, query_index int, time_stddev float, time_median float')
;
-- Report for partial queries that we could only run on the new server (e.g.
-- queries with new functions added in the tested PR).
create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv')
as select floor(time_median, 3) m, floor(time_stddev / time_median, 3) v,
as select floor(time_median, 3) time,
floor(time_stddev / time_median, 3) relative_time_stddev,
test, query_index, query_display_name
from file('analyze/partial-query-times.tsv', TSVWithNamesAndTypes,
'test text, query_index int, time_stddev float, time_median float') t
from partial_query_times
join query_display_names using (test, query_index)
order by test, query_index
;
-- WITH, ARRAY JOIN and CROSS JOIN do not like each other:
-- https://github.com/ClickHouse/ClickHouse/issues/11868
-- https://github.com/ClickHouse/ClickHouse/issues/11757
-- Because of this, we make a view with arrays first, and then apply all the
-- array joins.
create view query_metric_stat_arrays as
with (select * from file('analyze/query-run-metric-names.tsv',
TSV, 'n Array(String)')) as metric_name
select metric_name, left, right, diff, stat_threshold, test, query_index,
query_display_name
from file ('analyze/query-reports.tsv', TSV, 'left Array(float),
right Array(float), diff Array(float), stat_threshold Array(float),
test text, query_index int') reports
left join query_display_names
on reports.test = query_display_names.test
and reports.query_index = query_display_names.query_index
;
create table query_metric_stats engine File(TSVWithNamesAndTypes,
'report/query-metric-stats.tsv')
as
select metric_name, left, right, diff, stat_threshold, test, query_index,
query_display_name
from query_metric_stat_arrays
left array join metric_name, left, right, diff, stat_threshold
create view query_metric_stats as
select * from file('analyze/query-metric-stats-denorm.tsv',
TSVWithNamesAndTypes,
'test text, query_index int, metric_name text, left float, right float,
diff float, stat_threshold float')
;
-- Main statistics for queries -- query time as reported in query log.
create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv')
as select
-- FIXME Comparison mode doesn't make sense for queries that complete
-- immediately (on the same order of time as noise). We compute average
-- run time between old and new version, and if it is below a threshold,
-- we just skip the query. If there is a significant regression, the
-- average will be above threshold, we'll process it normally and will
-- detect the regression.
(left + right) / 2 < 0.02 as short,
-- Comparison mode doesn't make sense for queries that complete
-- immediately (on the same order of time as noise). If query duration is
-- less that some threshold, we just skip it. If there is a significant
-- regression in such query, the time will exceed the threshold, and we
-- well process it normally and detect the regression.
right < $short_query_threshold as short,
not short and abs(diff) > report_threshold and abs(diff) > stat_threshold as changed_fail,
not short and abs(diff) > report_threshold - 0.05 and abs(diff) > stat_threshold as changed_show,
@ -464,68 +498,28 @@ create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv')
left, right, diff, stat_threshold,
if(report_threshold > 0, report_threshold, 0.10) as report_threshold,
test, query_index, query_display_name
query_metric_stats.test test, query_metric_stats.query_index query_index,
query_display_name
from query_metric_stats
left join file('analyze/report-thresholds.tsv', TSV,
'test text, report_threshold float') thresholds
on query_metric_stats.test = thresholds.test
left join query_display_names
on query_metric_stats.test = query_display_names.test
and query_metric_stats.query_index = query_display_names.query_index
where metric_name = 'server_time'
order by test, query_index, metric_name
;
-- keep the table in old format so that we can analyze new and old data together
create table queries_old_format engine File(TSVWithNamesAndTypes, 'queries.rep')
as select short, changed_fail, unstable_fail, left, right, diff,
stat_threshold, test, query_display_name query
from queries
;
-- save all test runs as JSON for the new comparison page
create table all_query_runs_json engine File(JSON, 'report/all-query-runs.json') as
select test, query_index, query_display_name query,
left, right, diff, stat_threshold, report_threshold,
versions_runs[1] runs_left, versions_runs[2] runs_right
from (
select
test, query_index,
groupArrayInsertAt(runs, version) versions_runs
from (
select
test, query_index, version,
groupArray(metrics[1]) runs
from file('analyze/query-run-metrics.tsv', TSV,
'test text, query_index int, run int, version UInt8, metrics Array(float)')
group by test, query_index, version
)
group by test, query_index
) runs
left join query_display_names
on runs.test = query_display_names.test
and runs.query_index = query_display_names.query_index
left join file('analyze/report-thresholds.tsv',
TSV, 'test text, report_threshold float') thresholds
on runs.test = thresholds.test
left join query_metric_stats
on runs.test = query_metric_stats.test
and runs.query_index = query_metric_stats.query_index
where
query_metric_stats.metric_name = 'server_time'
;
create table changed_perf_tsv engine File(TSV, 'report/changed-perf.tsv') as
create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') as
select left, right, diff, stat_threshold, changed_fail, test, query_index, query_display_name
from queries where changed_show order by abs(diff) desc;
create table unstable_queries_tsv engine File(TSV, 'report/unstable-queries.tsv') as
create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') as
select left, right, diff, stat_threshold, unstable_fail, test, query_index, query_display_name
from queries where unstable_show order by stat_threshold desc;
create table queries_for_flamegraph engine File(TSVWithNamesAndTypes,
'report/queries-for-flamegraph.tsv') as
select test, query_index from queries where unstable_show or changed_show
;
create table test_time_changes_tsv engine File(TSV, 'report/test-time-changes.tsv') as
create table test_time_changes engine File(TSV, 'report/test-time-changes.tsv') as
select test, queries, average_time_change from (
select test, count(*) queries,
sum(left) as left, sum(right) as right,
@ -536,22 +530,22 @@ create table test_time_changes_tsv engine File(TSV, 'report/test-time-changes.ts
)
;
create table unstable_tests_tsv engine File(TSV, 'report/unstable-tests.tsv') as
create table unstable_tests engine File(TSV, 'report/unstable-tests.tsv') as
select test, sum(unstable_show) total_unstable, sum(changed_show) total_changed
from queries
group by test
order by total_unstable + total_changed desc
;
create table test_perf_changes_tsv engine File(TSV, 'report/test-perf-changes.tsv') as
create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes.tsv') as
select test,
queries,
coalesce(total_unstable, 0) total_unstable,
coalesce(total_changed, 0) total_changed,
total_unstable + total_changed total_bad,
coalesce(toString(floor(average_time_change, 3)), '??') average_time_change_str
from test_time_changes_tsv
full join unstable_tests_tsv
from test_time_changes
full join unstable_tests
using test
where (abs(average_time_change) > 0.05 and queries > 5)
or (total_bad > 0)
@ -559,28 +553,28 @@ create table test_perf_changes_tsv engine File(TSV, 'report/test-perf-changes.ts
settings join_use_nulls = 1
;
create table query_time engine Memory as select *
create view total_client_time_per_query as select *
from file('analyze/client-times.tsv', TSV,
'test text, query_index int, client float, server float');
create table wall_clock engine Memory as select *
from file('wall-clock-times.tsv', TSV, 'test text, real float, user float, system float');
create table slow_on_client_tsv engine File(TSV, 'report/slow-on-client.tsv') as
create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') as
select client, server, floor(client/server, 3) p, test, query_display_name
from query_time left join query_display_names using (test, query_index)
from total_client_time_per_query left join query_display_names using (test, query_index)
where p > 1.02 order by p desc;
create table wall_clock_time_per_test engine Memory as select *
from file('wall-clock-times.tsv', TSV, 'test text, real float, user float, system float');
create table test_time engine Memory as
select test, sum(client) total_client_time,
maxIf(client, not short) query_max,
minIf(client, not short) query_min,
count(*) queries, sum(short) short_queries
from query_time full join queries using (test, query_index)
from total_client_time_per_query full join queries using (test, query_index)
group by test;
create table test_times_tsv engine File(TSV, 'report/test-times.tsv') as
select wall_clock.test, real,
create table test_times_report engine File(TSV, 'report/test-times.tsv') as
select wall_clock_time_per_test.test, real,
floor(total_client_time, 3),
queries,
short_queries,
@ -590,23 +584,62 @@ create table test_times_tsv engine File(TSV, 'report/test-times.tsv') as
from test_time
-- wall clock times are also measured for skipped tests, so don't
-- do full join
left join wall_clock using test
left join wall_clock_time_per_test using test
order by avg_real_per_query desc;
-- report for all queries page, only main metric
create table all_tests_tsv engine File(TSV, 'report/all-queries.tsv') as
create table all_tests_report engine File(TSV, 'report/all-queries.tsv') as
select changed_fail, unstable_fail,
left, right, diff,
floor(left > right ? left / right : right / left, 3),
stat_threshold, test, query_index, query_display_name
from queries order by test, query_index;
-- queries for which we will build flamegraphs (see below)
create table queries_for_flamegraph engine File(TSVWithNamesAndTypes,
'report/queries-for-flamegraph.tsv') as
select test, query_index from queries where unstable_show or changed_show
;
-- List of queries that have 'short' duration, but are not marked as 'short' by
-- the test author (we report them).
create table unmarked_short_queries_report
engine File(TSV, 'report/unmarked-short-queries.tsv')
as select time, test, query_index, query_display_name
from (
select right time, test, query_index from queries where short
union all
select time_median, test, query_index from partial_query_times
where time_median < $short_query_threshold
) times
left join query_display_names
on times.test = query_display_names.test
and times.query_index = query_display_names.query_index
where (test, query_index) not in
(select * from file('analyze/marked-short-queries.tsv', TSV,
'test text, query_index int'))
order by test, query_index
;
--------------------------------------------------------------------------------
-- various compatibility data formats follow, not related to the main report
-- keep the table in old format so that we can analyze new and old data together
create table queries_old_format engine File(TSVWithNamesAndTypes, 'queries.rep')
as select short, changed_fail, unstable_fail, left, right, diff,
stat_threshold, test, query_display_name query
from queries
;
-- new report for all queries with all metrics (no page yet)
create table all_query_metrics_tsv engine File(TSV, 'report/all-query-metrics.tsv') as
select metric_name, left, right, diff,
floor(left > right ? left / right : right / left, 3),
stat_threshold, test, query_index, query_display_name
from query_metric_stats
left join query_display_names
on query_metric_stats.test = query_display_names.test
and query_metric_stats.query_index = query_display_names.query_index
order by test, query_index;
" 2> >(tee -a report/errors.log 1>&2)
@ -634,7 +667,8 @@ create view query_display_names as select * from
create table unstable_query_runs engine File(TSVWithNamesAndTypes,
'unstable-query-runs.$version.rep') as
select test, query_index, query_display_name, query_id
select query_runs.test test, query_runs.query_index query_index,
query_display_name, query_id
from query_runs
join queries_for_flamegraph on
query_runs.test = queries_for_flamegraph.test

View File

@ -37,21 +37,44 @@ available_parameters = {} # { 'table': ['hits_10m', 'hits_100m'], ... }
for e in subst_elems:
available_parameters[e.find('name').text] = [v.text for v in e.findall('values/value')]
# Take care to keep the order of queries -- sometimes we have DROP IF EXISTS
# Takes parallel lists of templates, substitutes them with all combos of
# parameters. The set of parameters is determined based on the first list.
# Note: keep the order of queries -- sometimes we have DROP IF EXISTS
# followed by CREATE in create queries section, so the order matters.
def substitute_parameters(query_templates):
result = []
for q in query_templates:
def substitute_parameters(query_templates, other_templates = []):
query_results = []
other_results = [[]] * (len(other_templates))
for i, q in enumerate(query_templates):
keys = set(n for _, n, _, _ in string.Formatter().parse(q) if n)
values = [available_parameters[k] for k in keys]
result.extend([
q.format(**dict(zip(keys, values_combo)))
for values_combo in itertools.product(*values)])
return result
combos = itertools.product(*values)
for c in combos:
with_keys = dict(zip(keys, c))
query_results.append(q.format(**with_keys))
for j, t in enumerate(other_templates):
other_results[j].append(t[i].format(**with_keys))
if len(other_templates):
return query_results, other_results
else:
return query_results
# Build a list of test queries, substituting parameters to query templates,
# and reporting the queries marked as short.
test_queries = []
for e in root.findall('query'):
new_queries = []
if 'short' in e.attrib:
new_queries, [is_short] = substitute_parameters([e.text], [[e.attrib['short']]])
for i, s in enumerate(is_short):
# Don't print this if we only need to print the queries.
if eval(s) and not args.print_queries:
print(f'short\t{i + len(test_queries)}')
else:
new_queries = substitute_parameters([e.text])
test_queries += new_queries
# Build a list of test queries, processing all substitutions
test_query_templates = [q.text for q in root.findall('query')]
test_queries = substitute_parameters(test_query_templates)
# If we're only asked to print the queries, do that and exit
if args.print_queries:
@ -166,7 +189,7 @@ for conn_index, c in enumerate(connections):
c.execute(q)
print(f'fill\t{conn_index}\t{c.last_query.elapsed}\t{tsv_escape(q)}')
# Run test queries
# Run test queries.
for query_index, q in enumerate(test_queries):
query_prefix = f'{test_name}.query{query_index}'

View File

@ -196,6 +196,12 @@ if args.report == 'main':
['Client time,&nbsp;s', 'Server time,&nbsp;s', 'Ratio', 'Test', 'Query'],
slow_on_client_rows)
unmarked_short_rows = tsvRows('report/unmarked-short-queries.tsv')
error_tests += len(unmarked_short_rows)
printSimpleTable('Short queries not marked as short',
['New client time, s', 'Test', '#', 'Query'],
unmarked_short_rows)
def print_partial():
rows = tsvRows('report/partial-queries-report.tsv')
if not rows:

View File

@ -23,28 +23,7 @@ RUN apt-get update -y \
brotli
COPY ./stress /stress
COPY run.sh /
ENV DATASETS="hits visits"
CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \
dpkg -i package_folder/clickhouse-common-static-dbg_*.deb; \
dpkg -i package_folder/clickhouse-server_*.deb; \
dpkg -i package_folder/clickhouse-client_*.deb; \
dpkg -i package_folder/clickhouse-test_*.deb; \
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \
echo "TSAN_OPTIONS='halt_on_error=1 history_size=7 ignore_noninstrumented_modules=1 verbosity=1'" >> /etc/environment; \
echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \
echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment; \
service clickhouse-server start && sleep 5 \
&& /s3downloader --dataset-names $DATASETS \
&& chmod 777 -R /var/lib/clickhouse \
&& clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" \
&& clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" \
&& service clickhouse-server restart && sleep 5 \
&& clickhouse-client --query "SHOW TABLES FROM datasets" \
&& clickhouse-client --query "SHOW TABLES FROM test" \
&& clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" \
&& clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" \
&& clickhouse-client --query "SHOW TABLES FROM test" \
&& ./stress --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION"
CMD ["/bin/bash", "/run.sh"]

56
docker/test/stress/run.sh Executable file
View File

@ -0,0 +1,56 @@
#!/bin/bash
set -x
dpkg -i package_folder/clickhouse-common-static_*.deb
dpkg -i package_folder/clickhouse-common-static-dbg_*.deb
dpkg -i package_folder/clickhouse-server_*.deb
dpkg -i package_folder/clickhouse-client_*.deb
dpkg -i package_folder/clickhouse-test_*.deb
function wait_server()
{
counter=0
until clickhouse-client --query "SELECT 1"
do
if [ "$counter" -gt 120 ]
then
break
fi
sleep 0.5
counter=$(($counter + 1))
done
}
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/
echo "TSAN_OPTIONS='halt_on_error=1 history_size=7 ignore_noninstrumented_modules=1 verbosity=1'" >> /etc/environment
echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment
echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment
service clickhouse-server start
wait_server
/s3downloader --dataset-names $DATASETS
chmod 777 -R /var/lib/clickhouse
clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary"
clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test"
service clickhouse-server restart
wait_server
clickhouse-client --query "SHOW TABLES FROM datasets"
clickhouse-client --query "SHOW TABLES FROM test"
clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits"
clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits"
clickhouse-client --query "SHOW TABLES FROM test"
./stress --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION"
service clickhouse-server restart
wait_server
clickhouse-client --query "SELECT 'Server successfuly started'" > /test_output/alive_check.txt || echo 'Server failed to start' > /test_output/alive_check.txt

View File

@ -41,15 +41,6 @@ def run_func_test(cmd, output_prefix, num_processes, skip_tests_option):
return pipes
def check_clickhouse_alive(cmd):
try:
logging.info("Checking ClickHouse still alive")
check_call("{} --query \"select 'Still alive'\"".format(cmd), shell=True)
return True
except:
return False
if __name__ == "__main__":
logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s')
parser = argparse.ArgumentParser(description="ClickHouse script for running stresstest")
@ -65,29 +56,18 @@ if __name__ == "__main__":
args = parser.parse_args()
func_pipes = []
perf_process = None
try:
perf_process = run_perf_test(args.perf_test_cmd, args.perf_test_xml_path, args.output_folder)
func_pipes = run_func_test(args.test_cmd, args.output_folder, args.num_parallel, args.skip_func_tests)
perf_process = run_perf_test(args.perf_test_cmd, args.perf_test_xml_path, args.output_folder)
func_pipes = run_func_test(args.test_cmd, args.output_folder, args.num_parallel, args.skip_func_tests)
logging.info("Will wait functests to finish")
while True:
retcodes = []
for p in func_pipes:
if p.poll() is not None:
retcodes.append(p.returncode)
if len(retcodes) == len(func_pipes):
break
logging.info("Finished %s from %s processes", len(retcodes), len(func_pipes))
time.sleep(5)
logging.info("Will wait functests to finish")
while True:
retcodes = []
for p in func_pipes:
if p.poll() is not None:
retcodes.append(p.returncode)
if len(retcodes) == len(func_pipes):
break
logging.info("Finished %s from %s processes", len(retcodes), len(func_pipes))
time.sleep(5)
if not check_clickhouse_alive(args.client_cmd):
raise Exception("Stress failed, results in logs")
else:
logging.info("Stress is ok")
except Exception as ex:
raise ex
finally:
if os.path.exists(args.server_log_folder):
logging.info("Copying server log files")
for log_file in os.listdir(args.server_log_folder):
shutil.copy(os.path.join(args.server_log_folder, log_file), os.path.join(args.output_folder, log_file))
logging.info("Stress test finished")

View File

@ -0,0 +1,122 @@
---
toc_priority: 6
toc_title: RabbitMQ
---
# RabbitMQ Engine {#rabbitmq-engine}
This engine allows integrating ClickHouse with [RabbitMQ](https://www.rabbitmq.com).
RabbitMQ lets you:
- Publish or subscribe to data flows.
- Process streams as they become available.
## Creating a Table {#table_engine-rabbitmq-creating-a-table}
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
(
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2],
...
) ENGINE = RabbitMQ SETTINGS
rabbitmq_host_port = 'host:port',
rabbitmq_exchange_name = 'exchange_name',
rabbitmq_format = 'data_format'[,]
[rabbitmq_exchange_type = 'exchange_type',]
[rabbitmq_routing_key_list = 'key1,key2,...',]
[rabbitmq_row_delimiter = 'delimiter_symbol',]
[rabbitmq_num_consumers = N,]
[rabbitmq_num_queues = N,]
[rabbitmq_transactional_channel = 0]
```
Required parameters:
- `rabbitmq_host_port` host:port (for example, `localhost:5672`).
- `rabbitmq_exchange_name` RabbitMQ exchange name.
- `rabbitmq_format` Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section.
Optional parameters:
- `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_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. Single queue can contain up to 50K messages at the same time.
- `rabbitmq_transactional_channel` Wrap insert queries in transactions. Default: `0`.
Required configuration:
The RabbitMQ server configuration should be added using the ClickHouse config file.
``` xml
<rabbitmq>
<username>root</username>
<password>clickhouse</password>
</rabbitmq>
```
Example:
``` sql
CREATE TABLE queue (
key UInt64,
value UInt64
) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'localhost:5672',
rabbitmq_exchange_name = 'exchange1',
rabbitmq_format = 'JSONEachRow',
rabbitmq_num_consumers = 5;
```
## Description {#description}
`SELECT` is not particularly useful for reading messages (except for debugging), because each message can be read only once. It is more practical to create real-time threads using materialized views. To do this:
1. Use the engine to create a RabbitMQ consumer and consider it a data stream.
2. Create a table with the desired structure.
3. Create a materialized view that converts data from the engine and puts it into a previously created table.
When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the background. This allows you to continually receive messages from RabbitMQ and convert them to the required format using `SELECT`.
One RabbitMQ table can have as many materialized views as you like.
Data can be channeled based on `rabbitmq_exchange_type` and the specified `rabbitmq_routing_key_list`.
There can be no more than one exchange per table. One exchange can be shared between multiple tables - it enables routing into multiple tables at the same time.
Exchange type options:
- `direct` - Routing is based on exact matching of keys. Example table key list: `key1,key2,key3,key4,key5`, message key can eqaul any of them.
- `fanout` - Routing to all tables (where exchange name is the same) regardless of the keys.
- `topic` - Routing is based on patterns with dot-separated keys. Examples: `*.logs`, `records.*.*.2020`, `*.2018,*.2019,*.2020`.
- `headers` - Routing is based on `key=value` matches with a setting `x-match=all` or `x-match=any`. Example table key list: `x-match=all,format=logs,type=report,year=2020`.
- `consistent-hash` - Data is evenly distributed between all bound tables (where 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 then any other, especially when `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` parameters are set.
If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` parameters 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).
Example:
``` sql
CREATE TABLE queue (
key UInt64,
value UInt64
) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'localhost:5672',
rabbitmq_exchange_name = 'exchange1',
rabbitmq_exchange_type = 'headers',
rabbitmq_routing_key_list = 'format=logs,type=report,year=2020',
rabbitmq_format = 'JSONEachRow',
rabbitmq_num_consumers = 5;
CREATE TABLE daily (key UInt64, value UInt64)
ENGINE = MergeTree();
CREATE MATERIALIZED VIEW consumer TO daily
AS SELECT key, value FROM queue;
SELECT key, value FROM daily ORDER BY key;
```

View File

@ -26,7 +26,7 @@ Engines:
During `INSERT` queries, the table is locked, and other queries for reading and writing data both wait for the table to unlock. If there are no data writing queries, any number of data reading queries can be performed concurrently.
- Do not support [mutation](../../../sql-reference/statements/alter.md#alter-mutations) operations.
- Do not support [mutations](../../../sql-reference/statements/alter/index.md#alter-mutations).
- Do not support indexes.

View File

@ -116,10 +116,10 @@ drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached
The folders 201901\_1\_1\_0, 201901\_1\_7\_1 and so on are the directories of the parts. Each part relates to a corresponding partition and contains data just for a certain month (the table in this example has partitioning by month).
The `detached` directory contains parts that were detached from the table using the [DETACH](../../../sql-reference/statements/alter.md#alter_detach-partition) query. The corrupted parts are also moved to this directory, instead of being deleted. The server does not use the parts from the `detached` directory. You can add, delete, or modify the data in this directory at any time the server will not know about this until you run the [ATTACH](../../../sql-reference/statements/alter.md#alter_attach-partition) query.
The `detached` directory contains parts that were detached from the table using the [DETACH](../../../sql-reference/statements/alter/partition.md#alter_detach-partition) query. The corrupted parts are also moved to this directory, instead of being deleted. The server does not use the parts from the `detached` directory. You can add, delete, or modify the data in this directory at any time the server will not know about this until you run the [ATTACH](../../../sql-reference/statements/alter/partition.md#alter_attach-partition) query.
Note that on the operating server, you cannot manually change the set of parts or their data on the file system, since the server will not know about it. For non-replicated tables, you can do this when the server is stopped, but it isnt recommended. For replicated tables, the set of parts cannot be changed in any case.
ClickHouse allows you to perform operations with the partitions: delete them, copy from one table to another, or create a backup. See the list of all operations in the section [Manipulations With Partitions and Parts](../../../sql-reference/statements/alter.md#alter_manipulations-with-partitions).
ClickHouse allows you to perform operations with the partitions: delete them, copy from one table to another, or create a backup. See the list of all operations in the section [Manipulations With Partitions and Parts](../../../sql-reference/statements/alter/partition.md#alter_manipulations-with-partitions).
[Original article](https://clickhouse.tech/docs/en/operations/table_engines/custom_partitioning_key/) <!--hide-->

View File

@ -212,7 +212,7 @@ This feature is helpful when using the [SummingMergeTree](../../../engines/table
In this case it makes sense to leave only a few columns in the primary key that will provide efficient range scans and add the remaining dimension columns to the sorting key tuple.
[ALTER](../../../sql-reference/statements/alter.md) of the sorting key is a lightweight operation because when a new column is simultaneously added to the table and to the sorting key, existing data parts dont need to be changed. Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, the data is sorted by both the old and new sorting keys at the moment of table modification.
[ALTER](../../../sql-reference/statements/alter/index.md) of the sorting key is a lightweight operation because when a new column is simultaneously added to the table and to the sorting key, existing data parts dont need to be changed. Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, the data is sorted by both the old and new sorting keys at the moment of table modification.
### Use of Indexes and Partitions in Queries {#use-of-indexes-and-partitions-in-queries}
@ -490,7 +490,7 @@ If you perform the `SELECT` query between merges, you may get expired data. To a
`MergeTree` family table engines can store data on multiple block devices. For example, it can be useful when the data of a certain table are implicitly split into “hot” and “cold”. The most recent data is regularly requested but requires only a small amount of space. On the contrary, the fat-tailed historical data is requested rarely. If several disks are available, the “hot” data may be located on fast disks (for example, NVMe SSDs or in memory), while the “cold” data - on relatively slow ones (for example, HDD).
Data part is the minimum movable unit for `MergeTree`-engine tables. The data belonging to one part are stored on one disk. Data parts can be moved between disks in the background (according to user settings) as well as by means of the [ALTER](../../../sql-reference/statements/alter.md#alter_move-partition) queries.
Data part is the minimum movable unit for `MergeTree`-engine tables. The data belonging to one part are stored on one disk. Data parts can be moved between disks in the background (according to user settings) as well as by means of the [ALTER](../../../sql-reference/statements/alter/partition.md#alter_move-partition) queries.
### Terms {#terms}
@ -636,9 +636,9 @@ The number of threads performing background moves of data parts can be changed b
In the case of `MergeTree` tables, data is getting to disk in different ways:
- As a result of an insert (`INSERT` query).
- During background merges and [mutations](../../../sql-reference/statements/alter.md#alter-mutations).
- During background merges and [mutations](../../../sql-reference/statements/alter/index.md#alter-mutations).
- When downloading from another replica.
- As a result of partition freezing [ALTER TABLE … FREEZE PARTITION](../../../sql-reference/statements/alter.md#alter_freeze-partition).
- As a result of partition freezing [ALTER TABLE … FREEZE PARTITION](../../../sql-reference/statements/alter/partition.md#alter_freeze-partition).
In all these cases except for mutations and partition freezing, a part is stored on a volume and a disk according to the given storage policy:
@ -650,7 +650,7 @@ Under the hood, mutations and partition freezing make use of [hard links](https:
In the background, parts are moved between volumes on the basis of the amount of free space (`move_factor` parameter) according to the order the volumes are declared in the configuration file.
Data is never transferred from the last one and into the first one. One may use system tables [system.part\_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (field `type = MOVE_PART`) and [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (fields `path` and `disk`) to monitor background moves. Also, the detailed information can be found in server logs.
User can force moving a part or a partition from one volume to another using the query [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition), all the restrictions for background operations are taken into account. The query initiates a move on its own and does not wait for background operations to be completed. User will get an error message if not enough free space is available or if any of the required conditions are not met.
User can force moving a part or a partition from one volume to another using the query [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter/partition.md#alter_move-partition), all the restrictions for background operations are taken into account. The query initiates a move on its own and does not wait for background operations to be completed. User will get an error message if not enough free space is available or if any of the required conditions are not met.
Moving data does not interfere with data replication. Therefore, different storage policies can be specified for the same table on different replicas.

View File

@ -19,7 +19,7 @@ Replication works at the level of an individual table, not the entire server. A
Replication does not depend on sharding. Each shard has its own independent replication.
Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](../../../sql-reference/statements/alter.md#query_language_queries_alter)).
Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](../../../sql-reference/statements/alter/index.md#query_language_queries_alter)).
`CREATE`, `DROP`, `ATTACH`, `DETACH` and `RENAME` queries are executed on a single server and are not replicated:

View File

@ -27,16 +27,16 @@ ClickHouse doesnt have real-time point deletes like in [OLTP](https://en.wiki
This is the most common approach to make your system based on ClickHouse [GDPR](https://gdpr-info.eu)-compliant.
More details on [mutations](../../sql-reference/statements/alter.md#alter-mutations).
More details on [mutations](../../sql-reference/statements/alter/index.md#alter-mutations).
## DROP PARTITION {#drop-partition}
`ALTER TABLE ... DROP PARTITION` provides a cost-efficient way to drop a whole partition. Its not that flexible and needs proper partitioning scheme configured on table creation, but still covers most common cases. Like mutations need to be executed from an external system for regular use.
More details on [manipulating partitions](../../sql-reference/statements/alter.md#alter_drop-partition).
More details on [manipulating partitions](../../sql-reference/statements/alter/partition.md#alter_drop-partition).
## TRUNCATE {#truncate}
Its rather radical to drop all data from a table, but in some cases it might be exactly what you need.
More details on [table truncation](../../sql-reference/statements/alter.md#alter_drop-partition).
More details on [table truncation](../../sql-reference/statements/alter/partition.md#alter_drop-partition).

View File

@ -13,7 +13,7 @@ The supported formats are:
| Format | Input | Output |
|-----------------------------------------------------------------|-------|--------|
| [TabSeparated](#tabseparated) | ✔ | ✔ |
| [TabSeparatedRaw](#tabseparatedraw) | | ✔ |
| [TabSeparatedRaw](#tabseparatedraw) | | ✔ |
| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ |
| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ |
| [Template](#format-template) | ✔ | ✔ |
@ -143,7 +143,7 @@ SELECT * FROM nestedt FORMAT TSV
## TabSeparatedRaw {#tabseparatedraw}
Differs from `TabSeparated` format in that the rows are written without escaping.
This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table).
When parsing with this format, tabs or linefeeds are not allowed in each field.
This format is also available under the name `TSVRaw`.

View File

@ -44,6 +44,7 @@ toc_title: Adopters
| <a href="https://www.lbl.gov" class="favicon">Lawrence Berkeley National Laboratory</a> | Research | Traffic analysis | 1 server | 11.8 TiB | [Slides in English, April 2019](https://www.smitasin.com/presentations/2019-04-17_DOE-NSM.pdf) |
| <a href="https://lifestreet.com/" class="favicon">LifeStreet</a> | Ad network | Main product | 75 servers (3 replicas) | 5.27 PiB | [Blog post in Russian, February 2017](https://habr.com/en/post/322620/) |
| <a href="https://mcs.mail.ru/" class="favicon">Mail.ru Cloud Solutions</a> | Cloud services | Main product | — | — | [Article in Russian](https://mcs.mail.ru/help/db-create/clickhouse#) |
| <a href="https://tech.mymarilyn.ru" class="favicon">Marilyn</a> | Advertising | Statistics | — | — | [Talk in Russian, June 2017](https://www.youtube.com/watch?v=iXlIgx2khwc) |
| <a href="https://www.messagebird.com" class="favicon">MessageBird</a> | Telecommunications | Statistics | — | — | [Slides in English, November 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) |
| <a href="https://www.mgid.com/" class="favicon">MGID</a> | Ad network | Web-analytics | — | — | [Blog post in Russian, April 2020](http://gs-studio.com/news-about-it/32777----clickhouse---c) |
| <a href="https://www.oneapm.com/" class="favicon">OneAPM</a> | Monitorings and Data Analysis | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) |

View File

@ -59,7 +59,7 @@ Privileges can be granted to a user account by the [GRANT](../sql-reference/stat
Management queries:
- [CREATE USER](../sql-reference/statements/create/user.md)
- [ALTER USER](../sql-reference/statements/alter.md#alter-user-statement)
- [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)
@ -85,7 +85,7 @@ Role contains:
Management queries:
- [CREATE ROLE](../sql-reference/statements/create/role.md)
- [ALTER ROLE](../sql-reference/statements/alter.md#alter-role-statement)
- [ALTER ROLE](../sql-reference/statements/alter/role.md#alter-role-statement)
- [DROP ROLE](../sql-reference/statements/drop.md)
- [SET ROLE](../sql-reference/statements/set-role.md)
- [SET DEFAULT ROLE](../sql-reference/statements/set-role.md#set-default-role-statement)
@ -100,7 +100,7 @@ Row policy is a filter that defines which of the rows are available to a user or
Management queries:
- [CREATE ROW POLICY](../sql-reference/statements/create/row-policy.md)
- [ALTER ROW POLICY](../sql-reference/statements/alter.md#alter-row-policy-statement)
- [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)
@ -111,7 +111,7 @@ Settings profile is a collection of [settings](../operations/settings/index.md).
Management queries:
- [CREATE SETTINGS PROFILE](../sql-reference/statements/create/settings-profile.md#create-settings-profile-statement)
- [ALTER SETTINGS PROFILE](../sql-reference/statements/alter.md#alter-settings-profile-statement)
- [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)
@ -124,7 +124,7 @@ Quota contains a set of limits for some durations, as well as a list of roles an
Management queries:
- [CREATE QUOTA](../sql-reference/statements/create/quota.md)
- [ALTER QUOTA](../sql-reference/statements/alter.md#alter-quota-statement)
- [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)

View File

@ -33,7 +33,7 @@ For smaller volumes of data, a simple `INSERT INTO ... SELECT ...` to remote tab
ClickHouse allows using the `ALTER TABLE ... FREEZE PARTITION ...` query to create a local copy of table partitions. This is implemented using hardlinks to the `/var/lib/clickhouse/shadow/` folder, so it usually does not consume extra disk space for old data. The created copies of files are not handled by ClickHouse server, so you can just leave them there: you will have a simple backup that doesnt require any additional external system, but it will still be prone to hardware issues. For this reason, its better to remotely copy them to another location and then remove the local copies. Distributed filesystems and object stores are still a good options for this, but normal attached file servers with a large enough capacity might work as well (in this case the transfer will occur via the network filesystem or maybe [rsync](https://en.wikipedia.org/wiki/Rsync)).
Data can be restored from backup using the `ALTER TABLE ... ATTACH PARTITION ...`
For more information about queries related to partition manipulations, see the [ALTER documentation](../sql-reference/statements/alter.md#alter_manipulations-with-partitions).
For more information about queries related to partition manipulations, see the [ALTER documentation](../sql-reference/statements/alter/partition.md#alter_manipulations-with-partitions).
A third-party tool is available to automate this approach: [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup).

View File

@ -1599,13 +1599,13 @@ Default value: 268435456.
- [min\_insert\_block\_size\_bytes](#min-insert-block-size-bytes)
## output_format_pretty_grid_charset {#output-format-pretty-grid-charset}
## output\_format\_pretty\_grid\_charset {#output-format-pretty-grid-charset}
Allows to change a charset which is used for printing grids borders. Available charsets are following: UTF-8, ASCII.
**Example**
**Example**
```text
``` text
SET output_format_pretty_grid_charset = 'UTF-8';
SELECT * FROM a;
┌─a─┐
@ -1619,5 +1619,4 @@ SELECT * FROM a;
+---+
```
[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) <!-- hide -->

View File

@ -2,8 +2,8 @@
Contains information about detached parts of [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables. The `reason` column specifies why the part was detached.
For user-detached parts, the reason is empty. Such parts can be attached with [ALTER TABLE ATTACH PARTITION\|PART](../../sql-reference/statements/alter.md#alter_attach-partition) command.
For user-detached parts, the reason is empty. Such parts can be attached with [ALTER TABLE ATTACH PARTITION\|PART](../../sql-reference/statements/alter/partition.md#alter_attach-partition) command.
For the description of other columns, see [system.parts](../../operations/system-tables/parts.md#system_tables-parts).
If part name is invalid, values of some columns may be `NULL`. Such parts can be deleted with [ALTER TABLE DROP DETACHED PART](../../sql-reference/statements/alter.md#alter_drop-detached).
If part name is invalid, values of some columns may be `NULL`. Such parts can be deleted with [ALTER TABLE DROP DETACHED PART](../../sql-reference/statements/alter/partition.md#alter_drop-detached).

View File

@ -22,7 +22,7 @@ Most of system tables store their data in RAM. A ClickHouse server creates such
Unlike other system tables, the system tables [metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query\_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one.
By default, table growth is unlimited. To control a size of a table, you can use [TTL](../../sql-reference/statements/alter.md#manipulations-with-table-ttl) settings for removing outdated log records. Also you can use the partitioning feature of `MergeTree`-engine tables.
By default, table growth is unlimited. To control a size of a table, you can use [TTL](../../sql-reference/statements/alter/ttl.md#manipulations-with-table-ttl) settings for removing outdated log records. Also you can use the partitioning feature of `MergeTree`-engine tables.
## Sources of System Metrics {#system-tables-sources-of-system-metrics}

View File

@ -1,6 +1,6 @@
# system.mutations {#system_tables-mutations}
The table contains information about [mutations](../../sql-reference/statements/alter.md#alter-mutations) of MergeTree tables and their progress. Each mutation command is represented by a single row. The table has the following columns:
The table contains information about [mutations](../../sql-reference/statements/alter/index.md#alter-mutations) of MergeTree tables and their progress. Each mutation command is represented by a single row. The table has the following columns:
**database**, **table** - The name of the database and table to which the mutation was applied.

View File

@ -10,7 +10,7 @@ The `system.part_log` table contains the following columns:
- `NEW_PART` — Inserting of a new data part.
- `MERGE_PARTS` — Merging of data parts.
- `DOWNLOAD_PART` — Downloading a data part.
- `REMOVE_PART` — Removing or detaching a data part using [DETACH PARTITION](../../sql-reference/statements/alter.md#alter_detach-partition).
- `REMOVE_PART` — Removing or detaching a data part using [DETACH PARTITION](../../sql-reference/statements/alter/partition.md#alter_detach-partition).
- `MUTATE_PART` — Mutating of a data part.
- `MOVE_PART` — Moving the data part from the one disk to another one.
- `event_date` (Date) — Event date.

View File

@ -6,7 +6,7 @@ Each row describes one data part.
Columns:
- `partition` (String) The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter.md#query_language_queries_alter) query.
- `partition` (String) The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md#query_language_queries_alter) query.
Formats:
@ -57,7 +57,7 @@ Columns:
- `primary_key_bytes_in_memory_allocated` (`UInt64`) The amount of memory (in bytes) reserved for primary key values.
- `is_frozen` (`UInt8`) Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup doesnt exist. For more details, see [FREEZE PARTITION](../../sql-reference/statements/alter.md#alter_freeze-partition)
- `is_frozen` (`UInt8`) Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup doesnt exist. For more details, see [FREEZE PARTITION](../../sql-reference/statements/alter/partition.md#alter_freeze-partition)
- `database` (`String`) Name of the database.

View File

@ -0,0 +1,17 @@
# system.quota\_limits {#system_tables-quota_limits}
Contains information about maximums for all intervals of all quotas. Any number of rows or zero can correspond to one quota.
Columns:
- `quota_name` ([String](../../sql-reference/data-types/string.md)) — Quota name.
- `duration` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Length of the time interval for calculating resource consumption, in seconds.
- `is_randomized_interval` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Logical value. It shows whether the interval is randomized. Interval always starts at the same time if it is not randomized. For example, an interval of 1 minute always starts at an integer number of minutes (i.e. it can start at 11:20:00, but it never starts at 11:20:01), an interval of one day always starts at midnight UTC. If interval is randomized, the very first interval starts at random time, and subsequent intervals starts one by one. Values:
- `0` — Interval is not randomized.
- `1` — Interval is randomized.
- `max_queries` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of queries.
- `max_errors` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of errors.
- `max_result_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of result rows.
- `max_result_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of RAM volume in bytes used to store a queries result.
- `max_read_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of rows read from all tables and table functions participated in queries.
- `max_read_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of bytes read from all tables and table functions participated in queries.
- `max_execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — Maximum of the query execution time, in seconds.

View File

@ -0,0 +1,24 @@
# system.quota\_usage {#system_tables-quota_usage}
Quota usage by the current user: how much is used and how much is left.
Columns:
- `quota_name` ([String](../../sql-reference/data-types/string.md)) — Quota name.
- `quota_key`([String](../../sql-reference/data-types/string.md)) — Key value. For example, if keys = \[`ip address`\], `quota_key` may have a value 192.168.1.1.
- `start_time`([Nullable](../../sql-reference/data-types/nullable.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Start time for calculating resource consumption.
- `end_time`([Nullable](../../sql-reference/data-types/nullable.md)([DateTime](../../sql-reference/data-types/datetime.md))) — End time for calculating resource consumption.
- `duration` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Length of the time interval for calculating resource consumption, in seconds.
- `queries` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The total number of requests on this interval.
- `max_queries` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of requests.
- `errors` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The number of queries that threw an exception.
- `max_errors` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of errors.
- `result_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The total number of rows given as a result.
- `max_result_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of result rows.
- `result_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — RAM volume in bytes used to store a queries result.
- `max_result_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum RAM volume used to store a queries result, in bytes.
- `read_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The total number of source rows read from tables for running the query on all remote servers.
- `max_read_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of rows read from all tables and table functions participated in queries.
- `read_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The total number of bytes read from all tables and table functions participated in queries.
- `max_read_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum of bytes read from all tables and table functions.
- `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.

View File

@ -0,0 +1,21 @@
# system.quotas {#system_tables-quotas}
Contains information about [quotas](../../operations/system-tables/quotas.md).
Columns:
- `name` ([String](../../sql-reference/data-types/string.md)) — Quota name.
- `id` ([UUID](../../sql-reference/data-types/uuid.md)) — Quota ID.
- `storage`([String](../../sql-reference/data-types/string.md)) — Storage of quotas. Possible value: “users.xml” if a quota configured in the users.xml file, “disk” if a quota configured by an SQL-query.
- `keys` ([Array](../../sql-reference/data-types/array.md)([Enum8](../../sql-reference/data-types/enum.md))) — Key specifies how the quota should be shared. If two connections use the same quota and key, they share the same amounts of resources. Values:
- `[]` — All users share the same quota.
- `['user_name']` — Connections with the same user name share the same quota.
- `['ip_address']` — Connections from the same IP share the same quota.
- `['client_key']` — Connections with the same key share the same quota. A key must be explicitly provided by a client. When using [clickhouse-client](../../interfaces/cli.md), pass a key value in the `--quota-key` parameter, or use the `quota_key` parameter in the client configuration file. When using HTTP interface, use the `X-ClickHouse-Quota` header.
- `['user_name', 'client_key']` — Connections with the same `client_key` share the same quota. If a key isnt provided by a client, the qouta is tracked for `user_name`.
- `['client_key', 'ip_address']` — Connections with the same `client_key` share the same quota. If a key isnt provided by a client, the qouta is tracked for `ip_address`.
- `durations` ([Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Time interval lengths in seconds.
- `apply_to_all` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Logical value. It shows which users the quota is applied to. Values:
- `0` — The quota applies to users specify in the `apply_to_list`.
- `1` — The quota applies to all users except those listed in `apply_to_except`.
- `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.

View File

@ -0,0 +1,25 @@
# system.quotas\_usage {#system_tables-quotas_usage}
Quota usage by all users.
Columns:
- `quota_name` ([String](../../sql-reference/data-types/string.md)) — Quota name.
- `quota_key` ([String](../../sql-reference/data-types/string.md)) — Key value.
- `is_current` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Quota usage for current user.
- `start_time` ([Nullable](../../sql-reference/data-types/nullable.md)([DateTime](../../sql-reference/data-types/datetime.md)))) — Start time for calculating resource consumption.
- `end_time` ([Nullable](../../sql-reference/data-types/nullable.md)([DateTime](../../sql-reference/data-types/datetime.md)))) — End time for calculating resource consumption.
- `duration` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt32](../../sql-reference/data-types/int-uint.md))) — Length of the time interval for calculating resource consumption, in seconds.
- `queries` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The total number of requests in this interval.
- `max_queries` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of requests.
- `errors` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The number of queries that threw an exception.
- `max_errors` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of errors.
- `result_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The total number of rows given as a result.
- `max_result_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum of source rows read from tables.
- `result_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — RAM volume in bytes used to store a queries result.
- `max_result_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum RAM volume used to store a queries result, in bytes.
- `read_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md)))) — The total number of source rows read from tables for running the query on all remote servers.
- `max_read_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum number of rows read from all tables and table functions participated in queries.
- `read_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The total number of bytes read from all tables and table functions participated in queries.
- `max_read_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum of bytes read from all tables and table functions.
- `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.

View File

@ -46,4 +46,8 @@ This table contains the following columns (the column type is shown in brackets)
- If the table stores data on disk, returns used space on disk (i.e. compressed).
- If the table stores data in memory, returns approximated number of used bytes in memory.
- `lifetime_rows` (Nullable(UInt64)) - Total number of rows INSERTed since server start (only for `Buffer` tables).
- `lifetime_bytes` (Nullable(UInt64)) - Total number of bytes INSERTed since server start (only for `Buffer` tables).
The `system.tables` table is used in `SHOW TABLES` query implementation.

View File

@ -12,7 +12,7 @@ ClickHouse supports the following types of queries:
- [SELECT](../sql-reference/statements/select/index.md)
- [INSERT INTO](../sql-reference/statements/insert-into.md)
- [CREATE](../sql-reference/statements/create/index.md)
- [ALTER](../sql-reference/statements/alter.md#query_language_queries_alter)
- [ALTER](../sql-reference/statements/alter/index.md)
- [Other types of queries](../sql-reference/statements/index.md)
[Original article](https://clickhouse.tech/docs/en/sql-reference/) <!--hide-->

View File

@ -1,600 +0,0 @@
---
toc_priority: 36
toc_title: ALTER
---
## ALTER Statement {#query_language_queries_alter}
The `ALTER` query is only supported for `*MergeTree` tables, as well as `Merge`and`Distributed`. The query has several variations.
### Column Manipulations {#column-manipulations}
Changing the table structure.
``` sql
ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN ...
```
In the query, specify a list of one or more comma-separated actions.
Each action is an operation on a column.
The following actions are supported:
- [ADD COLUMN](#alter_add-column) — Adds a new column to the table.
- [DROP COLUMN](#alter_drop-column) — Deletes the column.
- [CLEAR COLUMN](#alter_clear-column) — Resets column values.
- [COMMENT COLUMN](#alter_comment-column) — Adds a text comment to the column.
- [MODIFY COLUMN](#alter_modify-column) — Changes columns type, default expression and TTL.
These actions are described in detail below.
#### ADD COLUMN {#alter_add-column}
``` sql
ADD COLUMN [IF NOT EXISTS] name [type] [default_expr] [codec] [AFTER name_after]
```
Adds a new column to the table with the specified `name`, `type`, [`codec`](../../sql-reference/statements/create/table.md#codecs) and `default_expr` (see the section [Default expressions](../../sql-reference/statements/create/table.md#create-default-values)).
If the `IF NOT EXISTS` clause is included, the query wont return an error if the column already exists. If you specify `AFTER name_after` (the name of another column), the column is added after the specified one in the list of table columns. Otherwise, the column is added to the end of the table. Note that there is no way to add a column to the beginning of a table. For a chain of actions, `name_after` can be the name of a column that is added in one of the previous actions.
Adding a column just changes the table structure, without performing any actions with data. The data doesnt appear on the disk after `ALTER`. If the data is missing for a column when reading from the table, it is filled in with default values (by performing the default expression if there is one, or using zeros or empty strings). The column appears on the disk after merging data parts (see [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)).
This approach allows us to complete the `ALTER` query instantly, without increasing the volume of old data.
Example:
``` sql
ALTER TABLE visits ADD COLUMN browser String AFTER user_id
```
#### DROP COLUMN {#alter_drop-column}
``` sql
DROP COLUMN [IF EXISTS] name
```
Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, the query wont return an error if the column doesnt exist.
Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly.
Example:
``` sql
ALTER TABLE visits DROP COLUMN browser
```
#### CLEAR COLUMN {#alter_clear-column}
``` sql
CLEAR COLUMN [IF EXISTS] name IN PARTITION partition_name
```
Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to specify the partition expression](#alter-how-to-specify-part-expr).
If the `IF EXISTS` clause is specified, the query wont return an error if the column doesnt exist.
Example:
``` sql
ALTER TABLE visits CLEAR COLUMN browser IN PARTITION tuple()
```
#### COMMENT COLUMN {#alter_comment-column}
``` sql
COMMENT COLUMN [IF EXISTS] name 'comment'
```
Adds a comment to the column. If the `IF EXISTS` clause is specified, the query wont return an error if the column doesnt exist.
Each column can have one comment. If a comment already exists for the column, a new comment overwrites the previous comment.
Comments are stored in the `comment_expression` column returned by the [DESCRIBE TABLE](../../sql-reference/statements/misc.md#misc-describe-table) query.
Example:
``` sql
ALTER TABLE visits COMMENT COLUMN browser 'The table shows the browser used for accessing the site.'
```
#### MODIFY COLUMN {#alter_modify-column}
``` sql
MODIFY COLUMN [IF EXISTS] name [type] [default_expr] [TTL]
```
This query changes the `name` column properties:
- Type
- Default expression
- TTL
For examples of columns TTL modifying, see [Column TTL](../engines/table_engines/mergetree_family/mergetree.md#mergetree-column-ttl).
If the `IF EXISTS` clause is specified, the query wont return an error if the column doesnt exist.
When changing the type, values are converted as if the [toType](../../sql-reference/functions/type-conversion-functions.md) functions were applied to them. If only the default expression is changed, the query doesnt do anything complex, and is completed almost instantly.
Example:
``` sql
ALTER TABLE visits MODIFY COLUMN browser Array(String)
```
Changing the column type is the only complex action it changes the contents of files with data. For large tables, this may take a long time.
There are several processing stages:
- Preparing temporary (new) files with modified data.
- Renaming old files.
- Renaming the temporary (new) files to the old names.
- Deleting the old files.
Only the first stage takes time. If there is a failure at this stage, the data is not changed.
If there is a failure during one of the successive stages, data can be restored manually. The exception is if the old files were deleted from the file system but the data for the new files did not get written to the disk and was lost.
The `ALTER` query for changing columns is replicated. The instructions are saved in ZooKeeper, then each replica applies them. All `ALTER` queries are run in the same order. The query waits for the appropriate actions to be completed on the other replicas. However, a query to change columns in a replicated table can be interrupted, and all actions will be performed asynchronously.
#### ALTER Query Limitations {#alter-query-limitations}
The `ALTER` query lets you create and delete separate elements (columns) in nested data structures, but not whole nested data structures. To add a nested data structure, you can add columns with a name like `name.nested_name` and the type `Array(T)`. A nested data structure is equivalent to multiple array columns with a name that has the same prefix before the dot.
There is no support for deleting columns in the primary key or the sampling key (columns that are used in the `ENGINE` expression). Changing the type for columns that are included in the primary key is only possible if this change does not cause the data to be modified (for example, you are allowed to add values to an Enum or to change a type from `DateTime` to `UInt32`).
If the `ALTER` query is not sufficient to make the table changes you need, you can create a new table, copy the data to it using the [INSERT SELECT](../../sql-reference/statements/insert-into.md#insert_query_insert-select) query, then switch the tables using the [RENAME](../../sql-reference/statements/misc.md#misc_operations-rename) query and delete the old table. You can use the [clickhouse-copier](../../operations/utilities/clickhouse-copier.md) as an alternative to the `INSERT SELECT` query.
The `ALTER` query blocks all reads and writes for the table. In other words, if a long `SELECT` is running at the time of the `ALTER` query, the `ALTER` query will wait for it to complete. At the same time, all new queries to the same table will wait while this `ALTER` is running.
For tables that dont store data themselves (such as `Merge` and `Distributed`), `ALTER` just changes the table structure, and does not change the structure of subordinate tables. For example, when running ALTER for a `Distributed` table, you will also need to run `ALTER` for the tables on all remote servers.
### Manipulations with Key Expressions {#manipulations-with-key-expressions}
The following command is supported:
``` sql
MODIFY ORDER BY new_expression
```
It only works for tables in the [`MergeTree`](../../engines/table-engines/mergetree-family/mergetree.md) family (including
[replicated](../../engines/table-engines/mergetree-family/replication.md) tables). The command changes the
[sorting key](../../engines/table-engines/mergetree-family/mergetree.md) of the table
to `new_expression` (an expression or a tuple of expressions). Primary key remains the same.
The command is lightweight in a sense that it only changes metadata. To keep the property that data part
rows are ordered by the sorting key expression you cannot add expressions containing existing columns
to the sorting key (only columns added by the `ADD COLUMN` command in the same `ALTER` query).
### Manipulations with Data Skipping Indices {#manipulations-with-data-skipping-indices}
It only works for tables in the [`*MergeTree`](../../engines/table-engines/mergetree-family/mergetree.md) family (including
[replicated](../../engines/table-engines/mergetree-family/replication.md) tables). The following operations
are available:
- `ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value AFTER name [AFTER name2]` - Adds index description to tables metadata.
- `ALTER TABLE [db].name DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk.
These commands are lightweight in a sense that they only change metadata or remove files.
Also, they are replicated (syncing indices metadata through ZooKeeper).
### Manipulations with Constraints {#manipulations-with-constraints}
See more on [constraints](../../sql-reference/statements/create/table.md#constraints)
Constraints could be added or deleted using following syntax:
``` sql
ALTER TABLE [db].name ADD CONSTRAINT constraint_name CHECK expression;
ALTER TABLE [db].name DROP CONSTRAINT constraint_name;
```
Queries will add or remove metadata about constraints from table so they are processed immediately.
Constraint check *will not be executed* on existing data if it was added.
All changes on replicated tables are broadcasting to ZooKeeper so will be applied on other replicas.
### Manipulations with Partitions and Parts {#alter_manipulations-with-partitions}
The following operations with [partitions](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) are available:
- [DETACH PARTITION](#alter_detach-partition) — Moves a partition to the `detached` directory and forget it.
- [DROP PARTITION](#alter_drop-partition) — Deletes a partition.
- [ATTACH PART\|PARTITION](#alter_attach-partition) — Adds a part or partition from the `detached` directory to the table.
- [ATTACH PARTITION FROM](#alter_attach-partition-from) — Copies the data partition from one table to another and adds.
- [REPLACE PARTITION](#alter_replace-partition) — Copies the data partition from one table to another and replaces.
- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition) — Moves the data partition from one table to another.
- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — Resets the value of a specified column in a partition.
- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — Resets the specified secondary index in a partition.
- [FREEZE PARTITION](#alter_freeze-partition) — Creates a backup of a partition.
- [FETCH PARTITION](#alter_fetch-partition) — Downloads a partition from another server.
- [MOVE PARTITION\|PART](#alter_move-partition) — Move partition/data part to another disk or volume.
<!-- -->
#### DETACH PARTITION {#alter_detach-partition}
``` sql
ALTER TABLE table_name DETACH PARTITION partition_expr
```
Moves all data for the specified partition to the `detached` directory. The server forgets about the detached data partition as if it does not exist. The server will not know about this data until you make the [ATTACH](#alter_attach-partition) query.
Example:
``` sql
ALTER TABLE visits DETACH PARTITION 201901
```
Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
After the query is executed, you can do whatever you want with the data in the `detached` directory — delete it from the file system, or just leave it.
This query is replicated it moves the data to the `detached` directory on all replicas. Note that you can execute this query only on a leader replica. To find out if a replica is a leader, perform the `SELECT` query to the [system.replicas](../../operations/system-tables/replicas.md#system_tables-replicas) table. Alternatively, it is easier to make a `DETACH` query on all replicas - all the replicas throw an exception, except the leader replica.
#### DROP PARTITION {#alter_drop-partition}
``` sql
ALTER TABLE table_name DROP PARTITION partition_expr
```
Deletes the specified partition from the table. This query tags the partition as inactive and deletes data completely, approximately in 10 minutes.
Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
The query is replicated it deletes data on all replicas.
#### DROP DETACHED PARTITION\|PART {#alter_drop-detached}
``` sql
ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr
```
Removes the specified part or all parts of the specified partition from `detached`.
Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
#### ATTACH PARTITION\|PART {#alter_attach-partition}
``` sql
ALTER TABLE table_name ATTACH PARTITION|PART partition_expr
```
Adds data to the table from the `detached` directory. It is possible to add data for an entire partition or for a separate part. Examples:
``` sql
ALTER TABLE visits ATTACH PARTITION 201901;
ALTER TABLE visits ATTACH PART 201901_2_2_0;
```
Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
This query is replicated. The replica-initiator checks whether there is data in the `detached` directory. If data exists, the query checks its integrity. If everything is correct, the query adds the data to the table. All other replicas download the data from the replica-initiator.
So you can put data to the `detached` directory on one replica, and use the `ALTER ... ATTACH` query to add it to the table on all replicas.
#### ATTACH PARTITION FROM {#alter_attach-partition-from}
``` sql
ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1
```
This query copies the data partition from the `table1` to `table2` adds data to exsisting in the `table2`. Note that data wont be deleted from `table1`.
For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key.
#### REPLACE PARTITION {#alter_replace-partition}
``` sql
ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1
```
This query copies the data partition from the `table1` to `table2` and replaces existing partition in the `table2`. Note that data wont be deleted from `table1`.
For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key.
#### MOVE PARTITION TO TABLE {#alter_move_to_table-partition}
``` sql
ALTER TABLE table_source MOVE PARTITION partition_expr TO TABLE table_dest
```
This query moves the data partition from the `table_source` to `table_dest` with deleting the data from `table_source`.
For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key.
- Both tables must be the same engine family (replicated or non-replicated).
- Both tables must have the same storage policy.
#### CLEAR COLUMN IN PARTITION {#alter_clear-column-partition}
``` sql
ALTER TABLE table_name CLEAR COLUMN column_name IN PARTITION partition_expr
```
Resets all values in the specified column in a partition. If the `DEFAULT` clause was determined when creating a table, this query sets the column value to a specified default value.
Example:
``` sql
ALTER TABLE visits CLEAR COLUMN hour in PARTITION 201902
```
#### FREEZE PARTITION {#alter_freeze-partition}
``` sql
ALTER TABLE table_name FREEZE [PARTITION partition_expr]
```
This query creates a local backup of a specified partition. If the `PARTITION` clause is omitted, the query creates the backup of all partitions at once.
!!! note "Note"
The entire backup process is performed without stopping the server.
Note that for old-styled tables you can specify the prefix of the partition name (for example, 2019) - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
At the time of execution, for a data snapshot, the query creates hardlinks to a table data. Hardlinks are placed in the directory `/var/lib/clickhouse/shadow/N/...`, where:
- `/var/lib/clickhouse/` is the working ClickHouse directory specified in the config.
- `N` is the incremental number of the backup.
!!! note "Note"
If you use [a set of disks for data storage in a table](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes), the `shadow/N` directory appears on every disk, storing data parts that matched by the `PARTITION` expression.
The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs chmod for all files, forbidding writing into them.
After creating the backup, you can copy the data from `/var/lib/clickhouse/shadow/` to the remote server and then delete it from the local server. Note that the `ALTER t FREEZE PARTITION` query is not replicated. It creates a local backup only on the local server.
The query creates backup almost instantly (but first it waits for the current queries to the corresponding table to finish running).
`ALTER TABLE t FREEZE PARTITION` copies only the data, not table metadata. To make a backup of table metadata, copy the file `/var/lib/clickhouse/metadata/database/table.sql`
To restore data from a backup, do the following:
1. Create the table if it does not exist. To view the query, use the .sql file (replace `ATTACH` in it with `CREATE`).
2. Copy the data from the `data/database/table/` directory inside the backup to the `/var/lib/clickhouse/data/database/table/detached/` directory.
3. Run `ALTER TABLE t ATTACH PARTITION` queries to add the data to a table.
Restoring from a backup doesnt require stopping the server.
For more information about backups and restoring data, see the [Data Backup](../../operations/backup.md) section.
#### CLEAR INDEX IN PARTITION {#alter_clear-index-partition}
``` sql
ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr
```
The query works similar to `CLEAR COLUMN`, but it resets an index instead of a column data.
#### FETCH PARTITION {#alter_fetch-partition}
``` sql
ALTER TABLE table_name FETCH PARTITION partition_expr FROM 'path-in-zookeeper'
```
Downloads a partition from another server. This query only works for the replicated tables.
The query does the following:
1. Downloads the partition from the specified shard. In path-in-zookeeper you must specify a path to the shard in ZooKeeper.
2. Then the query puts the downloaded data to the `detached` directory of the `table_name` table. Use the [ATTACH PARTITION\|PART](#alter_attach-partition) query to add the data to the table.
For example:
``` sql
ALTER TABLE users FETCH PARTITION 201902 FROM '/clickhouse/tables/01-01/visits';
ALTER TABLE users ATTACH PARTITION 201902;
```
Note that:
- The `ALTER ... FETCH PARTITION` query isnt replicated. It places the partition to the `detached` directory only on the local server.
- The `ALTER TABLE ... ATTACH` query is replicated. It adds the data to all replicas. The data is added to one of the replicas from the `detached` directory, and to the others - from neighboring replicas.
Before downloading, the system checks if the partition exists and the table structure matches. The most appropriate replica is selected automatically from the healthy replicas.
Although the query is called `ALTER TABLE`, it does not change the table structure and does not immediately change the data available in the table.
#### MOVE PARTITION\|PART {#alter_move-partition}
Moves partitions or data parts to another volume or disk for `MergeTree`-engine tables. See [Using Multiple Block Devices for Data Storage](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes).
``` sql
ALTER TABLE table_name MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name'
```
The `ALTER TABLE t MOVE` query:
- Not replicated, because different replicas can have different storage policies.
- Returns an error if the specified disk or volume is not configured. Query also returns an error if conditions of data moving, that specified in the storage policy, cant be applied.
- Can return an error in the case, when data to be moved is already moved by a background process, concurrent `ALTER TABLE t MOVE` query or as a result of background data merging. A user shouldnt perform any additional actions in this case.
Example:
``` sql
ALTER TABLE hits MOVE PART '20190301_14343_16206_438' TO VOLUME 'slow'
ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd'
```
#### How to Set Partition Expression {#alter-how-to-specify-part-expr}
You can specify the partition expression in `ALTER ... PARTITION` queries in different ways:
- As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`.
- As the expression from the table column. Constants and constant expressions are supported. For example, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`.
- Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`.
- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached\_parts](../../operations/system-tables/detached_parts.md#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`.
Usage of quotes when specifying the partition depends on the type of partition expression. For example, for the `String` type, you have to specify its name in quotes (`'`). For the `Date` and `Int*` types no quotes are needed.
For old-style tables, you can specify the partition either as a number `201901` or a string `'201901'`. The syntax for the new-style tables is stricter with types (similar to the parser for the VALUES input format).
All the rules above are also true for the [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) query. If you need to specify the only partition when optimizing a non-partitioned table, set the expression `PARTITION tuple()`. For example:
``` sql
OPTIMIZE TABLE table_not_partitioned PARTITION tuple() FINAL;
```
The examples of `ALTER ... PARTITION` queries are demonstrated in the tests [`00502_custom_partitioning_local`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_local.sql) and [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql).
### Manipulations with Table TTL {#manipulations-with-table-ttl}
You can change [table TTL](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) with a request of the following form:
``` sql
ALTER TABLE table-name MODIFY TTL ttl-expression
```
### Synchronicity of ALTER Queries {#synchronicity-of-alter-queries}
For non-replicatable tables, all `ALTER` queries are performed synchronously. For replicatable tables, the query just adds instructions for the appropriate actions to `ZooKeeper`, and the actions themselves are performed as soon as possible. However, the query can wait for these actions to be completed on all the replicas.
For `ALTER ... ATTACH|DETACH|DROP` queries, you can use the `replication_alter_partitions_sync` setting to set up waiting.
Possible values: `0` do not wait; `1` only wait for own execution (default); `2` wait for all.
### Mutations {#alter-mutations}
Mutations are an ALTER query variant that allows changing or deleting rows in a table. In contrast to standard `UPDATE` and `DELETE` queries that are intended for point data changes, mutations are intended for heavy operations that change a lot of rows in a table. Supported for the `MergeTree` family of table engines including the engines with replication support.
Existing tables are ready for mutations as-is (no conversion necessary), but after the first mutation is applied to a table, its metadata format becomes incompatible with previous server versions and falling back to a previous version becomes impossible.
Currently available commands:
``` sql
ALTER TABLE [db.]table DELETE WHERE filter_expr
```
The `filter_expr` must be of type `UInt8`. The query deletes rows in the table for which this expression takes a non-zero value.
``` sql
ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr
```
The `filter_expr` must be of type `UInt8`. This query updates values of specified columns to the values of corresponding expressions in rows for which the `filter_expr` takes a non-zero value. Values are casted to the column type using the `CAST` operator. Updating columns that are used in the calculation of the primary or the partition key is not supported.
``` sql
ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name
```
The query rebuilds the secondary index `name` in the partition `partition_name`.
One query can contain several commands separated by commas.
For \*MergeTree tables mutations execute by rewriting whole data parts. There is no atomicity - parts are substituted for mutated parts as soon as they are ready and a `SELECT` query that started executing during a mutation will see data from parts that have already been mutated along with data from parts that have not been mutated yet.
Mutations are totally ordered by their creation order and are applied to each part in that order. Mutations are also partially ordered with INSERTs - data that was inserted into the table before the mutation was submitted will be mutated and data that was inserted after that will not be mutated. Note that mutations do not block INSERTs in any way.
A mutation query returns immediately after the mutation entry is added (in case of replicated tables to ZooKeeper, for nonreplicated tables - to the filesystem). The mutation itself executes asynchronously using the system profile settings. To track the progress of mutations you can use the [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations) table. A mutation that was successfully submitted will continue to execute even if ClickHouse servers are restarted. There is no way to roll back the mutation once it is submitted, but if the mutation is stuck for some reason it can be cancelled with the [`KILL MUTATION`](../../sql-reference/statements/misc.md#kill-mutation) query.
Entries for finished mutations are not deleted right away (the number of preserved entries is determined by the `finished_mutations_to_keep` storage engine parameter). Older mutation entries are deleted.
## ALTER USER {#alter-user-statement}
Changes ClickHouse user accounts.
### Syntax {#alter-user-syntax}
``` sql
ALTER USER [IF EXISTS] name [ON CLUSTER cluster_name]
[RENAME TO new_name]
[IDENTIFIED [WITH {PLAINTEXT_PASSWORD|SHA256_PASSWORD|DOUBLE_SHA1_PASSWORD}] BY {'password'|'hash'}]
[[ADD|DROP] HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
[DEFAULT ROLE role [,...] | ALL | ALL EXCEPT role [,...] ]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]
```
### Description {#alter-user-dscr}
To use `ALTER USER` you must have the [ALTER USER](../../sql-reference/statements/grant.md#grant-access-management) privilege.
### Examples {#alter-user-examples}
Set assigned roles as default:
``` sql
ALTER USER user DEFAULT ROLE role1, role2
```
If roles arent previously assigned to a user, ClickHouse throws an exception.
Set all the assigned roles to default:
``` sql
ALTER USER user DEFAULT ROLE ALL
```
If a role is assigned to a user in the future, it will become default automatically.
Set all the assigned roles to default, excepting `role1` and `role2`:
``` sql
ALTER USER user DEFAULT ROLE ALL EXCEPT role1, role2
```
## ALTER ROLE {#alter-role-statement}
Changes roles.
### Syntax {#alter-role-syntax}
``` sql
ALTER ROLE [IF EXISTS] name [ON CLUSTER cluster_name]
[RENAME TO new_name]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]
```
## ALTER ROW POLICY {#alter-row-policy-statement}
Changes row policy.
### Syntax {#alter-row-policy-syntax}
``` sql
ALTER [ROW] POLICY [IF EXISTS] name [ON CLUSTER cluster_name] ON [database.]table
[RENAME TO new_name]
[AS {PERMISSIVE | RESTRICTIVE}]
[FOR SELECT]
[USING {condition | NONE}][,...]
[TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
```
## ALTER QUOTA {#alter-quota-statement}
Changes quotas.
### Syntax {#alter-quota-syntax}
``` sql
ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name]
[RENAME TO new_name]
[KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
[FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}
{MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |
NO LIMITS | TRACKING ONLY} [,...]]
[TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
```
## ALTER SETTINGS PROFILE {#alter-settings-profile-statement}
Changes settings profiles.
### Syntax {#alter-settings-profile-syntax}
``` sql
ALTER SETTINGS PROFILE [IF EXISTS] name [ON CLUSTER cluster_name]
[RENAME TO new_name]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | INHERIT 'profile_name'] [,...]
```
[Original article](https://clickhouse.tech/docs/en/query_language/alter/) <!--hide-->

View File

@ -0,0 +1,149 @@
---
toc_priority: 37
toc_title: COLUMN
---
# Column Manipulations {#column-manipulations}
A set of queries that allow changing the table structure.
Syntax:
``` sql
ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN ...
```
In the query, specify a list of one or more comma-separated actions.
Each action is an operation on a column.
The following actions are supported:
- [ADD COLUMN](#alter_add-column) — Adds a new column to the table.
- [DROP COLUMN](#alter_drop-column) — Deletes the column.
- [CLEAR COLUMN](#alter_clear-column) — Resets column values.
- [COMMENT COLUMN](#alter_comment-column) — Adds a text comment to the column.
- [MODIFY COLUMN](#alter_modify-column) — Changes columns type, default expression and TTL.
These actions are described in detail below.
## ADD COLUMN {#alter_add-column}
``` sql
ADD COLUMN [IF NOT EXISTS] name [type] [default_expr] [codec] [AFTER name_after]
```
Adds a new column to the table with the specified `name`, `type`, [`codec`](../../../sql-reference/statements/create/table.md#codecs) and `default_expr` (see the section [Default expressions](../../../sql-reference/statements/create/table.md#create-default-values)).
If the `IF NOT EXISTS` clause is included, the query wont return an error if the column already exists. If you specify `AFTER name_after` (the name of another column), the column is added after the specified one in the list of table columns. Otherwise, the column is added to the end of the table. Note that there is no way to add a column to the beginning of a table. For a chain of actions, `name_after` can be the name of a column that is added in one of the previous actions.
Adding a column just changes the table structure, without performing any actions with data. The data doesnt appear on the disk after `ALTER`. If the data is missing for a column when reading from the table, it is filled in with default values (by performing the default expression if there is one, or using zeros or empty strings). The column appears on the disk after merging data parts (see [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)).
This approach allows us to complete the `ALTER` query instantly, without increasing the volume of old data.
Example:
``` sql
ALTER TABLE visits ADD COLUMN browser String AFTER user_id
```
## DROP COLUMN {#alter_drop-column}
``` sql
DROP COLUMN [IF EXISTS] name
```
Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, the query wont return an error if the column doesnt exist.
Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly.
Example:
``` sql
ALTER TABLE visits DROP COLUMN browser
```
## CLEAR COLUMN {#alter_clear-column}
``` sql
CLEAR COLUMN [IF EXISTS] name IN PARTITION partition_name
```
Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to specify the partition expression](#alter-how-to-specify-part-expr).
If the `IF EXISTS` clause is specified, the query wont return an error if the column doesnt exist.
Example:
``` sql
ALTER TABLE visits CLEAR COLUMN browser IN PARTITION tuple()
```
## COMMENT COLUMN {#alter_comment-column}
``` sql
COMMENT COLUMN [IF EXISTS] name 'comment'
```
Adds a comment to the column. If the `IF EXISTS` clause is specified, the query wont return an error if the column doesnt exist.
Each column can have one comment. If a comment already exists for the column, a new comment overwrites the previous comment.
Comments are stored in the `comment_expression` column returned by the [DESCRIBE TABLE](../../../sql-reference/statements/misc.md#misc-describe-table) query.
Example:
``` sql
ALTER TABLE visits COMMENT COLUMN browser 'The table shows the browser used for accessing the site.'
```
## MODIFY COLUMN {#alter_modify-column}
``` sql
MODIFY COLUMN [IF EXISTS] name [type] [default_expr] [TTL]
```
This query changes the `name` column properties:
- Type
- Default expression
- TTL
For examples of columns TTL modifying, see [Column TTL](../../engines/table_engines/mergetree_family/mergetree.md#mergetree-column-ttl).
If the `IF EXISTS` clause is specified, the query wont return an error if the column doesnt exist.
When changing the type, values are converted as if the [toType](../../../sql-reference/functions/type-conversion-functions.md) functions were applied to them. If only the default expression is changed, the query doesnt do anything complex, and is completed almost instantly.
Example:
``` sql
ALTER TABLE visits MODIFY COLUMN browser Array(String)
```
Changing the column type is the only complex action it changes the contents of files with data. For large tables, this may take a long time.
There are several processing stages:
- Preparing temporary (new) files with modified data.
- Renaming old files.
- Renaming the temporary (new) files to the old names.
- Deleting the old files.
Only the first stage takes time. If there is a failure at this stage, the data is not changed.
If there is a failure during one of the successive stages, data can be restored manually. The exception is if the old files were deleted from the file system but the data for the new files did not get written to the disk and was lost.
The `ALTER` query for changing columns is replicated. The instructions are saved in ZooKeeper, then each replica applies them. All `ALTER` queries are run in the same order. The query waits for the appropriate actions to be completed on the other replicas. However, a query to change columns in a replicated table can be interrupted, and all actions will be performed asynchronously.
## Limitations {#alter-query-limitations}
The `ALTER` query lets you create and delete separate elements (columns) in nested data structures, but not whole nested data structures. To add a nested data structure, you can add columns with a name like `name.nested_name` and the type `Array(T)`. A nested data structure is equivalent to multiple array columns with a name that has the same prefix before the dot.
There is no support for deleting columns in the primary key or the sampling key (columns that are used in the `ENGINE` expression). Changing the type for columns that are included in the primary key is only possible if this change does not cause the data to be modified (for example, you are allowed to add values to an Enum or to change a type from `DateTime` to `UInt32`).
If the `ALTER` query is not sufficient to make the table changes you need, you can create a new table, copy the data to it using the [INSERT SELECT](../../../sql-reference/statements/insert-into.md#insert_query_insert-select) query, then switch the tables using the [RENAME](../../../sql-reference/statements/misc.md#misc_operations-rename) query and delete the old table. You can use the [clickhouse-copier](../../../operations/utilities/clickhouse-copier.md) as an alternative to the `INSERT SELECT` query.
The `ALTER` query blocks all reads and writes for the table. In other words, if a long `SELECT` is running at the time of the `ALTER` query, the `ALTER` query will wait for it to complete. At the same time, all new queries to the same table will wait while this `ALTER` is running.
For tables that dont store data themselves (such as `Merge` and `Distributed`), `ALTER` just changes the table structure, and does not change the structure of subordinate tables. For example, when running ALTER for a `Distributed` table, you will also need to run `ALTER` for the tables on all remote servers.

View File

@ -0,0 +1,22 @@
---
toc_priority: 43
toc_title: CONSTRAINT
---
# Manipulating Constraints {#manipulations-with-constraints}
Constraints could be added or deleted using following syntax:
``` sql
ALTER TABLE [db].name ADD CONSTRAINT constraint_name CHECK expression;
ALTER TABLE [db].name DROP CONSTRAINT constraint_name;
```
See more on [constraints](../../../sql-reference/statements/create/table.md#constraints).
Queries will add or remove metadata about constraints from table so they are processed immediately.
!!! warning "Warning"
Constraint check **will not be executed** on existing data if it was added.
All changes on replicated tables are broadcasted to ZooKeeper and will be applied on other replicas as well.

View File

@ -0,0 +1,19 @@
---
toc_priority: 39
toc_title: DELETE
---
# ALTER TABLE … DELETE Statement {#alter-mutations}
``` sql
ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr
```
Allows to asynchronously delete data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/index.md#mutations).
!!! note "Note"
The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use.
The `filter_expr` must be of type `UInt8`. The query deletes rows in the table for which this expression takes a non-zero value.
One query can contain several commands separated by commas.

View File

@ -0,0 +1,48 @@
---
toc_priority: 36
toc_title: ALTER
---
## ALTER {#query_language_queries_alter}
Most `ALTER` queries modify table settings or data:
- [COLUMN](../../../sql-reference/statements/alter/column.md)
- [PARTITION](../../../sql-reference/statements/alter/partition.md)
- [DELETE](../../../sql-reference/statements/alter/delete.md)
- [UPDATE](../../../sql-reference/statements/alter/update.md)
- [ORDER BY](../../../sql-reference/statements/alter/order-by.md)
- [INDEX](../../../sql-reference/statements/alter/index/index.md)
- [CONSTRAINT](../../../sql-reference/statements/alter/constraint.md)
- [TTL](../../../sql-reference/statements/alter/ttl.md)
!!! note "Note"
Most `ALTER` queries are supported only for [\*MergeTree](../../../engines/table-engines/mergetree-family/index.md) tables, as well as [Merge](../../../engines/table-engines/special/merge.md) and [Distributed](../../../engines/table-engines/special/distributed.md).
While these `ALTER` settings modify entities related to role-based access control:
- [USER](../../../sql-reference/statements/alter/user.md)
- [ROLE](../../../sql-reference/statements/alter/role.md)
- [QUOTA](../../../sql-reference/statements/alter/quota.md)
- [ROW POLICY](../../../sql-reference/statements/alter/row-policy.md)
- [SETTINGS PROFILE](../../../sql-reference/statements/alter/settings-profile.md)
## Synchronicity of ALTER Queries {#synchronicity-of-alter-queries}
For non-replicated tables, all `ALTER` queries are performed synchronously. For replicated tables, the query just adds instructions for the appropriate actions to `ZooKeeper`, and the actions themselves are performed as soon as possible. However, the query can wait for these actions to be completed on all the replicas.
For `ALTER ... ATTACH|DETACH|DROP` queries, you can use the `replication_alter_partitions_sync` setting to set up waiting. Possible values: `0` do not wait; `1` only wait for own execution (default); `2` wait for all.
## Mutations {#mutations}
`ALTER` queries that are intended to manipulate table data are implemented with a mechanism called “mutations”, most notably [ALTER TABLE … DELETE](../../../sql-reference/statements/alter/delete.md) and [ALTER TABLE … UPDATE](../../../sql-reference/statements/alter/update.md). They are asynchronous background processes similar to merges in [MergeTree](../../../engines/table-engines/mergetree-family/index.md) tables that to produce new “mutated” versions of parts.
For `*MergeTree` tables mutations execute by **rewriting whole data parts**. There is no atomicity - parts are substituted for mutated parts as soon as they are ready and a `SELECT` query that started executing during a mutation will see data from parts that have already been mutated along with data from parts that have not been mutated yet.
Mutations are totally ordered by their creation order and are applied to each part in that order. Mutations are also partially ordered with `INSERT INTO` queries: data that was inserted into the table before the mutation was submitted will be mutated and data that was inserted after that will not be mutated. Note that mutations do not block inserts in any way.
A mutation query returns immediately after the mutation entry is added (in case of replicated tables to ZooKeeper, for non-replicated tables - to the filesystem). The mutation itself executes asynchronously using the system profile settings. To track the progress of mutations you can use the [`system.mutations`](../../../operations/system-tables/mutations.md#system_tables-mutations) table. A mutation that was successfully submitted will continue to execute even if ClickHouse servers are restarted. There is no way to roll back the mutation once it is submitted, but if the mutation is stuck for some reason it can be cancelled with the [`KILL MUTATION`](../../../sql-reference/statements/misc.md#kill-mutation) query.
Entries for finished mutations are not deleted right away (the number of preserved entries is determined by the `finished_mutations_to_keep` storage engine parameter). Older mutation entries are deleted.
[Original article](https://clickhouse.tech/docs/en/query_language/alter/) <!--hide-->

View File

@ -0,0 +1,23 @@
---
toc_hidden_folder: true
toc_priority: 42
toc_title: INDEX
---
# Manipulating Data Skipping Indices {#manipulations-with-data-skipping-indices}
The following operations are available:
- `ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value AFTER name [AFTER name2]` - Adds index description to tables metadata.
- `ALTER TABLE [db].name DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk.
- `ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name` - The query rebuilds the secondary index `name` in the partition `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations).
The first two commands areare lightweight in a sense that they only change metadata or remove files.
Also, they are replicated, syncing indices metadata via ZooKeeper.
!!! note "Note"
Index manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including
[replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants).

View File

@ -0,0 +1,18 @@
---
toc_priority: 41
toc_title: ORDER BY
---
# Manipulating Key Expressions {#manipulations-with-key-expressions}
``` sql
ALTER TABLE [db].name [ON CLUSTER cluster] MODIFY ORDER BY new_expression
```
The command changes the [sorting key](../../../engines/table-engines/mergetree-family/mergetree.md) of the table to `new_expression` (an expression or a tuple of expressions). Primary key remains the same.
The command is lightweight in a sense that it only changes metadata. To keep the property that data part rows are ordered by the sorting key expression you cannot add expressions containing existing columns to the sorting key (only columns added by the `ADD COLUMN` command in the same `ALTER` query).
!!! note "Note"
It only works for tables in the [`MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) family (including
[replicated](../../../engines/table-engines/mergetree-family/replication.md) tables).

View File

@ -0,0 +1,253 @@
---
toc_priority: 38
toc_title: PARTITION
---
# Manipulating Partitions and Parts {#alter_manipulations-with-partitions}
The following operations with [partitions](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md) are available:
- [DETACH PARTITION](#alter_detach-partition) — Moves a partition to the `detached` directory and forget it.
- [DROP PARTITION](#alter_drop-partition) — Deletes a partition.
- [ATTACH PART\|PARTITION](#alter_attach-partition) — Adds a part or partition from the `detached` directory to the table.
- [ATTACH PARTITION FROM](#alter_attach-partition-from) — Copies the data partition from one table to another and adds.
- [REPLACE PARTITION](#alter_replace-partition) — Copies the data partition from one table to another and replaces.
- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition) — Moves the data partition from one table to another.
- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — Resets the value of a specified column in a partition.
- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — Resets the specified secondary index in a partition.
- [FREEZE PARTITION](#alter_freeze-partition) — Creates a backup of a partition.
- [FETCH PARTITION](#alter_fetch-partition) — Downloads a partition from another server.
- [MOVE PARTITION\|PART](#alter_move-partition) — Move partition/data part to another disk or volume.
<!-- -->
## DETACH PARTITION {#alter_detach-partition}
``` sql
ALTER TABLE table_name DETACH PARTITION partition_expr
```
Moves all data for the specified partition to the `detached` directory. The server forgets about the detached data partition as if it does not exist. The server will not know about this data until you make the [ATTACH](#alter_attach-partition) query.
Example:
``` sql
ALTER TABLE visits DETACH PARTITION 201901
```
Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
After the query is executed, you can do whatever you want with the data in the `detached` directory — delete it from the file system, or just leave it.
This query is replicated it moves the data to the `detached` directory on all replicas. Note that you can execute this query only on a leader replica. To find out if a replica is a leader, perform the `SELECT` query to the [system.replicas](../../../operations/system-tables/replicas.md#system_tables-replicas) table. Alternatively, it is easier to make a `DETACH` query on all replicas - all the replicas throw an exception, except the leader replica.
## DROP PARTITION {#alter_drop-partition}
``` sql
ALTER TABLE table_name DROP PARTITION partition_expr
```
Deletes the specified partition from the table. This query tags the partition as inactive and deletes data completely, approximately in 10 minutes.
Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
The query is replicated it deletes data on all replicas.
## DROP DETACHED PARTITION\|PART {#alter_drop-detached}
``` sql
ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr
```
Removes the specified part or all parts of the specified partition from `detached`.
Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
## ATTACH PARTITION\|PART {#alter_attach-partition}
``` sql
ALTER TABLE table_name ATTACH PARTITION|PART partition_expr
```
Adds data to the table from the `detached` directory. It is possible to add data for an entire partition or for a separate part. Examples:
``` sql
ALTER TABLE visits ATTACH PARTITION 201901;
ALTER TABLE visits ATTACH PART 201901_2_2_0;
```
Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
This query is replicated. The replica-initiator checks whether there is data in the `detached` directory. If data exists, the query checks its integrity. If everything is correct, the query adds the data to the table. All other replicas download the data from the replica-initiator.
So you can put data to the `detached` directory on one replica, and use the `ALTER ... ATTACH` query to add it to the table on all replicas.
## ATTACH PARTITION FROM {#alter_attach-partition-from}
``` sql
ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1
```
This query copies the data partition from the `table1` to `table2` adds data to exsisting in the `table2`. Note that data wont be deleted from `table1`.
For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key.
## REPLACE PARTITION {#alter_replace-partition}
``` sql
ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1
```
This query copies the data partition from the `table1` to `table2` and replaces existing partition in the `table2`. Note that data wont be deleted from `table1`.
For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key.
## MOVE PARTITION TO TABLE {#alter_move_to_table-partition}
``` sql
ALTER TABLE table_source MOVE PARTITION partition_expr TO TABLE table_dest
```
This query moves the data partition from the `table_source` to `table_dest` with deleting the data from `table_source`.
For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key.
- Both tables must be the same engine family (replicated or non-replicated).
- Both tables must have the same storage policy.
## CLEAR COLUMN IN PARTITION {#alter_clear-column-partition}
``` sql
ALTER TABLE table_name CLEAR COLUMN column_name IN PARTITION partition_expr
```
Resets all values in the specified column in a partition. If the `DEFAULT` clause was determined when creating a table, this query sets the column value to a specified default value.
Example:
``` sql
ALTER TABLE visits CLEAR COLUMN hour in PARTITION 201902
```
## FREEZE PARTITION {#alter_freeze-partition}
``` sql
ALTER TABLE table_name FREEZE [PARTITION partition_expr]
```
This query creates a local backup of a specified partition. If the `PARTITION` clause is omitted, the query creates the backup of all partitions at once.
!!! note "Note"
The entire backup process is performed without stopping the server.
Note that for old-styled tables you can specify the prefix of the partition name (for example, 2019) - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
At the time of execution, for a data snapshot, the query creates hardlinks to a table data. Hardlinks are placed in the directory `/var/lib/clickhouse/shadow/N/...`, where:
- `/var/lib/clickhouse/` is the working ClickHouse directory specified in the config.
- `N` is the incremental number of the backup.
!!! note "Note"
If you use [a set of disks for data storage in a table](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes), the `shadow/N` directory appears on every disk, storing data parts that matched by the `PARTITION` expression.
The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs chmod for all files, forbidding writing into them.
After creating the backup, you can copy the data from `/var/lib/clickhouse/shadow/` to the remote server and then delete it from the local server. Note that the `ALTER t FREEZE PARTITION` query is not replicated. It creates a local backup only on the local server.
The query creates backup almost instantly (but first it waits for the current queries to the corresponding table to finish running).
`ALTER TABLE t FREEZE PARTITION` copies only the data, not table metadata. To make a backup of table metadata, copy the file `/var/lib/clickhouse/metadata/database/table.sql`
To restore data from a backup, do the following:
1. Create the table if it does not exist. To view the query, use the .sql file (replace `ATTACH` in it with `CREATE`).
2. Copy the data from the `data/database/table/` directory inside the backup to the `/var/lib/clickhouse/data/database/table/detached/` directory.
3. Run `ALTER TABLE t ATTACH PARTITION` queries to add the data to a table.
Restoring from a backup doesnt require stopping the server.
For more information about backups and restoring data, see the [Data Backup](../../../operations/backup.md) section.
## CLEAR INDEX IN PARTITION {#alter_clear-index-partition}
``` sql
ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr
```
The query works similar to `CLEAR COLUMN`, but it resets an index instead of a column data.
## FETCH PARTITION {#alter_fetch-partition}
``` sql
ALTER TABLE table_name FETCH PARTITION partition_expr FROM 'path-in-zookeeper'
```
Downloads a partition from another server. This query only works for the replicated tables.
The query does the following:
1. Downloads the partition from the specified shard. In path-in-zookeeper you must specify a path to the shard in ZooKeeper.
2. Then the query puts the downloaded data to the `detached` directory of the `table_name` table. Use the [ATTACH PARTITION\|PART](#alter_attach-partition) query to add the data to the table.
For example:
``` sql
ALTER TABLE users FETCH PARTITION 201902 FROM '/clickhouse/tables/01-01/visits';
ALTER TABLE users ATTACH PARTITION 201902;
```
Note that:
- The `ALTER ... FETCH PARTITION` query isnt replicated. It places the partition to the `detached` directory only on the local server.
- The `ALTER TABLE ... ATTACH` query is replicated. It adds the data to all replicas. The data is added to one of the replicas from the `detached` directory, and to the others - from neighboring replicas.
Before downloading, the system checks if the partition exists and the table structure matches. The most appropriate replica is selected automatically from the healthy replicas.
Although the query is called `ALTER TABLE`, it does not change the table structure and does not immediately change the data available in the table.
## MOVE PARTITION\|PART {#alter_move-partition}
Moves partitions or data parts to another volume or disk for `MergeTree`-engine tables. See [Using Multiple Block Devices for Data Storage](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes).
``` sql
ALTER TABLE table_name MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name'
```
The `ALTER TABLE t MOVE` query:
- Not replicated, because different replicas can have different storage policies.
- Returns an error if the specified disk or volume is not configured. Query also returns an error if conditions of data moving, that specified in the storage policy, cant be applied.
- Can return an error in the case, when data to be moved is already moved by a background process, concurrent `ALTER TABLE t MOVE` query or as a result of background data merging. A user shouldnt perform any additional actions in this case.
Example:
``` sql
ALTER TABLE hits MOVE PART '20190301_14343_16206_438' TO VOLUME 'slow'
ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd'
```
## How to Set Partition Expression {#alter-how-to-specify-part-expr}
You can specify the partition expression in `ALTER ... PARTITION` queries in different ways:
- As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`.
- As the expression from the table column. Constants and constant expressions are supported. For example, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`.
- Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`.
- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached\_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`.
Usage of quotes when specifying the partition depends on the type of partition expression. For example, for the `String` type, you have to specify its name in quotes (`'`). For the `Date` and `Int*` types no quotes are needed.
All the rules above are also true for the [OPTIMIZE](../../../sql-reference/statements/misc.md#misc_operations-optimize) query. If you need to specify the only partition when optimizing a non-partitioned table, set the expression `PARTITION tuple()`. For example:
``` sql
OPTIMIZE TABLE table_not_partitioned PARTITION tuple() FINAL;
```
The examples of `ALTER ... PARTITION` queries are demonstrated in the tests [`00502_custom_partitioning_local`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_local.sql) and [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql).

View File

@ -0,0 +1,20 @@
---
toc_priority: 46
toc_title: QUOTA
---
# ALTER QUOTA {#alter-quota-statement}
Changes quotas.
Syntax:
``` sql
ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name]
[RENAME TO new_name]
[KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
[FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}
{MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |
NO LIMITS | TRACKING ONLY} [,...]]
[TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
```

View File

@ -0,0 +1,16 @@
---
toc_priority: 46
toc_title: ROLE
---
## ALTER ROLE {#alter-role-statement}
Changes roles.
Syntax:
``` sql
ALTER ROLE [IF EXISTS] name [ON CLUSTER cluster_name]
[RENAME TO new_name]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]
```

View File

@ -0,0 +1,19 @@
---
toc_priority: 47
toc_title: ROW POLICY
---
# ALTER ROW POLICY {#alter-row-policy-statement}
Changes row policy.
Syntax:
``` sql
ALTER [ROW] POLICY [IF EXISTS] name [ON CLUSTER cluster_name] ON [database.]table
[RENAME TO new_name]
[AS {PERMISSIVE | RESTRICTIVE}]
[FOR SELECT]
[USING {condition | NONE}][,...]
[TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
```

View File

@ -0,0 +1,16 @@
---
toc_priority: 48
toc_title: SETTINGS PROFILE
---
## ALTER SETTINGS PROFILE {#alter-settings-profile-statement}
Changes settings profiles.
Syntax:
``` sql
ALTER SETTINGS PROFILE [IF EXISTS] 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

@ -0,0 +1,12 @@
---
toc_priority: 44
toc_title: TTL
---
### Manipulations with Table TTL {#manipulations-with-table-ttl}
You can change [table TTL](../../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) with a request of the following form:
``` sql
ALTER TABLE table-name MODIFY TTL ttl-expression
```

View File

@ -0,0 +1,19 @@
---
toc_priority: 40
toc_title: UPDATE
---
# ALTER TABLE … UPDATE Statements {#alter-table-update-statements}
``` sql
ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr
```
Allows to asynchronously manipulate data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/index.md#mutations).
!!! note "Note"
The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use.
The `filter_expr` must be of type `UInt8`. This query updates values of specified columns to the values of corresponding expressions in rows for which the `filter_expr` takes a non-zero value. Values are casted to the column type using the `CAST` operator. Updating columns that are used in the calculation of the primary or the partition key is not supported.
One query can contain several commands separated by commas.

View File

@ -0,0 +1,45 @@
---
toc_priority: 45
toc_title: USER
---
# ALTER USER {#alter-user-statement}
Changes ClickHouse user accounts.
Syntax:
``` sql
ALTER USER [IF EXISTS] name [ON CLUSTER cluster_name]
[RENAME TO new_name]
[IDENTIFIED [WITH {PLAINTEXT_PASSWORD|SHA256_PASSWORD|DOUBLE_SHA1_PASSWORD}] BY {'password'|'hash'}]
[[ADD|DROP] HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
[DEFAULT ROLE role [,...] | ALL | ALL EXCEPT role [,...] ]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]
```
To use `ALTER USER` you must have the [ALTER USER](../../../sql-reference/statements/grant.md#grant-access-management) privilege.
## Examples {#alter-user-examples}
Set assigned roles as default:
``` sql
ALTER USER user DEFAULT ROLE role1, role2
```
If roles arent previously assigned to a user, ClickHouse throws an exception.
Set all the assigned roles to default:
``` sql
ALTER USER user DEFAULT ROLE ALL
```
If a role is assigned to a user in the future, it will become default automatically.
Set all the assigned roles to default, excepting `role1` and `role2`:
``` sql
ALTER USER user DEFAULT ROLE ALL EXCEPT role1, role2
```

View File

@ -18,7 +18,7 @@ CREATE ROLE [IF NOT EXISTS | OR REPLACE] name
A user can be assigned multiple roles. Users can apply their assigned roles in arbitrary combinations by the [SET ROLE](../../../sql-reference/statements/set-role.md) statement. The final scope of privileges is a combined set of all the privileges of all the applied roles. If a user has privileges granted directly to its user account, they are also combined with the privileges granted by roles.
User can have default roles which apply at user login. To set default roles, use the [SET DEFAULT ROLE](../../../sql-reference/statements/set-role.md#set-default-role-statement) statement or the [ALTER USER](../../../sql-reference/statements/alter.md#alter-user-statement) statement.
User can have default roles which apply at user login. To set default roles, use the [SET DEFAULT ROLE](../../../sql-reference/statements/set-role.md#set-default-role-statement) statement or the [ALTER USER](../../../sql-reference/statements/alter/user.md#alter-user-statement) statement.
To revoke a role, use the [REVOKE](../../../sql-reference/statements/revoke.md) statement.

View File

@ -52,7 +52,7 @@ If you specify `POPULATE`, the existing table data is inserted in the view when
A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`… Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data wont be further aggregated. The exception is when using an `ENGINE` that independently performs data aggregation, such as `SummingMergeTree`.
The execution of [ALTER](../../../sql-reference/statements/alter.md) queries on materialized views has limitations, so they might be inconvenient. If the materialized view uses the construction `TO [db.]name`, you can `DETACH` the view, run `ALTER` for the target table, and then `ATTACH` the previously detached (`DETACH`) view.
The execution of [ALTER](../../../sql-reference/statements/alter/index.md) queries on materialized views has limitations, so they might be inconvenient. If the materialized view uses the construction `TO [db.]name`, you can `DETACH` the view, run `ALTER` for the target table, and then `ATTACH` the previously detached (`DETACH`) view.
Views look the same as normal tables. For example, they are listed in the result of the `SHOW TABLES` query.

View File

@ -251,7 +251,7 @@ The granted privilege allows `john` to insert data to the `x` and/or `y` columns
### ALTER {#grant-alter}
Allows executing [ALTER](../../sql-reference/statements/alter.md) queries according to the following hierarchy of privileges:
Allows executing [ALTER](../../sql-reference/statements/alter/index.md) queries according to the following hierarchy of privileges:
- `ALTER`. Level: `COLUMN`.
- `ALTER TABLE`. Level: `GROUP`

View File

@ -11,7 +11,7 @@ Statements represent various kinds of action you can perform using SQL queries.
- [SELECT](../../sql-reference/statements/select/index.md)
- [INSERT INTO](../../sql-reference/statements/insert-into.md)
- [CREATE](../../sql-reference/statements/create/index.md)
- [ALTER](../../sql-reference/statements/alter.md)
- [ALTER](../../sql-reference/statements/alter/index.md)
- [SYSTEM](../../sql-reference/statements/system.md)
- [SHOW](../../sql-reference/statements/show.md)
- [GRANT](../../sql-reference/statements/grant.md)

View File

@ -50,7 +50,7 @@ KILL MUTATION [ON CLUSTER cluster]
[FORMAT format]
```
Tries to cancel and remove [mutations](../../sql-reference/statements/alter.md#alter-mutations) that are currently executing. Mutations to cancel are selected from the [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations) table using the filter specified by the `WHERE` clause of the `KILL` query.
Tries to cancel and remove [mutations](../../sql-reference/statements/alter/index.md#alter-mutations) that are currently executing. Mutations to cancel are selected from the [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations) table using the filter specified by the `WHERE` clause of the `KILL` query.
A test query (`TEST`) only checks the users rights and displays a list of queries to stop.

View File

@ -15,7 +15,7 @@ The `OPTMIZE` query is also supported for the [MaterializedView](../../engines/t
When `OPTIMIZE` is used with the [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md) family of table engines, ClickHouse creates a task for merging and waits for execution on all nodes (if the `replication_alter_partitions_sync` setting is enabled).
- If `OPTIMIZE` doesnt perform a merge for any reason, it doesnt notify the client. To enable notifications, use the [optimize\_throw\_if\_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) setting.
- If you specify a `PARTITION`, only the specified partition is optimized. [How to set partition expression](../../sql-reference/statements/alter.md#alter-how-to-specify-part-expr).
- If you specify a `PARTITION`, only the specified partition is optimized. [How to set partition expression](../../sql-reference/statements/alter/index.md#alter-how-to-specify-part-expr).
- If you specify `FINAL`, optimization is performed even when all the data is already in one part.
- If you specify `DEDUPLICATE`, then completely identical rows will be deduplicated (all columns are compared), it makes sense only for the MergeTree engine.

View File

@ -22,9 +22,9 @@ SELECT [DISTINCT] expr_list
[WHERE expr]
[GROUP BY expr_list] [WITH TOTALS]
[HAVING expr]
[ORDER BY expr_list]
[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr]
[LIMIT [offset_value, ]n BY columns]
[LIMIT [n, ]m]
[LIMIT [n, ]m] [WITH TIES]
[UNION ALL ...]
[INTO OUTFILE filename]
[FORMAT format]

View File

@ -11,3 +11,52 @@ toc_title: LIMIT
`n` and `m` must be non-negative integers.
If there is no [ORDER BY](../../../sql-reference/statements/select/order-by.md) clause that explicitly sorts results, the choice of rows for the result may be arbitrary and non-deterministic.
## LIMIT … WITH TIES Modifier {#limit-with-ties}
When you set `WITH TIES` modifier for `LIMIT n[,m]` and specify `ORDER BY expr_list`, you will get in result first `n` or `n,m` rows and all rows with same `ORDER BY` fields values equal to row at position `n` for `LIMIT n` and `m` for `LIMIT n,m`.
This modifier also can be combined with [ORDER BY … WITH FILL modifier](../../../sql-reference/statements/select/order-by.md#orderby-with-fill).
For example, the following query
``` sql
SELECT * FROM (
SELECT number%50 AS n FROM numbers(100)
) ORDER BY n LIMIT 0,5
```
returns
``` text
┌─n─┐
│ 0 │
│ 0 │
│ 1 │
│ 1 │
│ 2 │
└───┘
```
but after apply `WITH TIES` modifier
``` sql
SELECT * FROM (
SELECT number%50 AS n FROM numbers(100)
) ORDER BY n LIMIT 0,5 WITH TIES
```
it returns another rows set
``` text
┌─n─┐
│ 0 │
│ 0 │
│ 1 │
│ 1 │
│ 2 │
│ 2 │
└───┘
```
cause row number 6 have same value “2” for field `n` as row number 5

View File

@ -69,3 +69,136 @@ If there is not enough RAM, it is possible to perform sorting in external memory
Running a query may use more memory than `max_bytes_before_external_sort`. For this reason, this setting must have a value significantly smaller than `max_memory_usage`. As an example, if your server has 128 GB of RAM and you need to run a single query, set `max_memory_usage` to 100 GB, and `max_bytes_before_external_sort` to 80 GB.
External sorting works much less effectively than sorting in RAM.
## ORDER BY Expr WITH FILL Modifier {#orderby-with-fill}
This modifier also can be combined with [LIMIT … WITH TIES modifier](../../../sql-reference/statements/select/limit.md#limit-with-ties).
`WITH FILL` modifier can be set after `ORDER BY expr` with optional `FROM expr`, `TO expr` and `STEP expr` parameters.
All missed values of `expr` column will be filled sequentially and other columns will be filled as defaults.
Use following syntax for filling multiple columns add `WITH FILL` modifier with optional parameters after each field name in `ORDER BY` section.
``` sql
ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr]
```
`WITH FILL` can be applied only for fields with Numeric (all kind of float, decimal, int) or Date/DateTime types.
When `FROM const_expr` not defined sequence of filling use minimal `expr` field value from `ORDER BY`.
When `TO const_expr` not defined sequence of filling use maximum `expr` field value from `ORDER BY`.
When `STEP const_numeric_expr` defined then `const_numeric_expr` interprets `as is` for numeric types as `days` for Date type and as `seconds` for DateTime type.
When `STEP const_numeric_expr` omitted then sequence of filling use `1.0` for numeric type, `1 day` for Date type and `1 second` for DateTime type.
For example, the following query
``` sql
SELECT n, source FROM (
SELECT toFloat32(number % 10) AS n, 'original' AS source
FROM numbers(10) WHERE number % 3 = 1
) ORDER BY n
```
returns
``` text
┌─n─┬─source───┐
│ 1 │ original │
│ 4 │ original │
│ 7 │ original │
└───┴──────────┘
```
but after apply `WITH FILL` modifier
``` sql
SELECT n, source FROM (
SELECT toFloat32(number % 10) AS n, 'original' AS source
FROM numbers(10) WHERE number % 3 = 1
) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5
```
returns
``` text
┌───n─┬─source───┐
│ 0 │ │
│ 0.5 │ │
│ 1 │ original │
│ 1.5 │ │
│ 2 │ │
│ 2.5 │ │
│ 3 │ │
│ 3.5 │ │
│ 4 │ original │
│ 4.5 │ │
│ 5 │ │
│ 5.5 │ │
│ 7 │ original │
└─────┴──────────┘
```
For the case when we have multiple fields `ORDER BY field2 WITH FILL, field1 WITH FILL` order of filling will follow the order of fields in `ORDER BY` clause.
Example:
``` sql
SELECT
toDate((number * 10) * 86400) AS d1,
toDate(number * 86400) AS d2,
'original' AS source
FROM numbers(10)
WHERE (number % 3) = 1
ORDER BY
d2 WITH FILL,
d1 WITH FILL STEP 5;
```
returns
``` text
┌───d1───────┬───d2───────┬─source───┐
│ 1970-01-11 │ 1970-01-02 │ original │
│ 0000-00-00 │ 1970-01-03 │ │
│ 0000-00-00 │ 1970-01-04 │ │
│ 1970-02-10 │ 1970-01-05 │ original │
│ 0000-00-00 │ 1970-01-06 │ │
│ 0000-00-00 │ 1970-01-07 │ │
│ 1970-03-12 │ 1970-01-08 │ original │
└────────────┴────────────┴──────────┘
```
Field `d1` doesnt fill and use default value cause we dont have repeated values for `d2` value, and sequence for `d1` cant be properly calculated.
The following query with a changed field in `ORDER BY`
``` sql
SELECT
toDate((number * 10) * 86400) AS d1,
toDate(number * 86400) AS d2,
'original' AS source
FROM numbers(10)
WHERE (number % 3) = 1
ORDER BY
d1 WITH FILL STEP 5,
d2 WITH FILL;
```
returns
``` text
┌───d1───────┬───d2───────┬─source───┐
│ 1970-01-11 │ 1970-01-02 │ original │
│ 1970-01-16 │ 0000-00-00 │ │
│ 1970-01-21 │ 0000-00-00 │ │
│ 1970-01-26 │ 0000-00-00 │ │
│ 1970-01-31 │ 0000-00-00 │ │
│ 1970-02-05 │ 0000-00-00 │ │
│ 1970-02-10 │ 1970-01-05 │ original │
│ 1970-02-15 │ 0000-00-00 │ │
│ 1970-02-20 │ 0000-00-00 │ │
│ 1970-02-25 │ 0000-00-00 │ │
│ 1970-03-02 │ 0000-00-00 │ │
│ 1970-03-07 │ 0000-00-00 │ │
│ 1970-03-12 │ 1970-01-08 │ original │
└────────────┴────────────┴──────────┘
```

View File

@ -20,9 +20,9 @@ SELECT [DISTINCT] expr_list
[WHERE expr]
[GROUP BY expr_list] [WITH TOTALS]
[HAVING expr]
[ORDER BY expr_list]
[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr]
[LIMIT [offset_value, ]n BY columns]
[LIMIT [n, ]m]
[LIMIT [n, ]m] [WITH TIES]
[UNION ALL ...]
[INTO OUTFILE filename]
[FORMAT format]

View File

@ -20,9 +20,9 @@ SELECT [DISTINCT] expr_list
[WHERE expr]
[GROUP BY expr_list] [WITH TOTALS]
[HAVING expr]
[ORDER BY expr_list]
[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr]
[LIMIT [offset_value, ]n BY columns]
[LIMIT [n, ]m]
[LIMIT [n, ]m] [WITH TIES]
[UNION ALL ...]
[INTO OUTFILE filename]
[FORMAT format]

View File

@ -20,9 +20,9 @@ SELECT [DISTINCT] expr_list
[WHERE expr]
[GROUP BY expr_list] [WITH TOTALS]
[HAVING expr]
[ORDER BY expr_list]
[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr]
[LIMIT [offset_value, ]n BY columns]
[LIMIT [n, ]m]
[LIMIT [n, ]m] [WITH TIES]
[UNION ALL ...]
[INTO OUTFILE filename]
[FORMAT format]

View File

@ -194,4 +194,4 @@ ClickHouse имеет сильную типизацию, поэтому нет
!!! note "Note"
Кластер ClickHouse состоит из независимых шардов, а каждый шард состоит из реплик. Кластер **не является эластичным** (not elastic), поэтому после добавления нового шарда данные не будут автоматически распределены между ними. Вместо этого нужно изменить настройки, чтобы выровнять нагрузку на кластер. Эта реализация дает вам больший контроль, и вполне приемлема для относительно небольших кластеров, таких как десятки узлов. Но для кластеров с сотнями узлов, которые мы используем в эксплуатации, такой подход становится существенным недостатком. Движки таблиц, которые охватывают весь кластер с динамически реплицируемыми областями, которые могут быть автоматически разделены и сбалансированы между кластерами, еще предстоит реализовать.
{## [Original article](https://clickhouse.tech/docs/ru/development/architecture/) ##}
{## [Original article](https://clickhouse.tech/docs/ru/development/architecture/) ##}

View File

@ -1261,4 +1261,91 @@ Cодержит информацию о дисках, заданных в [ко
Если политика хранения содержит несколько томов, то каждому тому соответствует отдельная запись в таблице.
## system.quotas {#system_tables-quotas}
Содержит информацию о [квотах](quotas.md).
Столбцы:
- `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)) — Хранилище квот. Возможные значения: "users.xml", если квота задана в файле users.xml, "disk" — если квота задана в SQL-запросе.
- `keys` ([Array](../sql-reference/data-types/array.md)([Enum8](../sql-reference/data-types/enum.md))) — Ключ определяет совместное использование квоты. Если два соединения используют одну и ту же квоту, они совместно используют один и тот же объем ресурсов. Значения:
- `[]`Все пользователи используют одну и ту же квоту.
- `['user_name']` — Соединения с одинаковым именем пользователя используют одну и ту же квоту.
- `['ip_address']` — Соединения с одинаковым IP-адресом используют одну и ту же квоту.
- `['client_key']` — Соединения с одинаковым ключом используют одну и ту же квоту. Ключ может быть явно задан клиентом. При использовании [clickhouse-client](../interfaces/cli.md), передайте ключевое значение в параметре `--quota-key`, или используйте параметр `quota_key` файле настроек клиента. В случае использования HTTP интерфейса, используйте заголовок `X-ClickHouse-Quota`.
- `['user_name', 'client_key']` — Соединения с одинаковым ключом используют одну и ту же квоту. Если ключ не предоставлен клиентом, то квота отслеживается для `user_name`.
- `['client_key', 'ip_address']` — Соединения с одинаковым ключом используют одну и ту же квоту. Если ключ не предоставлен клиентом, то квота отслеживается для `ip_address`.
- `durations` ([Array](../sql-reference/data-types/array.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Длины временных интервалов для расчета потребления ресурсов, в секундах.
- `apply_to_all` ([UInt8](../sql-reference/data-types/int-uint.md#uint-ranges)) — Логическое значение. Он показывает, к каким пользователям применяется квота. Значения:
- `0` — Квота применяется к пользователям, перечисленным в списке `apply_to_list`.
- `1` — Квота применяется к пользователям, за исключением тех, что перечислены в списке `apply_to_except`.
- `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))) — Список имен пользователей/ролей к которым квота применяться не должна.
## system.quota_limits {#system_tables-quota_limits}
Содержит информацию о максимумах для всех интервалов всех квот. Одной квоте могут соответствовать любое количество строк или ноль.
Столбцы:
- `quota_name` ([String](../sql-reference/data-types/string.md)) — Имя квоты.
- `duration` ([UInt32](../sql-reference/data-types/int-uint.md)) — Длина временного интервала для расчета потребления ресурсов, в секундах.
- `is_randomized_interval` ([UInt8](../sql-reference/data-types/int-uint.md#uint-ranges)) — Логическое значение. Оно показывает, является ли интервал рандомизированным. Интервал всегда начинается в одно и то же время, если он не рандомизирован. Например, интервал в 1 минуту всегда начинается с целого числа минут (то есть он может начинаться в 11:20:00, но никогда не начинается в 11:20:01), интервал в один день всегда начинается в полночь UTC. Если интервал рандомизирован, то самый первый интервал начинается в произвольное время, а последующие интервалы начинаются один за другим. Значения:
- `0` — Интервал рандомизирован.
- `1` — Интервал не рандомизирован.
- `max_queries` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное число запросов.
- `max_errors` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное количество ошибок.
- `max_result_rows` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное количество строк результата.
- `max_result_bytes` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальный объем оперативной памяти в байтах, используемый для хранения результата запроса.
- `max_read_rows` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное количество строк, считываемых из всех таблиц и табличных функций, участвующих в запросе.
- `max_read_bytes` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное количество байтов, считываемых из всех таблиц и табличных функций, участвующих в запросе.
- `max_execution_time` ([Nullable](../sql-reference/data-types/nullable.md)([Float64](../sql-reference/data-types/float.md))) — Максимальное время выполнения запроса, в секундах.
## system.quota_usage {#system_tables-quota_usage}
Использование квоты текущим пользователем: сколько используется и сколько осталось.
Столбцы:
- `quota_name` ([String](../sql-reference/data-types/string.md)) — Имя квоты.
- `quota_key`([String](../sql-reference/data-types/string.md)) — Значение ключа. Например, если keys = `ip_address`, `quota_key` может иметь значение '192.168.1.1'.
- `start_time`([Nullable](../sql-reference/data-types/nullable.md)([DateTime](../sql-reference/data-types/datetime.md))) — Время начала расчета потребления ресурсов.
- `end_time`([Nullable](../sql-reference/data-types/nullable.md)([DateTime](../sql-reference/data-types/datetime.md))) — Время окончания расчета потребления ресурс
- `duration` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Длина временного интервала для расчета потребления ресурсов, в секундах.
- `queries` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Общее количество запросов на этом интервале.
- `max_queries` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное количество запросов.
- `errors` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Число запросов, вызвавших ошибки.
- `max_errors` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное число ошибок.
- `result_rows` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Общее количество строк результата.
- `max_result_rows` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное количество строк результата.
- `result_bytes` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Объем оперативной памяти в байтах, используемый для хранения результата запроса.
- `max_result_bytes` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальный объем оперативной памяти, используемый для хранения результата запроса, в байтах.
- `read_rows` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Общее число исходных строк, считываемых из таблиц для выполнения запроса на всех удаленных серверах.
- `max_read_rows` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное количество строк, считываемых из всех таблиц и табличных функций, участвующих в запросах.
- `read_bytes` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Общее количество байт, считанных из всех таблиц и табличных функций, участвующих в запросах.
- `max_read_bytes` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное количество байт, считываемых из всех таблиц и табличных функций.
- `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))) — Максимальное время выполнения запроса.
## system.quotas_usage {#system_tables-quotas_usage}
Использование квот всеми пользователями.
Столбцы:
- `quota_name` ([String](../sql-reference/data-types/string.md)) — Имя квоты.
- `quota_key` ([String](../sql-reference/data-types/string.md)) — Ключ квоты.
- `is_current` ([UInt8](../sql-reference/data-types/int-uint.md#uint-ranges)) — Квота используется для текущего пользователя.
- `start_time` ([Nullable](../sql-reference/data-types/nullable.md)([DateTime](../sql-reference/data-types/datetime.md)))) — Время начала расчета потребления ресурсов.
- `end_time` ([Nullable](../sql-reference/data-types/nullable.md)([DateTime](../sql-reference/data-types/datetime.md)))) — Время окончания расчета потребления ресурсов.
- `duration` ([Nullable](../sql-reference/data-types/nullable.md)([UInt32](../sql-reference/data-types/int-uint.md))) — Длина временного интервала для расчета потребления ресурсов, в секундах.
- `queries` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Общее количество запросов на этом интервале.
- `max_queries` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное число запросов.
- `errors` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Число запросов, вызвавших ошибки.
- `max_errors` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное число ошибок.
- `result_rows` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — The total number of rows given as a result.
- `max_result_rows` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Maximum of source rows read from tables.
- `result_bytes` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Объем оперативной памяти в байтах, используемый для хранения результата запроса.
- `max_result_bytes` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальный объем оперативной памяти, используемый для хранения результата запроса, в байтах.
- `read_rows` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Общее число исходных строк, считываемых из таблиц для выполнения запроса на всех удаленных серверах.
- `max_read_rows` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное количество строк, считываемых из всех таблиц и табличных функций, участвующих в запросах.
- `read_bytes` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Общее количество байт, считанных из всех таблиц и табличных функций, участвующих в запросах.
- `max_read_bytes` ([Nullable](../sql-reference/data-types/nullable.md)([UInt64](../sql-reference/data-types/int-uint.md))) — Максимальное количество байт, считываемых из всех таблиц и табличных функций.
- `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))) — Максимальное время выполнения запроса.
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/) <!--hide-->

View File

@ -18,9 +18,9 @@ SELECT [DISTINCT] expr_list
[WHERE expr]
[GROUP BY expr_list] [WITH TOTALS]
[HAVING expr]
[ORDER BY expr_list]
[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr]
[LIMIT [offset_value, ]n BY columns]
[LIMIT [n, ]m]
[LIMIT [n, ]m] [WITH TIES]
[UNION ALL ...]
[INTO OUTFILE filename]
[FORMAT format]

View File

@ -1,3 +1,7 @@
---
toc_title: LIMIT
---
# Секция LIMIT {#limit-clause}
`LIMIT m` позволяет выбрать из результата первые `m` строк.
@ -7,3 +11,47 @@
`n` и `m` должны быть неотрицательными целыми числами.
При отсутствии секции [ORDER BY](order-by.md), однозначно сортирующей результат, результат может быть произвольным и может являться недетерминированным.
## Модификатор LIMIT ... WITH TIES {#limit-with-ties}
Когда вы установите модификатор WITH TIES для `LIMIT n[,m]` и указываете `ORDER BY expr_list`, вы получите первые `n` или `n,m` строк и дополнительно все строки с теми же самым значениями полей указанных в `ORDER BY` равными строке на позиции `n` для `LIMIT n` или `m` для `LIMIT n,m`.
Этот модификатор также может быть скомбинирован с [ORDER BY ... WITH FILL модификатором](../../../sql-reference/statements/select/order-by.md#orderby-with-fill)
Для примера следующий запрос
```sql
SELECT * FROM (
SELECT number%50 AS n FROM numbers(100)
) ORDER BY n LIMIT 0,5
```
возвращает
```text
┌─n─┐
│ 0 │
│ 0 │
│ 1 │
│ 1 │
│ 2 │
└───┘
```
но после применения модификатора `WITH TIES`
```sql
SELECT * FROM (
SELECT number%50 AS n FROM numbers(100)
) ORDER BY n LIMIT 0,5 WITH TIES
```
возвращает другой набор строк
```text
┌─n─┐
│ 0 │
│ 0 │
│ 1 │
│ 1 │
│ 2 │
│ 2 │
└───┘
```
поскольку строка на позиции 6 имеет тоже самое значение "2" для поля `n` что и строка на позиции 5

View File

@ -1,3 +1,7 @@
---
toc_title: ORDER BY
---
# Секция ORDER BY {#select-order-by}
Секция `ORDER BY` содержит список выражений, к каждому из которых также может быть приписано `DESC` или `ASC` (направление сортировки). Если ничего не приписано - это аналогично приписыванию `ASC`. `ASC` - сортировка по возрастанию, `DESC` - сортировка по убыванию. Обозначение направления сортировки действует на одно выражение, а не на весь список. Пример: `ORDER BY Visits DESC, SearchPhrase`
@ -31,7 +35,7 @@
└───┴──────┘
```
Выполнение запроса `SELECT * FROM t_null_nan ORDER BY y NULLS FIRST` получить:
Выполните запрос `SELECT * FROM t_null_nan ORDER BY y NULLS FIRST` чтобы получить:
``` text
┌─x─┬────y─┐
@ -66,3 +70,128 @@
Внешняя сортировка работает существенно менее эффективно, чем сортировка в оперативке.
## Модификатор ORDER BY expr WITH FILL {#orderby-with-fill}
Этот модификатор также может быть скобинирован с модификатором [LIMIT ... WITH TIES](../../../sql-reference/statements/select/limit.md#limit-with-ties)
`WITH FILL` модификатор может быть установлен после `ORDER BY expr` с опциональными параметрами `FROM expr`, `TO expr` и `STEP expr`.
Все пропущенные значнеия для колонки `expr` будут заполненые значениями соответсвующими предполагаемой последовательности значений колонки, другие колонки будут заполнены значенями по умолчанию.
Используйте следующую конструкцию для заполнения нескольких колонок с модификатором `WITH FILL` с необязательными параметрами после каждого имени поля в секции `ORDER BY`.
```sql
ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr]
```
`WITH FILL` может быть применене только к полям с числовыми (все разновидности float, int, decimal) или временными (все разновидности Date, DateTime) типами.
Когда не определен `FROM const_expr`, последовательность заполнения использует минимальное значение поля `expr` из `ORDER BY`.
Когда не определен `TO const_expr`, последовательность заполнения использует максимальное значение поля `expr` из `ORDER BY`.
Когда `STEP const_numeric_expr` определен, тогда `const_numeric_expr` интерпретируется `как есть` для числовых типов, как `дни` для типа Date и как `секунды` для типа DateTime.
Когда `STEP const_numeric_expr` не указан, тогда используется `1.0` для числовых типов, `1 день` для типа Date и `1 секунда` для типа DateTime.
Для примера, следующий запрос
```sql
SELECT n, source FROM (
SELECT toFloat32(number % 10) AS n, 'original' AS source
FROM numbers(10) WHERE number % 3 = 1
) ORDER BY n
```
возвращает
```text
┌─n─┬─source───┐
│ 1 │ original │
│ 4 │ original │
│ 7 │ original │
└───┴──────────┘
```
но после применения модификатора `WITH FILL`
```sql
SELECT n, source FROM (
SELECT toFloat32(number % 10) AS n, 'original' AS source
FROM numbers(10) WHERE number % 3 = 1
) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5
```
возвращает
```text
┌───n─┬─source───┐
│ 0 │ │
│ 0.5 │ │
│ 1 │ original │
│ 1.5 │ │
│ 2 │ │
│ 2.5 │ │
│ 3 │ │
│ 3.5 │ │
│ 4 │ original │
│ 4.5 │ │
│ 5 │ │
│ 5.5 │ │
│ 7 │ original │
└─────┴──────────┘
```
Для случая когда у нас есть несколько полей `ORDER BY field2 WITH FILL, field1 WITH FILL` порядок заполнения будет следовать порядку полей в секции `ORDER BY`.
Пример:
```sql
SELECT
toDate((number * 10) * 86400) AS d1,
toDate(number * 86400) AS d2,
'original' AS source
FROM numbers(10)
WHERE (number % 3) = 1
ORDER BY
d2 WITH FILL,
d1 WITH FILL STEP 5;
```
возвращает
```text
┌───d1───────┬───d2───────┬─source───┐
│ 1970-01-11 │ 1970-01-02 │ original │
│ 0000-00-00 │ 1970-01-03 │ │
│ 0000-00-00 │ 1970-01-04 │ │
│ 1970-02-10 │ 1970-01-05 │ original │
│ 0000-00-00 │ 1970-01-06 │ │
│ 0000-00-00 │ 1970-01-07 │ │
│ 1970-03-12 │ 1970-01-08 │ original │
└────────────┴────────────┴──────────┘
```
Поле `d1` не заполняет и используется значение по умолчанию поскольку у нас нет повторяющихся значения для `d2` поэтому мы не можем правильно рассчитать последователность заполнения для`d1`.
едующий запрос (с измененым порядком в ORDER BY)
```sql
SELECT
toDate((number * 10) * 86400) AS d1,
toDate(number * 86400) AS d2,
'original' AS source
FROM numbers(10)
WHERE (number % 3) = 1
ORDER BY
d1 WITH FILL STEP 5,
d2 WITH FILL;
```
возвращает
```text
┌───d1───────┬───d2───────┬─source───┐
│ 1970-01-11 │ 1970-01-02 │ original │
│ 1970-01-16 │ 0000-00-00 │ │
│ 1970-01-21 │ 0000-00-00 │ │
│ 1970-01-26 │ 0000-00-00 │ │
│ 1970-01-31 │ 0000-00-00 │ │
│ 1970-02-05 │ 0000-00-00 │ │
│ 1970-02-10 │ 1970-01-05 │ original │
│ 1970-02-15 │ 0000-00-00 │ │
│ 1970-02-20 │ 0000-00-00 │ │
│ 1970-02-25 │ 0000-00-00 │ │
│ 1970-03-02 │ 0000-00-00 │ │
│ 1970-03-07 │ 0000-00-00 │ │
│ 1970-03-12 │ 1970-01-08 │ original │
└────────────┴────────────┴──────────┘
```

View File

@ -33,4 +33,4 @@ clusterAllReplicas('cluster_name', db, table)
**See Also**
- [skip\_unavailable\_shards](../../operations/settings/settings.md#settings-skip_unavailable_shards)
- [load\_balancing](../../operations/settings/settings.md#settings-load_balancing)
- [load\_balancing](../../operations/settings/settings.md#settings-load_balancing)

View File

@ -62,11 +62,12 @@ Upd. Включено для системных таблиц.
Вместо этого предлагается разрешить кускам таблиц типа MergeTree располагать данные в разных форматах. А именно: - в оперативной памяти; - на диске со всеми столбцами в одном файле; - на диске со столбцами в отдельных файлах: в зависимости от размера куска и прошедшего времени. Для размещения кусков в оперативной памяти, придётся также реализовать опциональную поддержку write-ahead log с настраиваемыми правилами по сбросу на диск. Это позволит избавиться от проблем с мелкими вставками для MergeTree таблиц. Для ReplicatedMergeTree таблиц, это решит проблему лишь частично.
### 1.7. Буферизация и WAL в MergeTree {#buferizatsiia-i-wal-v-mergetree}
### 1.7. + Буферизация и WAL в MergeTree {#buferizatsiia-i-wal-v-mergetree}
Требует 1.6. Антон Попов. Задача взята в работу. Q2.
Есть pull request.
Upd. В стадии код-ревью.
Upd. Готово для использования начиная с версии 20.6 в экспериментальном режиме.
### 1.8. + Перенос между разделами по TTL {#perenos-mezhdu-razdelami-po-ttl}
@ -127,6 +128,8 @@ Q4.
Upd. Олег будет делать только часть про HDFS.
Upd. Реализация поверх S3 является рабочей на уровне PoC.
Upd. Реализация поверх S3 ужасно тормозит и сейчас доделывается.
Upd. Реализацию поверх HDFS взял AmosBird для Kuaishou.
### 1.13. + Ускорение запросов с FINAL {#uskorenie-zaprosov-s-final}
@ -379,6 +382,7 @@ Upd. Появилась вторая версия LTS - 20.3.
### 6.8. Таблица system.crashes {#tablitsa-system-crashes}
Сравнительно простая задача, но только для опытных разработчиков.
Upd. В разработке.
### 6.9. + Отправлять информацию клиенту, если сервер падает по сигналу {#otpravliat-informatsiiu-klientu-esli-server-padaet-po-signalu}
@ -525,7 +529,7 @@ https://github.com/ClickHouse/ClickHouse/issues/8027\#issuecomment-566670282
Проверили на настоящем сервере Huawei, а также в специальном Docker контейнере, который содержит внутри qemu-user-static.
Также можно проверить на Cavium, на Raspberry Pi а также на твоём Android телефоне.
### 7.20. + Автосборка для FreeBSD x86\_64 {#avtosborka-dlia-freebsd-x86-64}
### 7.20. + Автосборка для FreeBSD x86_64 {#avtosborka-dlia-freebsd-x86-64}
[Иван Лежанкин](https://github.com/abyss7).
@ -604,6 +608,8 @@ Upd. Сделаны fuzzBits, fuzzBytes.
2. Генерация и выполнение случайных синтаксически корректных запросов на случайных данных.
Upd. Александр Кузьменков сделал фазер запросов на основе мутаций AST.
### 7.25. + Синхронизация релизов в Аркадию {#sinkhronizatsiia-relizov-v-arkadiiu}
Изначально занимался Олег Алексеенков. Сейчас он перешёл работать в дружественный отдел, но обещает продолжать синхронизацию.
@ -626,10 +632,10 @@ Upd: Добавлены все файлы побайтово.
Команда DevTools. Прогресс по задаче под вопросом.
Upd. Готово (все директории кроме contrib).
### 7.27. Запуск автотестов в Аркадии {#zapusk-avtotestov-v-arkadii}
### 7.27. + Запуск автотестов в Аркадии {#zapusk-avtotestov-v-arkadii}
Требует 7.26. Коллеги начали делать, есть результат.
Upd. В Аркадии частично работает небольшая часть тестов.
Upd. В Аркадии частично работает небольшая часть тестов. И этого достаточно.
### 7.29. Опции clickhouse install, stop, start вместо postinst, init.d, systemd скриптов {#optsii-clickhouse-install-stop-start-vmesto-postinst-init-d-systemd-skriptov}
@ -659,10 +665,11 @@ Upd. В Аркадии частично работает небольшая ча
В очереди. Иван Лежанкин.
Отсутствует прогресс.
### 7.34. Бэкпортировать bugfix автоматически {#bekportirovat-bugfix-avtomaticheski}
### 7.34. + Бэкпортировать bugfix автоматически {#bekportirovat-bugfix-avtomaticheski}
В очереди. Иван Лежанкин.
Присутствует прогресс.
Сделано.
### 7.35. Начальные правила для авто-merge {#nachalnye-pravila-dlia-avto-merge}
@ -676,7 +683,7 @@ Upd. В Аркадии частично работает небольшая ча
Сейчас добавляем некоторых доверенных контрибьюторов в ручном режиме.
Upd. Всё ещё добавляем в ручном режиме.
### 7.37. Разобраться с repo.yandex.ru {#razobratsia-s-repo-yandex-ru}
### 7.37. + Разобраться с repo.yandex.ru {#razobratsia-s-repo-yandex-ru}
Есть жалобы на скорость загрузки и неудобство maintenance, operations, visibility.
@ -733,11 +740,12 @@ Upd. Задача взята в работу.
Артемий Бобровский, ВШЭ
Есть pull request.
### 8.11. Движок таблиц для чтения из Mongo {#dvizhok-tablits-dlia-chteniia-iz-mongo}
### 8.11. + Движок таблиц для чтения из Mongo {#dvizhok-tablits-dlia-chteniia-iz-mongo}
Артемий Бобровский, ВШЭ
Есть pull request.
Upd. В стадии код-ревью.
Готово.
### 8.12. Пропуск столбцов в форматах Parquet, ORC {#propusk-stolbtsov-v-formatakh-parquet-orc}
@ -745,9 +753,9 @@ Upd. В стадии код-ревью.
### 8.13. Поддержка массивов в Parquet, ORC {#podderzhka-massivov-v-parquet-orc}
### 8.14. Запись данных в ORC {#zapis-dannykh-v-orc}
### 8.14. + Запись данных в ORC {#zapis-dannykh-v-orc}
Павел Круглов, ВШЭ. Есть pull request.
Павел Круглов, ВШЭ. Есть pull request. Готово.
### 8.15. Запись данных в CapNProto {#zapis-dannykh-v-capnproto}
@ -790,7 +798,7 @@ Upd. Готово.
Maxim Fedotov, Wargaming + Yuri Baranov, Яндекс.
### 8.19. Интеграция с RabbitMQ {#integratsiia-s-rabbitmq}
### 8.19. + Интеграция с RabbitMQ {#integratsiia-s-rabbitmq}
Ксения Сумарокова, ВШЭ.
@ -800,10 +808,11 @@ Maxim Fedotov, Wargaming + Yuri Baranov, Яндекс.
Есть pull request в процессе разработки.
Upd. В процессе code review.
Upd. Готово.
### 8.20. Интеграция с SQS {#integratsiia-s-sqs}
### 8.20. - Интеграция с SQS {#integratsiia-s-sqs}
Низкий приоритет.
Низкий приоритет. Отменено.
### 8.21. Поддержка произвольного количества языков для имён регионов {#podderzhka-proizvolnogo-kolichestva-iazykov-dlia-imion-regionov}
@ -832,8 +841,6 @@ Upd. Сделано теми людьми, кому не запрещено ра
### 9.2. Преднастроенные именованные соединения к внешним БД {#prednastroennye-imenovannye-soedineniia-k-vneshnim-bd}
Валерий Батурин, ВШЭ.
ClickHouse предоставляет возможность обратиться к внешней базе данных из языка запросов. Это реализовано в виде табличных функций. В параметрах к табличной функции указывается адрес удалённой базы данных (хост, порт), а также аутентификационные данные (имя пользователя, пароль). Аутентификационные данные указываются в запросе в открытом виде и, таким образом, попадают в историю запросов и в логи, что компрометирует безопасность системы.
Вместо этого предлагается описывать необходимые данные в конфигурационном файле сервера или в отдельном сервисе и ссылаться на них по именам.
@ -883,12 +890,8 @@ Upd. Ура, нашли причину и исправили.
### 10.7. Поддержка Nullable в словарях {#podderzhka-nullable-v-slovariakh}
Артём Стрельцов, Николай Дегтеринский, Наталия Михненко, ВШЭ.
### 10.8. Поддержка массивов в словарях {#podderzhka-massivov-v-slovariakh}
Артём Стрельцов, Николай Дегтеринский, Наталия Михненко, ВШЭ.
### 10.9. - Уменьшение блокировок для cache словарей за счёт одновременных запросов одного и того же {#umenshenie-blokirovok-dlia-cache-slovarei-za-schiot-odnovremennykh-zaprosov-odnogo-i-togo-zhe}
Заменено в пользу 10.10, 10.11.
@ -1022,6 +1025,8 @@ Q3.
### 13.3. Пулы ресурсов {#puly-resursov}
Александр Казаков.
Требует 13.2 или сможем сделать более неудобную реализацию раньше.
Обсуждается вариант неудобной реализации. Пока средний приоритет, целимся на Q1/Q2.
Вариант реализации выбрал Александр Казаков.
@ -1065,10 +1070,10 @@ zhang2014
Upd. Есть pull request-ы.
Upd. DISTINCT готов.
### 14.9. Поддержка запроса EXPLAIN {#podderzhka-zaprosa-explain}
### 14.9. + Поддержка запроса EXPLAIN {#podderzhka-zaprosa-explain}
Требует 2.1. [Николай Кочетов](https://github.com/KochetovNicolai).
Upd. Есть pull request.
Upd. Есть pull request. Готово.
### 14.10. arrayReduce как функция высшего порядка {#arrayreduce-kak-funktsiia-vysshego-poriadka}
@ -1229,7 +1234,7 @@ Upd. Есть pull request.
Предлагается реализовать в ClickHouse статистические тесты (Analysis of Variance, тесты нормальности распределения и т. п.) в виде агрегатных функций. Пример: `welchTTest(value, sample_idx)`.
Сделали прототип двух тестов, есть pull request.
Сделали прототип двух тестов, есть pull request. Также есть pull request для корелляции рангов.
### 18.3. Инфраструктура для тренировки моделей в ClickHouse {#infrastruktura-dlia-trenirovki-modelei-v-clickhouse}
@ -1346,6 +1351,8 @@ Upd. Для DISTINCT есть pull request.
### 21.6. Уменьшение числа потоков для SELECT в случае тривиального INSERT SELECT {#umenshenie-chisla-potokov-dlia-select-v-sluchae-trivialnogo-insert-select}
ucasFL, в разработке.
### 21.7. Кэш результатов запросов {#kesh-rezultatov-zaprosov}
[Achimbab](https://github.com/achimbab).
@ -1363,6 +1370,7 @@ Upd. В обсуждении.
Для domain-specific кэшей (как например, кэш разжатых данных) выгодно, чтобы они использовали как можно больший объём свободной памяти. Но в этом случае, памяти может не хватить для других структур данных в программе. Если аллокатор памяти знает про кэш, то выделение памяти можно было бы делать путём вытеснения данных из кэша.
Upd. Есть нерабочий прототип, скорее всего будет отложено.
Upd. Отложено до осени.
### 21.8.1. Отдельный аллокатор для кэшей с ASLR {#otdelnyi-allokator-dlia-keshei-s-aslr}
@ -1390,6 +1398,7 @@ Amos Bird.
Сделана замена цепочек if на multiIf, но внезапно оказалось, что это является не оптимизацией, а наоборот.
Сделано ещё несколько оптимизаций.
Upd. Все вышеперечисленные оптимизации доступны в pull requests.
Upd. Из них почти все помержены, осталась одна.
### 21.12. Алгебраические оптимизации запросов {#algebraicheskie-optimizatsii-zaprosov}
@ -1404,6 +1413,8 @@ Upd. Все вышеперечисленные оптимизации досту
- Удаление дублирующихся DISTINCT, ORDER BY из подзапросов.
Несколько оптимизаций есть в PR.
Upd. Все оптимизации кроме "Обращение инъективных функций в сравнениях на равенство" есть в PR.
Upd. Из них больше половины помержены, осталось ещё две.
### 21.13. Fusion агрегатных функций {#fusion-agregatnykh-funktsii}
@ -1415,6 +1426,8 @@ Constraints позволяют задать выражение, истиннос
Если выражение содержит равенство, то встретив в запросе одну из частей равенства, её можно заменить на другую часть равенства, если это сделает проще чтение данных или вычисление выражения. Например, задан constraint: `URLDomain = domain(URL)`. Значит, выражение `domain(URL)` можно заменить на `URLDomain`.
Upd. Возможно будет отложено на следующий год.
### 21.15. Многоступенчатое чтение данных вместо PREWHERE {#mnogostupenchatoe-chtenie-dannykh-vmesto-prewhere}
Требует 2.1 и 21.10.
@ -1459,6 +1472,7 @@ Constraints позволяют задать выражение, истиннос
### 21.20. Использование материализованных представлений для оптимизации запросов {#ispolzovanie-materializovannykh-predstavlenii-dlia-optimizatsii-zaprosov}
В ByteDance есть готовая реализация, но они её боятся из-за, возможно, низкого качества кода.
Upd. Вместо этого будем делать задачу 1.16.
### 21.21. + Чтение больших файлов с помощью mmap {#chtenie-bolshikh-failov-s-pomoshchiu-mmap}
@ -1540,11 +1554,12 @@ Upd. Добавили таймауты.
### 22.15. Нормализация коммитов в Kafka и идемпотентности операций {#normalizatsiia-kommitov-v-kafka-i-idempotentnosti-operatsii}
Altinity.
Я не в курсе, какой статус.
### 22.16. + Исправление низкой производительности кодека DoubleDelta {#ispravlenie-nizkoi-proizvoditelnosti-kodeka-doubledelta}
Василий Немков, Altinity - в процессе.
Можно считать, что сделано, хотя отсутствие SIMD оптимизаций для variable length кодеков - это ужасно.
Upd. Готово.
### 22.17. Консистентно работающий POPULATE для MaterializedView {#konsistentno-rabotaiushchii-populate-dlia-materializedview}
@ -1602,16 +1617,16 @@ Altinity.
## 23. Default Festival {#default-festival}
### 23.1. + Включение minimalistic\_part\_header в ZooKeeper {#vkliuchenie-minimalistic-part-header-v-zookeeper}
### 23.1. + Включение minimalistic_part_header в ZooKeeper {#vkliuchenie-minimalistic-part-header-v-zookeeper}
Сильно уменьшает объём данных в ZooKeeper. Уже год в продакшене в Яндекс.Метрике.
Алексей Миловидов, ноябрь 2019.
### 23.2. Включение distributed\_aggregation\_memory\_efficient {#vkliuchenie-distributed-aggregation-memory-efficient}
### 23.2. Включение distributed_aggregation_memory_efficient {#vkliuchenie-distributed-aggregation-memory-efficient}
Есть риски меньшей производительности лёгких запросов, хотя производительность тяжёлых запросов всегда увеличивается.
### 23.3. Включение min\_bytes\_to\_external\_sort и min\_bytes\_to\_external\_group\_by {#vkliuchenie-min-bytes-to-external-sort-i-min-bytes-to-external-group-by}
### 23.3. Включение min_bytes_to_external_sort и min_bytes_to_external_group_by {#vkliuchenie-min-bytes-to-external-sort-i-min-bytes-to-external-group-by}
Желательно 5.2. и 13.1.
@ -1619,11 +1634,11 @@ Altinity.
Есть гипотеза, что плохо работает на очень больших кластерах.
### 23.5. Включение compile\_expressions {#vkliuchenie-compile-expressions}
### 23.5. Включение compile_expressions {#vkliuchenie-compile-expressions}
Требует 7.2. Задачу изначально на 99% сделал Денис Скоробогатов, ВШЭ и Яндекс. Остальной процент доделывал Алексей Миловидов, а затем [Александр Сапин](https://github.com/alesapin).
### 23.6. Включение учёта порядка столбцов в CSV {#vkliuchenie-uchiota-poriadka-stolbtsov-v-csv}
### 23.6. + Включение учёта порядка столбцов в CSV {#vkliuchenie-uchiota-poriadka-stolbtsov-v-csv}
Просто аккуратно включить.
@ -1668,13 +1683,11 @@ ClickHouse поддерживает LZ4 и ZSTD для сжатия данных
### 24.3. Экспериментальные кодеки {#eksperimentalnye-kodeki}
Вероника Фалчикова, Лада Торчик, ВШЭ.
Существуют специализированные алгоритмы кодирования числовых последовательностей: Group VarInt, MaskedVByte, PFOR. Необходимо изучить наиболее эффективные реализации этих алгоритмов. Примеры вы сможете найти на https://github.com/lemire и https://github.com/powturbo/ а также https://github.com/schizofreny/middle-out
Внедрить их в ClickHouse в виде кодеков и изучить их работу на тестовых датасетах.
Upd. Есть два pull requests в начальной стадии, возможно будет отложено.
Upd. Есть два pull requests в начальной стадии, отложено.
### 24.4. Шифрование в ClickHouse на уровне VFS {#shifrovanie-v-clickhouse-na-urovne-vfs}
@ -1692,7 +1705,7 @@ Upd. Есть два pull requests в начальной стадии, возм
2. Шифрование отдельных значений.
Для этого требуется реализовать функции шифрования и расшифрования, доступные из SQL. Для шифрования реализовать возможность добавления нужного количества случайных бит для исключения одинаковых зашифрованных значений на одинаковых данных. Это позволит реализовать возможность «забывания» данных без удаления строк таблицы: можно шифровать данные разных клиентов разными ключами, и для того, чтобы забыть данные одного клиента, потребуется всего лишь удалить ключ.
Будет делать Василий Немков, Altinity
Делает Василий Немков, Altinity
### 24.6. Userspace RAID {#userspace-raid}
@ -1796,6 +1809,8 @@ Upd. Прототип bitonic sort помержен, но целесообраз
Требует 2.1.
Upd. Есть два прототипа от внешних контрибьюторов.
### 24.15. Поддержка полуструктурированных данных {#podderzhka-polustrukturirovannykh-dannykh}
Требует 1.14 и 2.10.
@ -1889,7 +1904,7 @@ ucasFL, ICT.
Жанна Зосимова, ВШЭ.
Upd. Пока поддержали Arrow как формат ввода-вывода.
### - 24.30. - ClickHouse как графовая СУБД {#clickhouse-kak-grafovaia-subd}
### 24.30. - ClickHouse как графовая СУБД {#clickhouse-kak-grafovaia-subd}
Amos Bird, но его решение слишком громоздкое и пока не open-source. Отменено.
@ -1971,10 +1986,14 @@ Amos Bird, но его решение слишком громоздкое и п
Алексей Миловидов и все подготовленные докладчики.
Upd. Есть Saint HighLoad online.
Upd. Есть C++ Russia.
CodeFest, DUMP, UWDC отменились.
### 25.15. Конференции зарубежные: Percona, DataOps, попытка попасть на более крупные {#konferentsii-zarubezhnye-percona-dataops-popytka-popast-na-bolee-krupnye}
Алексей Миловидов и все подготовленные докладчики
Upd. Есть Percona.
DataOps отменилась.
### 25.16. Сайт play.clickhouse {#sait-play-clickhouse}
@ -1995,6 +2014,7 @@ Upd. Есть Saint HighLoad online.
Алексей Миловидов и вся группа разработки.
Благодаря Robert Hodges добавлен CMU.
Upd. Взаимодействие с ВШЭ 2019/2020 успешно выполнено.
### 25.18. - Лекция в ШАД {#lektsiia-v-shad}
@ -2012,6 +2032,8 @@ Upd. Есть поползновения с TDEngine.
### 25.21. Повторное награждение контрибьюторов в Китае {#povtornoe-nagrazhdenie-kontribiutorov-v-kitae}
Upd. Ждём снятия ограничений и восстановления активности по онлайн митапам.
### 25.22. On-site помощь с ClickHouse компаниям в дни рядом с мероприятиями {#on-site-pomoshch-s-clickhouse-kompaniiam-v-dni-riadom-s-meropriiatiiami}
[Иван Блинков](https://github.com/blinkov/) - организация. Провёл мероприятие для турецкой компании.
@ -2019,6 +2041,8 @@ Upd. On-site заменяется на Online.
### 25.23. Новый мерч для ClickHouse {#novyi-merch-dlia-clickhouse}
Upd. Старого пока хватает, раздача уменьшилась.
### 25.24. Конкурсы bughunter или оптимизации кода на C++ {#konkursy-bughunter-ili-optimizatsii-koda-na-c}
Проведение конкурсов должно начинаться для сотрудников Яндекса, пока нет согласования.

View File

@ -3,9 +3,9 @@ toc_priority: 35
toc_title: 算术函数
---
# 算术函数 {#suan-zhu-han-shu}
# 算术函数 {#arithmetic-functions}
对于所有算术函数,结果类型为结果适合的最小数字类型(如果存在这样的类型)。最小数字类型是根据数字的位数,是否有符号以及是否是浮点类型而同时进行的。如果没有足够的位,则采用最高位类型。
对于所有算术函数,结果类型为结果适合的最小数值类型(如果存在这样的类型)。最小数值类型是根据数值的位数,是否有符号以及是否是浮点类型而同时进行的。如果没有足够的位,则采用最高位类型。
例如:
@ -21,61 +21,63 @@ SELECT toTypeName(0), toTypeName(0 + 0), toTypeName(0 + 0 + 0), toTypeName(0 + 0
溢出的产生方式与C++相同。
## 加(a,b),a+b {#plusa-b-a-b}
## plus(a, b), a + b operator {#plusa-b-a-b-operator}
计算数的总和。
您还可以将Date或DateTime与整数进行相加。在Date的情况下添加的整数意味着添加相应的天数。对于DateTime这意味这添加相应的描述
计算数的总和。
您还可以将Date或DateTime与整数进行相加。在Date的情况下和整数相加整数意味着添加相应的天数。对于DateTime这意味着添加相应的秒数
## 减(a,b),a-b {#minusa-b-a-b}
## minus(a, b), a - b operator {#minusa-b-a-b-operator}
计算数之间的差,结果总是有符号的。
计算数之间的差,结果总是有符号的。
您还可以将Date或DateTime与整数进行相减。见上面的plus
## 乘(a,b),a\*b {#multiplya-b-a-b}
## multiply(a, b), a \* b operator {#multiplya-b-a-b-operator}
计算数的乘积。
计算数的乘积。
## 除以(a,b),a/b {#dividea-b-a-b}
## divide(a, b), a / b operator {#dividea-b-a-b-operator}
计算数的商。结果类型始终是浮点类型。
计算数的商。结果类型始终是浮点类型。
它不是整数除法。对于整数除法请使用intDiv函数。
当除以零时你得到inf- infnan
## intDiv(a,b) {#intdiva-b}
计算整数数字的商,向下舍入(按绝对值)。
计算数值的商,向下舍入取整(按绝对值)。
除以零或将最小负数除以-1时抛出异常。
## intDivOrZero(a,b) {#intdivorzeroa-b}
intDiv的不同之处在于它在除以零或将最小负数除以-1时返回零。
## 模(a,b),a%b {#moduloa-b-a-b}
## modulo(a, b), a % b operator {#modulo}
计算除法后的余数。
如果参数是浮点数,则通过删除小数部分将它们预转换为整数。
其余部分与C++中的含义相同。截断除法用于负数。
除以零或将最小负数除以-1时抛出异常。
## 否定(a),-a {#negatea-a}
## moduloOrZero(a, b) {#modulo-or-zero}
计算一个数字的
用反转符号计算一个数字。结果始终是签名的。
计算具有反向符号的数字。 结果始终签名。
和[modulo](#modulo)不同之处在于除以0时结果返回0
## negate(a), -a operator {#negatea-a-operator}
通过改变数值的符号位对数值取反,结果总是有符号的
## abs(a) {#arithm_func-abs}
计算数a的绝对值。也就是说如果a lt; 0它返回-a。对于无符号类型它不执行任何操作。对于有符号整数类型它返回无符号数。
计算数a的绝对值。也就是说如果a \< 0它返回-a。对于无符号类型它不执行任何操作。对于有符号整数类型它返回无符号数。
## gcd(a,b) {#gcda-b}
返回数的最大公约数。
返回数的最大公约数。
除以零或将最小负数除以-1时抛出异常。
## lcm(a,b) {#lcma-b}
返回数的最小公倍数。
返回数的最小公倍数。
除以零或将最小负数除以-1时抛出异常。
[来源文章](https://clickhouse.tech/docs/en/query_language/functions/arithmetic_functions/) <!--hide-->

View File

@ -1,19 +1,24 @@
---
toc_priority: 60
toc_title: IN 运算符
---
# IN运算符相关函数 {#inyun-suan-fu-xiang-guan-han-shu}
## in,notIn,globalIn,globalNotIn {#in-notin-globalin-globalnotin}
请参阅[IN 运算符](../operators/in.md#select-in-operators)部分。
请参阅[IN 运算符](../../sql-reference/operators/in.md#select-in-operators)部分。
## tuple(x, y, …), operator (x, y, …) {#tuplex-y-operator-x-y}
## tuple(x, y, …), 运算符 (x, y, …) {#tuplex-y-operator-x-y}
函数用于对多个列进行分组。
对于具有类型T1T2…的列它返回包含这些列的元组T1T2。 执行该函数没有任何成本。
元组通常用作IN运算符的中间参数值或用于创建lambda函数的形参列表。 元组不能写入表。
## 元组元素(元组,n),运算符x.N {#tupleelementtuple-n-operator-x-n}
## tupleElement(tuple, n), 运算符 x.N {#tupleelementtuple-n-operator-x-n}
函数用于从元组中获取列。
N是列索引从1开始。N必须是常量正整数常数,并且不大于元组的大小。
用于从元组中获取列的函数
N是列索引从1开始。N必须是正整数常量,并且不大于元组的大小。
执行该函数没有任何成本。
[来源文章](https://clickhouse.tech/docs/en/query_language/functions/in_functions/) <!--hide-->
[原始文章](https://clickhouse.tech/docs/en/query_language/functions/in_functions/) <!--hide-->

View File

@ -24,9 +24,9 @@ SELECT [DISTINCT] expr_list
[WHERE expr]
[GROUP BY expr_list] [WITH TOTALS]
[HAVING expr]
[ORDER BY expr_list]
[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr]
[LIMIT [offset_value, ]n BY columns]
[LIMIT [n, ]m]
[LIMIT [n, ]m] [WITH TIES]
[UNION ALL ...]
[INTO OUTFILE filename]
[FORMAT format]

View File

@ -84,7 +84,8 @@ public:
UInt16 cur_port = i >= ports_.size() ? 9000 : ports_[i];
std::string cur_host = i >= hosts_.size() ? "localhost" : hosts_[i];
connections.emplace_back(std::make_unique<ConnectionPool>(concurrency, cur_host, cur_port, default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure));
connections.emplace_back(std::make_unique<ConnectionPool>(
concurrency, cur_host, cur_port, default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure));
comparison_info_per_interval.emplace_back(std::make_shared<Stats>());
comparison_info_total.emplace_back(std::make_shared<Stats>());
}
@ -563,7 +564,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
desc.add_options()
("help", "produce help message")
("concurrency,c", value<unsigned>()->default_value(1), "number of parallel queries")
("delay,d", value<double>()->default_value(1), "delay between intermediate reports in seconds (set 0 to disable reports)")
("delay,d", value<double>()->default_value(1), "delay between intermediate reports in seconds (set 0 to disable reports)")
("stage", value<std::string>()->default_value("complete"), "request query processing up to specified stage: complete,fetch_columns,with_mergeable_state")
("iterations,i", value<size_t>()->default_value(0), "amount of queries to be executed")
("timelimit,t", value<double>()->default_value(0.), "stop launch of queries after specified time limit")
@ -600,8 +601,15 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
print_stacktrace = options.count("stacktrace");
/// NOTE Maybe clickhouse-benchmark should also respect .xml configuration of clickhouse-client.
UInt16 default_port = options.count("secure") ? DBMS_DEFAULT_SECURE_PORT : DBMS_DEFAULT_PORT;
UseSSL use_ssl;
Ports ports = options.count("port") ? options["port"].as<Ports>() : Ports({9000});
Ports ports = options.count("port")
? options["port"].as<Ports>()
: Ports({default_port});
Strings hosts = options.count("host") ? options["host"].as<Strings>() : Strings({"localhost"});
Benchmark benchmark(

View File

@ -1054,6 +1054,10 @@ private:
auto base_before_fuzz = fuzz_base->formatForErrorMessage();
ast_to_process = fuzz_base->clone();
std::stringstream dump_of_cloned_ast;
ast_to_process->dumpTree(dump_of_cloned_ast);
fuzzer.fuzzMain(ast_to_process);
auto base_after_fuzz = fuzz_base->formatForErrorMessage();
@ -1066,6 +1070,8 @@ private:
base_after_fuzz.c_str());
fprintf(stderr, "dump before fuzz:\n%s\n",
dump_before_fuzz.str().c_str());
fprintf(stderr, "dump of cloned ast:\n%s\n",
dump_of_cloned_ast.str().c_str());
fprintf(stderr, "dump after fuzz:\n");
fuzz_base->dumpTree(std::cerr);
assert(false);

View File

@ -1,3 +1,5 @@
#if defined(OS_LINUX)
#include <sys/types.h>
#include <sys/stat.h>
#include <fcntl.h>
@ -101,3 +103,5 @@ MemoryStatisticsOS::Data MemoryStatisticsOS::get() const
}
}
#endif

View File

@ -1,4 +1,5 @@
#pragma once
#if defined(OS_LINUX)
#include <cstdint>
@ -38,3 +39,5 @@ private:
};
}
#endif

View File

@ -212,6 +212,21 @@
M(NotCreatedLogEntryForMerge, "Log entry to merge parts in ReplicatedMergeTree is not created due to concurrent log update by another replica.") \
M(CreatedLogEntryForMutation, "Successfully created log entry to mutate parts in ReplicatedMergeTree.") \
M(NotCreatedLogEntryForMutation, "Log entry to mutate parts in ReplicatedMergeTree is not created due to concurrent log update by another replica.") \
\
M(S3ReadMicroseconds, "Time of GET and HEAD requests to S3 storage.") \
M(S3ReadBytes, "Read bytes (incoming) in GET and HEAD requests to S3 storage.") \
M(S3ReadRequestsCount, "Number of GET and HEAD requests to S3 storage.") \
M(S3ReadRequestsErrors, "Number of non-throttling errors in GET and HEAD requests to S3 storage.") \
M(S3ReadRequestsThrottling, "Number of 429 and 503 errors in GET and HEAD requests to S3 storage.") \
M(S3ReadRequestsRedirects, "Number of redirects in GET and HEAD requests to S3 storage.") \
\
M(S3WriteMicroseconds, "Time of POST, DELETE, PUT and PATCH requests to S3 storage.") \
M(S3WriteBytes, "Write bytes (outgoing) in POST, DELETE, PUT and PATCH requests to S3 storage.") \
M(S3WriteRequestsCount, "Number of POST, DELETE, PUT and PATCH requests to S3 storage.") \
M(S3WriteRequestsErrors, "Number of non-throttling errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \
M(S3WriteRequestsThrottling, "Number of 429 and 503 errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \
M(S3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to S3 storage.") \
namespace ProfileEvents
{

View File

@ -21,14 +21,14 @@
/** Which blocks by default read the data (by number of rows).
* Smaller values give better cache locality, less consumption of RAM, but more overhead to process the query.
*/
#define DEFAULT_BLOCK_SIZE 65536
#define DEFAULT_BLOCK_SIZE 65505 /// 65536 minus 16 + 15 bytes padding that we usually have in arrays
/** Which blocks should be formed for insertion into the table, if we control the formation of blocks.
* (Sometimes the blocks are inserted exactly such blocks that have been read / transmitted from the outside, and this parameter does not affect their size.)
* More than DEFAULT_BLOCK_SIZE, because in some tables a block of data on the disk is created for each block (quite a big thing),
* and if the parts were small, then it would be costly then to combine them.
*/
#define DEFAULT_INSERT_BLOCK_SIZE 1048576
#define DEFAULT_INSERT_BLOCK_SIZE 1048545 /// 1048576 minus 16 + 15 bytes padding that we usually have in arrays
/** The same, but for merge operations. Less DEFAULT_BLOCK_SIZE for saving RAM (since all the columns are read).
* Significantly less, since there are 10-way mergers.

View File

@ -370,13 +370,15 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \
M(SettingBool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \
M(SettingBool, optimize_duplicate_order_by_and_distinct, true, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \
M(SettingBool, optimize_redundant_functions_in_order_by, true, "Remove functions from ORDER BY if its argument is also in ORDER BY", 0) \
M(SettingBool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \
M(SettingBool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \
M(SettingBool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \
M(SettingBool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \
\
M(SettingBool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \
M(SettingBool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \
M(SettingUInt64, multiple_joins_rewriter_version, 1, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \
M(SettingUInt64, multiple_joins_rewriter_version, 2, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \
M(SettingBool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \
M(SettingUInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \
M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \

View File

@ -1,4 +1,4 @@
#if defined(__linux__) || defined(__FreeBSD__)
#if defined(OS_LINUX) || defined(__FreeBSD__)
#include "SSDCacheDictionary.h"

View File

@ -1,4 +1,4 @@
#if defined(__linux__) || defined(__FreeBSD__)
#if defined(OS_LINUX) || defined(__FreeBSD__)
#include "SSDComplexKeyCacheDictionary.h"

View File

@ -1,6 +1,6 @@
#pragma once
#if defined(__linux__) || defined(__FreeBSD__)
#if defined(OS_LINUX) || defined(__FreeBSD__)
#include "DictionaryStructure.h"
#include "IDictionary.h"

View File

@ -33,7 +33,7 @@ void registerDictionaries()
registerDictionaryFlat(factory);
registerDictionaryHashed(factory);
registerDictionaryCache(factory);
#if defined(__linux__) || defined(__FreeBSD__)
#if defined(OS_LINUX) || defined(__FreeBSD__)
registerDictionarySSDCache(factory);
registerDictionarySSDComplexKeyCache(factory);
#endif

View File

@ -7,6 +7,7 @@
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadBufferFromS3.h>
#include <IO/ReadHelpers.h>
#include <IO/SeekAvoidingReadBuffer.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromS3.h>
#include <IO/WriteHelpers.h>
@ -423,7 +424,8 @@ DiskS3::DiskS3(
String s3_root_path_,
String metadata_path_,
size_t min_upload_part_size_,
size_t min_multi_part_upload_size_)
size_t min_multi_part_upload_size_,
size_t min_bytes_for_seek_)
: name(std::move(name_))
, client(std::move(client_))
, proxy_configuration(std::move(proxy_configuration_))
@ -432,6 +434,7 @@ DiskS3::DiskS3(
, metadata_path(std::move(metadata_path_))
, min_upload_part_size(min_upload_part_size_)
, min_multi_part_upload_size(min_multi_part_upload_size_)
, min_bytes_for_seek(min_bytes_for_seek_)
{
}
@ -537,7 +540,8 @@ std::unique_ptr<ReadBufferFromFileBase> DiskS3::readFile(const String & path, si
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Read from file by path: {}. Existing S3 objects: {}",
backQuote(metadata_path + path), metadata.s3_objects.size());
return std::make_unique<ReadIndirectBufferFromS3>(client, bucket, metadata, buf_size);
auto reader = std::make_unique<ReadIndirectBufferFromS3>(client, bucket, metadata, buf_size);
return std::make_unique<SeekAvoidingReadBuffer>(std::move(reader), min_bytes_for_seek);
}
std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path, size_t buf_size, WriteMode mode, size_t estimated_size, size_t)

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