Merge remote-tracking branch 'origin/master' into HEAD

This commit is contained in:
Alexander Kuzmenkov 2020-07-15 19:09:43 +03:00
commit fda2595958
95 changed files with 1758 additions and 918 deletions

View File

@ -9,13 +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

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

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

@ -498,7 +498,8 @@ 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
@ -666,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

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

@ -2,11 +2,13 @@
toc_priority: 6
toc_title: RabbitMQ
---
# RabbitMQ Engine
# 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.
@ -43,7 +45,7 @@ Optional parameters:
- `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`.
- `rabbitmq_transactional_channel` Wrap insert queries in transactions. Default: `0`.
Required configuration:
@ -83,6 +85,7 @@ Data can be channeled based on `rabbitmq_exchange_type` and the specified `rabbi
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`.
@ -92,6 +95,7 @@ Exchange type options:
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).
@ -116,4 +120,3 @@ Example:
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

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

@ -1,16 +1,17 @@
# system.quota_limits {#system_tables-quota_limits}
# 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.
- `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

@ -1,24 +1,24 @@
# system.quota_usage {#system_tables-quota_usage}
# system.quota\_usage {#system_tables-quota_usage}
Quota usage by the current user: how much is used and how much is left.
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.
- `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

@ -1,20 +1,21 @@
# system.quotas {#system_tables-quotas}
Contains information about [quotas](quotas.md).
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 isn't 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 isn't 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.
- `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

@ -1,25 +1,25 @@
# system.quotas_usage {#system_tables-quotas_usage}
# system.quotas\_usage {#system_tables-quotas_usage}
Quota usage by all users.
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.
- `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

@ -22,14 +22,14 @@ Strings are compared by bytes. A shorter string is smaller than all strings that
## equals, a = b and a == b operator {#function-equals}
## notEquals, a != b and a <> b operator {#function-notequals}
## notEquals, a != b and a \<\> b operator {#function-notequals}
## less, < operator {#function-less}
## less, \< operator {#function-less}
## greater, > operator {#function-greater}
## greater, \> operator {#function-greater}
## lessOrEquals, <= operator {#function-lessorequals}
## lessOrEquals, \<= operator {#function-lessorequals}
## greaterOrEquals, >= operator {#function-greaterorequals}
## greaterOrEquals, \>= operator {#function-greaterorequals}
[Original article](https://clickhouse.tech/docs/en/query_language/functions/comparison_functions/) <!--hide-->

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,7 +22,7 @@ SELECT [DISTINCT] expr_list
[WHERE expr]
[GROUP BY expr_list] [WITH TOTALS]
[HAVING expr]
[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr]
[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr]
[LIMIT [offset_value, ]n BY columns]
[LIMIT [n, ]m] [WITH TIES]
[UNION ALL ...]

View File

@ -12,21 +12,23 @@ toc_title: LIMIT
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}
## 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).
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
``` sql
SELECT * FROM (
SELECT number%50 AS n FROM numbers(100)
) ORDER BY n LIMIT 0,5
```
returns
```text
returns
``` text
┌─n─┐
│ 0 │
│ 0 │
@ -37,14 +39,16 @@ returns
```
but after apply `WITH TIES` modifier
```sql
``` 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
``` text
┌─n─┐
│ 0 │
│ 0 │
@ -54,4 +58,5 @@ it returns another rows set
│ 2 │
└───┘
```
cause row number 6 have same value "2" for field `n` as row number 5
cause row number 6 have same value “2” for field `n` as row number 5

View File

@ -70,28 +70,28 @@ Running a query may use more memory than `max_bytes_before_external_sort`. For t
External sorting works much less effectively than sorting in RAM.
## ORDER BY expr WITH FILL modifier {#orderby-with-fill}
## 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).
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.
`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
``` 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` 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
``` sql
SELECT n, source FROM (
SELECT toFloat32(number % 10) AS n, 'original' AS source
FROM numbers(10) WHERE number % 3 = 1
@ -99,7 +99,8 @@ SELECT n, source FROM (
```
returns
```text
``` text
┌─n─┬─source───┐
│ 1 │ original │
│ 4 │ original │
@ -108,7 +109,8 @@ returns
```
but after apply `WITH FILL` modifier
```sql
``` sql
SELECT n, source FROM (
SELECT toFloat32(number % 10) AS n, 'original' AS source
FROM numbers(10) WHERE number % 3 = 1
@ -116,7 +118,8 @@ SELECT n, source FROM (
```
returns
```text
``` text
┌───n─┬─source───┐
│ 0 │ │
│ 0.5 │ │
@ -137,61 +140,65 @@ returns
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,
``` 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,
ORDER BY
d2 WITH FILL,
d1 WITH FILL STEP 5;
```
returns
```text
``` 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 │ │
│ 0000-00-00 │ 1970-01-07 │ │
│ 1970-03-12 │ 1970-01-08 │ original │
└────────────┴────────────┴──────────┘
└────────────┴────────────┴──────────┘
```
Field `d1` doesn't fill and use default value cause we don't have repeated values for `d2` value, and sequence for `d1` can't be properly calculated.
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,
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
ORDER BY
d1 WITH FILL STEP 5,
d2 WITH FILL;
d2 WITH FILL;
```
returns
```text
``` 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-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-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 │
└────────────┴────────────┴──────────┘
│ 1970-03-12 │ 1970-01-08 │ original │
└────────────┴────────────┴──────────┘
```

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

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

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

@ -372,6 +372,7 @@ struct Settings : public SettingsCollection<Settings>
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) \
\

View File

@ -1096,8 +1096,10 @@ public:
return;
}
auto * left_generic = block.getByPosition(arguments[0]).type.get();
auto * right_generic = block.getByPosition(arguments[1]).type.get();
const auto & left_argument = block.getByPosition(arguments[0]);
const auto & right_argument = block.getByPosition(arguments[1]);
auto * left_generic = left_argument.type.get();
auto * right_generic = right_argument.type.get();
bool valid = castBothTypes(left_generic, right_generic, [&](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
@ -1112,8 +1114,17 @@ public:
else
return executeNumeric(block, arguments, result, left, right);
});
if (!valid)
throw Exception(getName() + "'s arguments do not match the expected data types", ErrorCodes::LOGICAL_ERROR);
{
// This is a logical error, because the types should have been checked
// by getReturnTypeImpl().
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Arguments of '{}' have incorrect data types: '{}' of type '{}',"
" '{}' of type '{}'", getName(),
left_argument.name, left_argument.type->getName(),
right_argument.name, right_argument.type->getName());
}
}
#if USE_EMBEDDED_COMPILER

View File

@ -22,6 +22,12 @@
namespace
{
const char * S3_LOGGER_TAG_NAMES[][2] = {
{"AWSClient", "AWSClient"},
{"AWSAuthV4Signer", "AWSClient (AWSAuthV4Signer)"},
};
const std::pair<DB::LogsLevel, Poco::Message::Priority> & convertLogLevel(Aws::Utils::Logging::LogLevel log_level)
{
static const std::unordered_map<Aws::Utils::Logging::LogLevel, std::pair<DB::LogsLevel, Poco::Message::Priority>> mapping =
@ -40,26 +46,46 @@ const std::pair<DB::LogsLevel, Poco::Message::Priority> & convertLogLevel(Aws::U
class AWSLogger final : public Aws::Utils::Logging::LogSystemInterface
{
public:
AWSLogger()
{
for (auto [tag, name] : S3_LOGGER_TAG_NAMES)
tag_loggers[tag] = &Poco::Logger::get(name);
default_logger = tag_loggers[S3_LOGGER_TAG_NAMES[0][0]];
}
~AWSLogger() final = default;
Aws::Utils::Logging::LogLevel GetLogLevel() const final { return Aws::Utils::Logging::LogLevel::Trace; }
void Log(Aws::Utils::Logging::LogLevel log_level, const char * tag, const char * format_str, ...) final // NOLINT
{
const auto & [level, prio] = convertLogLevel(log_level);
LOG_IMPL(log, level, prio, "{}: {}", tag, format_str);
callLogImpl(log_level, tag, format_str); /// FIXME. Variadic arguments?
}
void LogStream(Aws::Utils::Logging::LogLevel log_level, const char * tag, const Aws::OStringStream & message_stream) final
{
callLogImpl(log_level, tag, message_stream.str().c_str());
}
void callLogImpl(Aws::Utils::Logging::LogLevel log_level, const char * tag, const char * message)
{
const auto & [level, prio] = convertLogLevel(log_level);
LOG_IMPL(log, level, prio, "{}: {}", tag, message_stream.str());
if (tag_loggers.count(tag) > 0)
{
LOG_IMPL(tag_loggers[tag], level, prio, "{}", message);
}
else
{
LOG_IMPL(default_logger, level, prio, "{}: {}", tag, message);
}
}
void Flush() final {}
private:
Poco::Logger * log = &Poco::Logger::get("AWSClient");
Poco::Logger * default_logger;
std::unordered_map<String, Poco::Logger *> tag_loggers;
};
class S3AuthSigner : public Aws::Client::AWSAuthV4Signer
@ -102,8 +128,10 @@ public:
private:
const DB::HeaderCollection headers;
};
}
namespace DB
{
namespace ErrorCodes

View File

@ -23,7 +23,6 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int SUPPORT_IS_DISABLED;
extern const int INCORRECT_QUERY;
}
@ -62,10 +61,6 @@ BlockIO InterpreterAlterQuery::execute()
alter_commands.emplace_back(std::move(*alter_command));
else if (auto partition_command = PartitionCommand::parse(command_ast))
{
if (partition_command->type == PartitionCommand::DROP_DETACHED_PARTITION
&& !context.getSettingsRef().allow_drop_detached)
throw DB::Exception("Cannot execute query: DROP DETACHED PART is disabled "
"(see allow_drop_detached setting)", ErrorCodes::SUPPORT_IS_DISABLED);
partition_commands.emplace_back(std::move(*partition_command));
}
else if (auto mut_command = MutationCommand::parse(command_ast))
@ -90,6 +85,7 @@ BlockIO InterpreterAlterQuery::execute()
if (!partition_commands.empty())
{
table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, context.getSettingsRef());
table->alterPartition(query_ptr, metadata_snapshot, partition_commands, context);
}

View File

@ -0,0 +1,142 @@
#pragma once
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <DataTypes/DataTypeFactory.h>
#include <Functions/FunctionFactory.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/IAST.h>
#include <Common/typeid_cast.h>
namespace DB
{
using Monotonicity = IFunctionBase::Monotonicity;
/// Checks from bottom to top if function composition is monotonous
class MonotonicityCheckMatcher
{
public:
struct Data
{
const TablesWithColumns & tables;
const Context & context;
const std::unordered_set<String> & group_by_function_hashes;
Monotonicity monotonicity{true, true, true};
ASTIdentifier * identifier = nullptr;
DataTypePtr arg_data_type = {};
void reject() { monotonicity.is_monotonic = false; }
bool isRejected() const { return !monotonicity.is_monotonic; }
bool canOptimize(const ASTFunction & ast_function) const
{
/// if GROUP BY contains the same function ORDER BY shouldn't be optimized
auto hash = ast_function.getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
if (group_by_function_hashes.count(key))
return false;
/// if ORDER BY contains aggregate function it shouldn't be optimized
if (AggregateFunctionFactory::instance().isAggregateFunctionName(ast_function.name))
return false;
return true;
}
bool extractIdentifierAndType(const ASTFunction & ast_function)
{
if (identifier)
return true;
identifier = ast_function.arguments->children[0]->as<ASTIdentifier>();
if (!identifier)
return false;
auto pos = IdentifierSemantic::getMembership(*identifier);
if (!pos)
pos = IdentifierSemantic::chooseTableColumnMatch(*identifier, tables, true);
if (!pos)
return false;
if (auto data_type_and_name = tables[*pos].columns.tryGetByName(identifier->shortName()))
{
arg_data_type = data_type_and_name->type;
return true;
}
return false;
}
};
static void visit(const ASTPtr & ast, Data & data)
{
if (const auto * ast_function = ast->as<ASTFunction>())
visit(*ast_function, data);
}
static void visit(const ASTFunction & ast_function, Data & data)
{
if (data.isRejected())
return;
/// TODO: monotonicity for fucntions of several arguments
auto arguments = ast_function.arguments;
if (arguments->children.size() != 1)
{
data.reject();
return;
}
if (!data.canOptimize(ast_function))
{
data.reject();
return;
}
const auto & function = FunctionFactory::instance().tryGet(ast_function.name, data.context);
if (!function)
{
data.reject();
return;
}
/// First time extract the most enclosed identifier and its data type
if (!data.arg_data_type && !data.extractIdentifierAndType(ast_function))
{
data.reject();
return;
}
ColumnsWithTypeAndName args;
args.emplace_back(data.arg_data_type, "tmp");
auto function_base = function->build(args);
if (function_base && function_base->hasInformationAboutMonotonicity())
{
bool is_positive = data.monotonicity.is_positive;
data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, Field(), Field());
if (!is_positive)
data.monotonicity.is_positive = !data.monotonicity.is_positive;
data.arg_data_type = function_base->getReturnType();
}
else
data.reject();
}
static bool needChildVisit(const ASTPtr &, const ASTPtr &)
{
return true;
}
};
using MonotonicityCheckVisitor = ConstInDepthNodeVisitor<MonotonicityCheckMatcher, false>;
}

View File

@ -30,6 +30,7 @@
#include <Interpreters/AnyInputOptimize.h>
#include <Interpreters/RemoveInjectiveFunctionsVisitor.h>
#include <Interpreters/RedundantFunctionsInOrderByVisitor.h>
#include <Interpreters/MonotonicityCheckVisitor.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
@ -529,6 +530,46 @@ void optimizeDuplicateOrderByAndDistinct(ASTPtr & query, const Context & context
DuplicateDistinctVisitor(distinct_data).visit(query);
}
/// Replace monotonous functions in ORDER BY if they don't participate in GROUP BY expression,
/// has a single argument and not an aggregate functions.
void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, const Context & context,
const TablesWithColumns & tables_with_columns)
{
auto order_by = select_query->orderBy();
if (!order_by)
return;
std::unordered_set<String> group_by_hashes;
if (auto group_by = select_query->groupBy())
{
for (auto & elem : group_by->children)
{
auto hash = elem->getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
group_by_hashes.insert(key);
}
}
for (auto & child : order_by->children)
{
auto * order_by_element = child->as<ASTOrderByElement>();
auto & ast_func = order_by_element->children[0];
if (!ast_func->as<ASTFunction>())
continue;
MonotonicityCheckVisitor::Data data{tables_with_columns, context, group_by_hashes};
MonotonicityCheckVisitor(data).visit(ast_func);
if (!data.isRejected())
{
ast_func = data.identifier->clone();
ast_func->setAlias("");
if (!data.monotonicity.is_positive)
order_by_element->direction *= -1;
}
}
}
/// If ORDER BY has argument x followed by f(x) transfroms it to ORDER BY x.
/// Optimize ORDER BY x, y, f(x), g(x, y), f(h(x)), t(f(x), g(x)) into ORDER BY x, y
/// in case if f(), g(), h(), t() are deterministic (in scope of query).
@ -1066,6 +1107,10 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect(
if (settings.optimize_redundant_functions_in_order_by)
optimizeRedundantFunctionsInOrderBy(select_query, context);
/// Replace monotonous functions with its argument
if (settings.optimize_monotonous_functions_in_order_by)
optimizeMonotonousFunctionsInOrderBy(select_query, context, tables_with_columns);
/// Remove duplicated elements from LIMIT BY clause.
optimizeLimitBy(select_query);

View File

@ -302,7 +302,7 @@ bool TableJoin::needStreamWithNonJoinedRows() const
return isRightOrFull(kind());
}
bool TableJoin::allowDictJoin(const String & dict_key, const Block & sample_block, Names & names, NamesAndTypesList & result_columns) const
bool TableJoin::allowDictJoin(const String & dict_key, const Block & sample_block, Names & src_names, NamesAndTypesList & dst_columns) const
{
/// Support ALL INNER, [ANY | ALL | SEMI | ANTI] LEFT
if (!isLeft(kind()) && !(isInner(kind()) && strictness() == ASTTableJoin::Strictness::All))
@ -312,18 +312,26 @@ bool TableJoin::allowDictJoin(const String & dict_key, const Block & sample_bloc
if (right_keys.size() != 1)
return false;
/// TODO: support 'JOIN ... ON expr(dict_key) = table_key'
auto it_key = original_names.find(right_keys[0]);
if (it_key == original_names.end())
return false;
if (dict_key != it_key->second)
return false; /// JOIN key != Dictionary key
for (const auto & col : sample_block)
{
String original = original_names.find(col.name)->second;
if (col.name == right_keys[0])
{
if (original != dict_key)
return false; /// JOIN key != Dictionary key
continue; /// do not extract key column
}
names.push_back(original);
result_columns.push_back({col.name, col.type});
auto it = original_names.find(col.name);
if (it != original_names.end())
{
String original = it->second;
src_names.push_back(original);
dst_columns.push_back({col.name, col.type});
}
}
return true;

View File

@ -12,7 +12,7 @@ ASTPtr ASTNameTypePair::clone() const
if (type)
{
res->type = type;
res->type = type->clone();
res->children.push_back(res->type);
}

View File

@ -1003,6 +1003,9 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
}
}
if (all_columns.empty())
throw Exception{"Cannot DROP or CLEAR all columns", ErrorCodes::BAD_ARGUMENTS};
validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context);
}

View File

@ -121,6 +121,11 @@ public:
return columns.size();
}
bool empty() const
{
return columns.empty();
}
/// Keep the sequence of columns and allow to lookup by name.
using Container = boost::multi_index_container<
ColumnDescription,

View File

@ -102,6 +102,10 @@ void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settin
}
}
void IStorage::checkAlterPartitionIsPossible(const PartitionCommands & /*commands*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & /*settings*/) const
{
throw Exception("Table engine " + getName() + " doesn't support partitioning", ErrorCodes::NOT_IMPLEMENTED);
}
StorageID IStorage::getStorageID() const
{

View File

@ -360,6 +360,9 @@ public:
throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/// Checks that partition commands can be applied to storage.
virtual void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const;
/** Perform any background work. For example, combining parts in a MergeTree type table.
* Returns whether any work has been done.
*/

View File

@ -4,6 +4,7 @@
#include <cppkafka/cppkafka.h>
#include <boost/algorithm/string/join.hpp>
#include <algorithm>
namespace DB
{
@ -388,24 +389,43 @@ bool ReadBufferFromKafkaConsumer::poll()
}
}
while (auto err = current->get_error())
filterMessageErrors();
if (current == messages.end())
{
++current;
// TODO: should throw exception instead
LOG_ERROR(log, "Consumer error: {}", err);
if (current == messages.end())
{
LOG_ERROR(log, "No actual messages polled, errors only.");
stalled_status = ERRORS_RETURNED;
return false;
}
LOG_ERROR(log, "Only errors left");
stalled_status = ERRORS_RETURNED;
return false;
}
stalled_status = NOT_STALLED;
allowed = true;
return true;
}
size_t ReadBufferFromKafkaConsumer::filterMessageErrors()
{
assert(current == messages.begin());
auto new_end = std::remove_if(messages.begin(), messages.end(), [this](auto & message)
{
if (auto error = message.get_error())
{
LOG_ERROR(log, "Consumer error: {}", error);
return true;
}
return false;
});
size_t skipped = std::distance(messages.end(), new_end);
if (skipped)
{
LOG_ERROR(log, "There were {} messages with an error", skipped);
messages.erase(new_end, messages.end());
}
return skipped;
}
void ReadBufferFromKafkaConsumer::resetIfStopped()
{
// we can react on stop only during fetching data

View File

@ -102,6 +102,8 @@ private:
void drain();
void cleanUnprocessed();
void resetIfStopped();
/// Return number of messages with an error.
size_t filterMessageErrors();
bool nextImpl() override;
};

View File

@ -110,6 +110,7 @@ namespace ErrorCodes
extern const int UNKNOWN_DISK;
extern const int NOT_ENOUGH_SPACE;
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
extern const int SUPPORT_IS_DISABLED;
}
@ -1421,12 +1422,20 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
columns_in_keys.insert(columns_alter_type_metadata_only.begin(), columns_alter_type_metadata_only.end());
columns_in_keys.insert(columns_alter_type_check_safe_for_partition.begin(), columns_alter_type_check_safe_for_partition.end());
NameSet dropped_columns;
std::map<String, const IDataType *> old_types;
for (const auto & column : old_metadata.getColumns().getAllPhysical())
old_types.emplace(column.name, column.type.get());
for (const AlterCommand & command : commands)
{
/// Just validate partition expression
if (command.partition)
{
getPartitionIDFromQuery(command.partition, global_context);
}
if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned)
{
throw Exception(
@ -1456,6 +1465,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
"Trying to ALTER DROP key " + backQuoteIfNeed(command.column_name) + " column which is a part of key expression",
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
}
dropped_columns.emplace(command.column_name);
}
else if (command.isModifyingData(getInMemoryMetadata()))
{
@ -1530,6 +1540,27 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
checkStoragePolicy(global_context.getStoragePolicy(changed_setting.value.safeGet<String>()));
}
}
for (const auto & part : getDataPartsVector())
{
bool at_least_one_column_rest = false;
for (const auto & column : part->getColumns())
{
if (!dropped_columns.count(column.name))
{
at_least_one_column_rest = true;
break;
}
}
if (!at_least_one_column_rest)
{
std::string postfix;
if (dropped_columns.size() > 1)
postfix = "s";
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Cannot drop or clear column{} '{}', because all columns in part '{}' will be removed from disk. Empty parts are not allowed", postfix, boost::algorithm::join(dropped_columns, ", "), part->name);
}
}
}
MergeTreeDataPartType MergeTreeData::choosePartType(size_t bytes_uncompressed, size_t rows_count) const
@ -2525,6 +2556,45 @@ void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const StorageM
context);
}
void MergeTreeData::checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings) const
{
for (const auto & command : commands)
{
if (command.type == PartitionCommand::DROP_DETACHED_PARTITION
&& !settings.allow_drop_detached)
throw DB::Exception("Cannot execute query: DROP DETACHED PART is disabled "
"(see allow_drop_detached setting)", ErrorCodes::SUPPORT_IS_DISABLED);
if (command.partition && command.type != PartitionCommand::DROP_DETACHED_PARTITION)
{
if (command.part)
{
auto part_name = command.partition->as<ASTLiteral &>().value.safeGet<String>();
/// We able to parse it
MergeTreePartInfo::fromPartName(part_name, format_version);
}
else
{
/// We able to parse it
getPartitionIDFromQuery(command.partition, global_context);
}
}
}
}
void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition)
{
const String partition_id = getPartitionIDFromQuery(partition, global_context);
auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
UInt64 partition_size = 0;
for (const auto & part : parts_to_remove)
partition_size += part->getBytesOnDisk();
auto table_id = getStorageID();
global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size);
}
void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & name, bool moving_part, const Context & context)
{
@ -2626,7 +2696,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String
}
String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context & context)
String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context & context) const
{
const auto & partition_ast = ast->as<ASTPartition &>();
@ -3045,7 +3115,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector() const
}
MergeTreeData::DataPartPtr MergeTreeData::getAnyPartInPartition(
const String & partition_id, DataPartsLock & /*data_parts_lock*/)
const String & partition_id, DataPartsLock & /*data_parts_lock*/) const
{
auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndPartitionID{DataPartState::Committed, partition_id});

View File

@ -24,6 +24,7 @@
#include <Disks/StoragePolicy.h>
#include <Interpreters/Aggregator.h>
#include <Storages/extractKeyExpressionList.h>
#include <Storages/PartitionCommands.h>
#include <boost/multi_index_container.hpp>
#include <boost/multi_index/ordered_index.hpp>
@ -505,6 +506,9 @@ public:
/// If something is wrong, throws an exception.
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
/// Checks that partition name in all commands is valid
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override;
/// Change MergeTreeSettings
void changeSettings(
const ASTPtr & new_settings,
@ -547,6 +551,8 @@ public:
/// Moves partition to specified Volume
void movePartitionToVolume(const ASTPtr & partition, const String & name, bool moving_part, const Context & context);
void checkPartitionCanBeDropped(const ASTPtr & partition) override;
size_t getColumnCompressedSize(const std::string & name) const
{
auto lock = lockParts();
@ -561,7 +567,7 @@ public:
}
/// For ATTACH/DETACH/DROP PARTITION.
String getPartitionIDFromQuery(const ASTPtr & ast, const Context & context);
String getPartitionIDFromQuery(const ASTPtr & ast, const Context & context) const;
/// Extracts MergeTreeData of other *MergeTree* storage
/// and checks that their structure suitable for ALTER TABLE ATTACH PARTITION FROM
@ -815,7 +821,7 @@ protected:
void removePartContributionToColumnSizes(const DataPartPtr & part);
/// If there is no part in the partition with ID `partition_id`, returns empty ptr. Should be called under the lock.
DataPartPtr getAnyPartInPartition(const String & partition_id, DataPartsLock & data_parts_lock);
DataPartPtr getAnyPartInPartition(const String & partition_id, DataPartsLock & data_parts_lock) const;
/// Return parts in the Committed set that are covered by the new_part_info or the part that covers it.
/// Will check that the new part doesn't already exist and that it doesn't intersect existing part.

View File

@ -1068,20 +1068,6 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
}
}
/// TODO: it makes sense to check DROP_RANGE also
if (entry.type == LogEntry::CLEAR_COLUMN || entry.type == LogEntry::REPLACE_RANGE)
{
String conflicts_description;
String range_name = (entry.type == LogEntry::REPLACE_RANGE) ? entry.replace_range_entry->drop_range_part_name : entry.new_part_name;
auto range = MergeTreePartInfo::fromPartName(range_name, format_version);
if (0 != getConflictsCountForRange(range, entry, &conflicts_description, state_lock))
{
LOG_DEBUG(log, conflicts_description);
return false;
}
}
/// Alters must be executed one by one. First metadata change, and after that data alter (MUTATE_PART entries with).
/// corresponding alter_version.
if (entry.type == LogEntry::ALTER_METADATA)

View File

@ -50,6 +50,7 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
extern const int INFINITE_LOOP;
@ -842,9 +843,22 @@ void registerStorageBuffer(StorageFactory & factory)
" destination_database, destination_table, num_buckets, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
// Table and database name arguments accept expressions, evaluate them.
engine_args[0] = evaluateConstantExpressionForDatabaseName(engine_args[0], args.local_context);
engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.local_context);
// After we evaluated all expressions, check that all arguments are
// literals.
for (size_t i = 0; i < 9; i++)
{
if (!typeid_cast<ASTLiteral *>(engine_args[i].get()))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Storage Buffer expects a literal as an argument #{}, got '{}'"
" instead", i, engine_args[i]->formatForErrorMessage());
}
}
String destination_database = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
String destination_table = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();

View File

@ -257,6 +257,13 @@ void StorageMaterializedView::alterPartition(
getTargetTable()->alterPartition(query, metadata_snapshot, commands, context);
}
void StorageMaterializedView::checkAlterPartitionIsPossible(
const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const
{
checkStatementCanBeForwarded();
getTargetTable()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings);
}
void StorageMaterializedView::mutate(const MutationCommands & commands, const Context & context)
{
checkStatementCanBeForwarded();

View File

@ -53,6 +53,8 @@ public:
void alterPartition(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override;
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override;
void mutate(const MutationCommands & commands, const Context & context) override;
void renameInMemory(const StorageID & new_table_id) override;

View File

@ -209,22 +209,6 @@ void StorageMergeTree::checkTableCanBeDropped() const
global_context.checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes());
}
void StorageMergeTree::checkPartitionCanBeDropped(const ASTPtr & partition)
{
auto table_id = getStorageID();
const String partition_id = getPartitionIDFromQuery(partition, global_context);
auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
UInt64 partition_size = 0;
for (const auto & part : parts_to_remove)
{
partition_size += part->getBytesOnDisk();
}
global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size);
}
void StorageMergeTree::drop()
{
shutdown();

View File

@ -81,8 +81,6 @@ public:
void checkTableCanBeDropped() const override;
void checkPartitionCanBeDropped(const ASTPtr & partition) override;
ActionLock getActionLock(StorageActionBlockType action_type) override;
CheckResults checkData(const ASTPtr & query, const Context & context) override;

View File

@ -4054,22 +4054,6 @@ void StorageReplicatedMergeTree::checkTableCanBeDropped() const
global_context.checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes());
}
void StorageReplicatedMergeTree::checkPartitionCanBeDropped(const ASTPtr & partition)
{
const String partition_id = getPartitionIDFromQuery(partition, global_context);
auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
UInt64 partition_size = 0;
for (const auto & part : parts_to_remove)
partition_size += part->getBytesOnDisk();
auto table_id = getStorageID();
global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size);
}
void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
{
MergeTreeData::rename(new_path_to_table_data, new_table_id);

View File

@ -134,8 +134,6 @@ public:
void checkTableCanBeDropped() const override;
void checkPartitionCanBeDropped(const ASTPtr & partition) override;
ActionLock getActionLock(StorageActionBlockType action_type) override;
/// Wait when replication queue size becomes less or equal than queue_size

View File

@ -43,7 +43,7 @@ ColumnsDescription getStructureOfRemoteTable(
/// Expect at least some columns.
/// This is a hack to handle the empty block case returned by Connection when skip_unavailable_shards is set.
if (res.size() == 0)
if (res.empty())
continue;
return res;

View File

@ -0,0 +1,9 @@
<test>
<preconditions>
<table_exists>hits_10m_single</table_exists>
</preconditions>
<query>SELECT * FROM (SELECT CounterID, EventDate FROM hits_10m_single) ORDER BY toFloat32(toFloat64(toFloat32(toFloat64(CounterID)))) FORMAT Null</query>
<query>SELECT * FROM (SELECT CounterID, EventDate FROM hits_10m_single) ORDER BY toFloat32(toFloat64(toFloat32(toFloat64(CounterID)))) DESC, toFloat32(toFloat64(toFloat32(toFloat64(EventDate)))) ASC FORMAT Null</query>
</test>

View File

@ -0,0 +1,168 @@
0
1
2
0
1
2
0
1
2
2
1
0
0
1
2
0
1
2
0
1
2
2
1
0
2
1
0
2
1
0
0
1
2
2
1
0
2
1
0
SELECT number
FROM numbers(3)
ORDER BY number ASC
SELECT number
FROM numbers(3)
ORDER BY abs(toFloat32(number)) ASC
SELECT number
FROM numbers(3)
ORDER BY toFloat32(abs(number)) ASC
SELECT number
FROM numbers(3)
ORDER BY number DESC
SELECT number
FROM numbers(3)
ORDER BY exp(number) ASC
SELECT roundToExp2(number) AS x
FROM numbers(3)
ORDER BY
number ASC,
number ASC
SELECT number AS x
FROM numbers(3)
ORDER BY
number ASC,
number ASC
SELECT number
FROM numbers(3)
ORDER BY number DESC
SELECT number
FROM numbers(3)
ORDER BY abs(toFloat32(number)) DESC
SELECT number
FROM numbers(3)
ORDER BY toFloat32(abs(number)) DESC
SELECT number
FROM numbers(3)
ORDER BY number ASC
SELECT number
FROM numbers(3)
ORDER BY exp(number) DESC
SELECT roundToExp2(number) AS x
FROM numbers(3)
ORDER BY
number DESC,
number DESC
0
1
2
0
1
2
0
1
2
2
1
0
0
1
2
0
1
2
0
1
2
2
1
0
2
1
0
2
1
0
0
1
2
2
1
0
2
1
0
SELECT number
FROM numbers(3)
ORDER BY toFloat32(toFloat64(number)) ASC
SELECT number
FROM numbers(3)
ORDER BY abs(toFloat32(number)) ASC
SELECT number
FROM numbers(3)
ORDER BY toFloat32(abs(number)) ASC
SELECT number
FROM numbers(3)
ORDER BY -number ASC
SELECT number
FROM numbers(3)
ORDER BY exp(number) ASC
SELECT roundToExp2(number) AS x
FROM numbers(3)
ORDER BY
x ASC,
toFloat32(x) ASC
SELECT number AS x
FROM numbers(3)
ORDER BY
toFloat32(x) AS k ASC,
toFloat64(k) ASC
SELECT number
FROM numbers(3)
ORDER BY toFloat32(toFloat64(number)) DESC
SELECT number
FROM numbers(3)
ORDER BY abs(toFloat32(number)) DESC
SELECT number
FROM numbers(3)
ORDER BY toFloat32(abs(number)) DESC
SELECT number
FROM numbers(3)
ORDER BY -number DESC
SELECT number
FROM numbers(3)
ORDER BY exp(number) DESC
SELECT roundToExp2(number) AS x
FROM numbers(3)
ORDER BY
x DESC,
toFloat32(x) DESC

View File

@ -0,0 +1,59 @@
SET enable_debug_queries = 1;
SET optimize_monotonous_functions_in_order_by = 1;
SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number));
SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number));
SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number));
SELECT number FROM numbers(3) ORDER BY -number;
SELECT number FROM numbers(3) ORDER BY exp(number);
SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x, toFloat32(x);
SELECT number AS x FROM numbers(3) ORDER BY toFloat32(x) as k, toFloat64(k);
SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number)) DESC;
SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number)) DESC;
SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number)) DESC;
SELECT number FROM numbers(3) ORDER BY -number DESC;
SELECT number FROM numbers(3) ORDER BY exp(number) DESC;
SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x DESC, toFloat32(x) DESC;
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number));
analyze SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number));
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number));
analyze SELECT number FROM numbers(3) ORDER BY -number;
analyze SELECT number FROM numbers(3) ORDER BY exp(number);
analyze SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x, toFloat32(x);
analyze SELECT number AS x FROM numbers(3) ORDER BY toFloat32(x) as k, toFloat64(k);
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY -number DESC;
analyze SELECT number FROM numbers(3) ORDER BY exp(number) DESC;
analyze SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x DESC, toFloat32(x) DESC;
SET optimize_monotonous_functions_in_order_by = 0;
SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number));
SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number));
SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number));
SELECT number FROM numbers(3) ORDER BY -number;
SELECT number FROM numbers(3) ORDER BY exp(number);
SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x, toFloat32(x);
SELECT number AS x FROM numbers(3) ORDER BY toFloat32(x) as k, toFloat64(k);
SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number)) DESC;
SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number)) DESC;
SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number)) DESC;
SELECT number FROM numbers(3) ORDER BY -number DESC;
SELECT number FROM numbers(3) ORDER BY exp(number) DESC;
SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x DESC, toFloat32(x) DESC;
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number));
analyze SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number));
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number));
analyze SELECT number FROM numbers(3) ORDER BY -number;
analyze SELECT number FROM numbers(3) ORDER BY exp(number);
analyze SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x, toFloat32(x);
analyze SELECT number AS x FROM numbers(3) ORDER BY toFloat32(x) as k, toFloat64(k);
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY -number DESC;
analyze SELECT number FROM numbers(3) ORDER BY exp(number) DESC;
analyze SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x DESC, toFloat32(x) DESC;
-- TODO: exp() should be monotonous function

View File

@ -0,0 +1,32 @@
1 4 3
1 3 3
2 5 4
2 2 4
1 3 3
1 4 3
2 2 4
2 5 4
2
1
2
1 3 3
1 4 3
2 5 4
2 2 4
2
1 4 3
1 3 3
2 5 4
2 2 4
1 3 3
1 4 3
2 2 4
2 5 4
2
1
2
1 3 3
1 4 3
2 5 4
2 2 4
2

View File

@ -0,0 +1,21 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test (x Int8, y Int8, z Int8) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO test VALUES (1, 3, 3), (1, 4, 3), (2, 5, 4), (2, 2, 4);
SET optimize_monotonous_functions_in_order_by = 1;
SELECT * FROM test ORDER BY toFloat32(x), -y, -z DESC;
SELECT * FROM test ORDER BY toFloat32(x), -(-y), -z DESC;
SELECT max(x) as k FROM test ORDER BY k;
SELECT roundToExp2(x) as k FROM test GROUP BY k ORDER BY k;
SELECT roundToExp2(x) as k, y, z FROM test WHERE k >= 1 ORDER BY k;
SELECT max(x) as k FROM test HAVING k > 0 ORDER BY k;
SET optimize_monotonous_functions_in_order_by = 0;
SELECT * FROM test ORDER BY toFloat32(x), -y, -z DESC;
SELECT * FROM test ORDER BY toFloat32(x), -(-y), -z DESC;
SELECT max(x) as k FROM test ORDER BY k;
SELECT roundToExp2(x) as k From test GROUP BY k ORDER BY k;
SELECT roundToExp2(x) as k, y, z FROM test WHERE k >= 1 ORDER BY k;
SELECT max(x) as k FROM test HAVING k > 0 ORDER BY k;
DROP TABLE test;

View File

@ -0,0 +1,6 @@
0 1 Hello
0 1 Hello
0 2 Hello
0 2 Hello
0 3 Hello
0 3 Hello

View File

@ -0,0 +1,34 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test (x UInt8) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO test (x) VALUES (1), (2), (3);
ALTER TABLE test CLEAR COLUMN x; --{serverError 36}
DROP TABLE test;
DROP TABLE IF EXISTS test;
CREATE TABLE test (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO test (x, y) VALUES (1, 1), (2, 2), (3, 3);
ALTER TABLE test CLEAR COLUMN x;
ALTER TABLE test CLEAR COLUMN x IN PARTITION ''; --{serverError 248}
ALTER TABLE test CLEAR COLUMN x IN PARTITION 'asdasd'; --{serverError 248}
ALTER TABLE test CLEAR COLUMN x IN PARTITION '123'; --{serverError 248}
ALTER TABLE test CLEAR COLUMN y; --{serverError 36}
ALTER TABLE test ADD COLUMN z String DEFAULT 'Hello';
-- y is only real column in table
ALTER TABLE test CLEAR COLUMN y; --{serverError 36}
ALTER TABLE test CLEAR COLUMN x;
ALTER TABLE test CLEAR COLUMN z;
INSERT INTO test (x, y, z) VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c');
ALTER TABLE test CLEAR COLUMN z;
ALTER TABLE test CLEAR COLUMN x;
SELECT * FROM test ORDER BY y;
DROP TABLE IF EXISTS test;

View File

@ -0,0 +1 @@
0

View File

@ -0,0 +1,27 @@
DROP DATABASE IF EXISTS db_01391;
CREATE DATABASE db_01391 Engine = Ordinary;
USE db_01391;
DROP TABLE IF EXISTS t;
DROP TABLE IF EXISTS d_src;
DROP DICTIONARY IF EXISTS d;
CREATE TABLE t (click_city_id UInt32, click_country_id UInt32) Engine = Memory;
CREATE TABLE d_src (id UInt64, country_id UInt8, name String) Engine = Memory;
INSERT INTO t VALUES (0, 0);
INSERT INTO d_src VALUES (0, 0, 'n');
CREATE DICTIONARY d (id UInt32, country_id UInt8, name String)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' DB 'db_01391' table 'd_src'))
LIFETIME(MIN 1 MAX 1)
LAYOUT(HASHED());
select click_country_id from t cc
left join d on toUInt32(d.id) = cc.click_city_id;
DROP DICTIONARY d;
DROP TABLE t;
DROP TABLE d_src;
DROP DATABASE IF EXISTS db_01391;

View File

@ -0,0 +1,4 @@
1 --
1 RU
1 --
1 RU

View File

@ -0,0 +1,15 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test
(
id UInt32,
code LowCardinality(FixedString(2)) DEFAULT '--'
) ENGINE = MergeTree() PARTITION BY id ORDER BY id;
INSERT INTO test FORMAT CSV 1,RU
INSERT INTO test FORMAT CSV 1,
SELECT * FROM test ORDER BY code;
OPTIMIZE TABLE test;
SELECT * FROM test ORDER BY code;
DROP TABLE test;

View File

@ -0,0 +1 @@
SELECT toTimeZone(now(), 'Europe/Moscow') > '1970-01-01 00:00:00';

View File

@ -131,3 +131,4 @@
01370_client_autocomplete_word_break_characters
01319_optimize_skip_unused_shards_nesting
01376_GROUP_BY_injective_elimination_dictGet
01391_join_on_dict_crash

View File

@ -1,5 +1,6 @@
details {
background: #444451;
color: #eee;
padding: 1rem;
margin-bottom: 1rem;
margin-top: 1rem;
@ -7,7 +8,7 @@ details {
summary {
font-weight: bold;
color: #fff;
color: #eee;
}
#sidebar {