Merge branch 'master' into fix_watch_race_testkeeper

This commit is contained in:
alesapin 2020-12-16 23:03:07 +03:00
commit 1f3702ff9e
22 changed files with 270 additions and 19 deletions

View File

@ -31,6 +31,7 @@ find . -name '*.so.*' -print -exec mv '{}' /output \;
if [ "performance" == "$COMBINED_OUTPUT" ]
then
cp -r ../tests/performance /output
cp -r ../tests/config/top_level_domains /
cp -r ../docker/test/performance-comparison/config /output ||:
rm /output/unit_tests_dbms ||:
rm /output/clickhouse-odbc-bridge ||:

View File

@ -57,7 +57,7 @@ The command line is based on replxx (similar to readline). In other
By default, the format used is PrettyCompact. You can change the format in the FORMAT clause of the query, or by specifying `\G` at the end of the query, using the `--format` or `--vertical` argument in the command line, or using the client configuration file.
To exit the client, press Ctrl+D (or Ctrl+C), or enter one of the following instead of a query: “exit”, “quit”, “logout”, “exit;”, “quit;”, “logout;”, “q”, “Q”, “:q”
To exit the client, press Ctrl+D, or enter one of the following instead of a query: “exit”, “quit”, “logout”, “exit;”, “quit;”, “logout;”, “q”, “Q”, “:q”
When processing a query, the client shows:

View File

@ -39,7 +39,7 @@ data_compressed_bytes: 499
last_exception:
```
**See also**
**See Also**
- [Distributed table engine](../../engines/table-engines/special/distributed.md)

View File

@ -0,0 +1,81 @@
# system.replication_queue {#system_tables-replication_queue}
Contains information about tasks from replication queues stored in ZooKeeper for tables in the `ReplicatedMergeTree` family.
Columns:
- `database` ([String](../../sql-reference/data-types/string.md)) — Name of the database.
- `table` ([String](../../sql-reference/data-types/string.md)) — Name of the table.
- `replica_name` ([String](../../sql-reference/data-types/string.md)) — Replica name in ZooKeeper. Different replicas of the same table have different names.
- `position` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Position of the task in the queue.
- `node_name` ([String](../../sql-reference/data-types/string.md)) — Node name in ZooKeeper.
- `type` ([String](../../sql-reference/data-types/string.md)) — Type of the task in the queue: `GET_PARTS`, `MERGE_PARTS`, `DETACH_PARTS`, `DROP_PARTS`, or `MUTATE_PARTS`.
- `create_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — Date and time when the task was submitted for execution.
- `required_quorum` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of replicas waiting for the task to complete with confirmation of completion. This column is only relevant for the `GET_PARTS` task.
- `source_replica` ([String](../../sql-reference/data-types/string.md)) — Name of the source replica.
- `new_part_name` ([String](../../sql-reference/data-types/string.md)) — Name of the new part.
- `parts_to_merge` ([Array](../../sql-reference/data-types/array.md) ([String](../../sql-reference/data-types/string.md))) — Names of parts to merge or update.
- `is_detach` ([UInt8](../../sql-reference/data-types/int-uint.md)) — The flag indicates whether the `DETACH_PARTS` task is in the queue.
- `is_currently_executing` ([UInt8](../../sql-reference/data-types/int-uint.md)) — The flag indicates whether a specific task is being performed right now.
- `num_tries` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of failed attempts to complete the task.
- `last_exception` ([String](../../sql-reference/data-types/string.md)) — Text message about the last error that occurred (if any).
- `last_attempt_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — Date and time when the task was last attempted.
- `num_postponed` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of postponed tasks.
- `postpone_reason` ([String](../../sql-reference/data-types/string.md)) — The reason why the task was postponed.
- `last_postpone_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — Date and time when the task was last postponed.
- `merge_type` ([String](../../sql-reference/data-types/string.md)) — Type of the current merge. Empty if it's a mutation.
**Example**
``` sql
SELECT * FROM system.replication_queue LIMIT 1 FORMAT Vertical;
```
``` text
Row 1:
──────
database: merge
table: visits_v2
replica_name: mtgiga001-1t.metrika.yandex.net
position: 15
node_name: queue-0009325559
type: MERGE_PARTS
create_time: 2020-12-07 14:04:21
required_quorum: 0
source_replica: mtgiga001-1t.metrika.yandex.net
new_part_name: 20201130_121373_121384_2
parts_to_merge: ['20201130_121373_121378_1','20201130_121379_121379_0','20201130_121380_121380_0','20201130_121381_121381_0','20201130_121382_121382_0','20201130_121383_121383_0','20201130_121384_121384_0']
is_detach: 0
is_currently_executing: 0
num_tries: 36
last_exception: Code: 226, e.displayText() = DB::Exception: Marks file '/opt/clickhouse/data/merge/visits_v2/tmp_fetch_20201130_121373_121384_2/CounterID.mrk' doesn't exist (version 20.8.7.15 (official build))
last_attempt_time: 2020-12-08 17:35:54
num_postponed: 0
postpone_reason:
last_postpone_time: 1970-01-01 03:00:00
```
**See Also**
- [Managing ReplicatedMergeTree Tables](../../sql-reference/statements/system.md/#query-language-system-replicated)
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/replication_queue) <!--hide-->

View File

@ -204,7 +204,7 @@ SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
## Managing ReplicatedMergeTree Tables {#query-language-system-replicated}
ClickHouse can manage background replication related processes in [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication/#table_engines-replication) tables.
ClickHouse can manage background replication related processes in [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md#table_engines-replication) tables.
### STOP FETCHES {#query_language-system-stop-fetches}

View File

@ -59,7 +59,7 @@ La línea de comandos se basa en replxx (similar a readline). En otr
De forma predeterminada, el formato utilizado es PrettyCompact. Puede cambiar el formato en la cláusula FORMAT de la consulta o especificando `\G` al final de la consulta, utilizando el `--format` o `--vertical` en la línea de comandos, o utilizando el archivo de configuración del cliente.
Para salir del cliente, presione Ctrl+D (o Ctrl+C) o introduzca una de las siguientes opciones en lugar de una consulta: “exit”, “quit”, “logout”, “exit;”, “quit;”, “logout;”, “q”, “Q”, “:q”
Para salir del cliente, presione Ctrl+D o introduzca una de las siguientes opciones en lugar de una consulta: “exit”, “quit”, “logout”, “exit;”, “quit;”, “logout;”, “q”, “Q”, “:q”
Al procesar una consulta, el cliente muestra:

View File

@ -59,7 +59,7 @@ $ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FOR
既定では、使用される形式はPrettyCompactです。 クエリのFORMAT句で書式を変更するか、次のように指定することができます `\G` クエリの最後に、 `--format` または `--vertical` コマンドラインでの引数、またはクライアント構成ファイルの使用。
クライアントを終了するには、Ctrl+D(またはCtrl+C)を押すか、クエリの代わりに次のいずれかを入力します: “exit”, “quit”, “logout”, “exit;”, “quit;”, “logout;”, “q”, “Q”, “:q”
クライアントを終了するには、Ctrl+Dを押すか、クエリの代わりに次のいずれかを入力します: “exit”, “quit”, “logout”, “exit;”, “quit;”, “logout;”, “q”, “Q”, “:q”
が処理クエリー、クライアントを示し:

View File

@ -63,7 +63,7 @@ $ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FOR
По умолчанию, в качестве формата, используется формат PrettyCompact (красивые таблички). Вы можете изменить формат с помощью секции FORMAT запроса, или с помощью указания `\G` на конце запроса, с помощью аргумента командной строки `--format` или `--vertical`, или с помощью конфигурационного файла клиента.
Чтобы выйти из клиента, нажмите Ctrl+D (или Ctrl+C), или наберите вместо запроса одно из: «exit», «quit», «logout», «учше», «йгше», «дщпщге», «exit;», «quit;», «logout;», «учшеж», «йгшеж», «дщпщгеж», «q», «й», «q», «Q», «:q», «й», «Й», «Жй»
Чтобы выйти из клиента, нажмите Ctrl+D, или наберите вместо запроса одно из: «exit», «quit», «logout», «учше», «йгше», «дщпщге», «exit;», «quit;», «logout;», «учшеж», «йгшеж», «дщпщгеж», «q», «й», «q», «Q», «:q», «й», «Й», «Жй»
При выполнении запроса, клиент показывает:

View File

@ -0,0 +1,81 @@
# system.replication_queue {#system_tables-replication_queue}
Содержит информацию о задачах из очередей репликации, хранящихся в ZooKeeper, для таблиц семейства `ReplicatedMergeTree`.
Столбцы:
- `database` ([String](../../sql-reference/data-types/string.md)) — имя базы данных.
- `table` ([String](../../sql-reference/data-types/string.md)) — имя таблицы.
- `replica_name` ([String](../../sql-reference/data-types/string.md)) — имя реплики в ZooKeeper. Разные реплики одной и той же таблицы имеют различные имена.
- `position` ([UInt32](../../sql-reference/data-types/int-uint.md)) — позиция задачи в очереди.
- `node_name` ([String](../../sql-reference/data-types/string.md)) — имя узла в ZooKeeper.
- `type` ([String](../../sql-reference/data-types/string.md)) — тип задачи в очереди: `GET_PARTS`, `MERGE_PARTS`, `DETACH_PARTS`, `DROP_PARTS` или `MUTATE_PARTS`.
- `create_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — дата и время отправки задачи на выполнение.
- `required_quorum` ([UInt32](../../sql-reference/data-types/int-uint.md)) — количество реплик, ожидающих завершения задачи, с подтверждением о завершении. Этот столбец актуален только для задачи `GET_PARTS`.
- `source_replica` ([String](../../sql-reference/data-types/string.md)) — имя исходной реплики.
- `new_part_name` ([String](../../sql-reference/data-types/string.md)) — имя нового куска.
- `parts_to_merge` ([Array](../../sql-reference/data-types/array.md) ([String](../../sql-reference/data-types/string.md))) — имена кусков, которые требуется смержить или обновить.
- `is_detach` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, указывающий на присутствие в очереди задачи `DETACH_PARTS`.
- `is_currently_executing` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, указывающий на выполнение конкретной задачи на данный момент.
- `num_tries` ([UInt32](../../sql-reference/data-types/int-uint.md)) — количество неудачных попыток выполнить задачу.
- `last_exception` ([String](../../sql-reference/data-types/string.md)) — текст сообщения о последней возникшей ошибке, если таковые имеются.
- `last_attempt_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — дата и время последней попытки выполнить задачу.
- `num_postponed` ([UInt32](../../sql-reference/data-types/int-uint.md)) — количество отложенных задач.
- `postpone_reason` ([String](../../sql-reference/data-types/string.md)) — причина, по которой была отложена задача.
- `last_postpone_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — дата и время, когда была отложена задача в последний раз.
- `merge_type` ([String](../../sql-reference/data-types/string.md)) — тип текущего слияния. Пусто, если это мутация.
**Пример**
``` sql
SELECT * FROM system.replication_queue LIMIT 1 FORMAT Vertical;
```
``` text
Row 1:
──────
database: merge
table: visits_v2
replica_name: mtgiga001-1t.metrika.yandex.net
position: 15
node_name: queue-0009325559
type: MERGE_PARTS
create_time: 2020-12-07 14:04:21
required_quorum: 0
source_replica: mtgiga001-1t.metrika.yandex.net
new_part_name: 20201130_121373_121384_2
parts_to_merge: ['20201130_121373_121378_1','20201130_121379_121379_0','20201130_121380_121380_0','20201130_121381_121381_0','20201130_121382_121382_0','20201130_121383_121383_0','20201130_121384_121384_0']
is_detach: 0
is_currently_executing: 0
num_tries: 36
last_exception: Code: 226, e.displayText() = DB::Exception: Marks file '/opt/clickhouse/data/merge/visits_v2/tmp_fetch_20201130_121373_121384_2/CounterID.mrk' doesn't exist (version 20.8.7.15 (official build))
last_attempt_time: 2020-12-08 17:35:54
num_postponed: 0
postpone_reason:
last_postpone_time: 1970-01-01 03:00:00
```
**Смотрите также**
- [Управление таблицами ReplicatedMergeTree](../../sql-reference/statements/system.md/#query-language-system-replicated)
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/replication_queue) <!--hide-->

View File

@ -59,7 +59,7 @@ Komut satırı dayanmaktadır replxx (benzer readline). Başka bir d
Varsayılan olarak, kullanılan biçim PrettyCompact. Sorgunun biçim yan tümcesinde veya belirterek biçimi değiştirebilirsiniz `\G` sorgunun sonunda, `--format` veya `--vertical` komut satırında veya istemci yapılandırma dosyasını kullanarak bağımsız değişken.
İstemciden çıkmak için Ctrl+D (veya Ctrl+C) tuşlarına basın veya bir sorgu yerine aşağıdakilerden birini girin: “exit”, “quit”, “logout”, “exit;”, “quit;”, “logout;”, “q”, “Q”, “:q”
İstemciden çıkmak için Ctrl+D tuşlarına basın veya bir sorgu yerine aşağıdakilerden birini girin: “exit”, “quit”, “logout”, “exit;”, “quit;”, “logout;”, “q”, “Q”, “:q”
Bir sorguyu işlerken, istemci şunları gösterir:

View File

@ -79,6 +79,26 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
return rewritten_query.str();
}
/// SHOW SETTINGS
if (query.m_settings)
{
WriteBufferFromOwnString rewritten_query;
rewritten_query << "SELECT name, type, value FROM system.settings";
if (query.changed)
rewritten_query << " WHERE changed = 1";
if (!query.like.empty())
{
rewritten_query
<< (query.changed ? " AND name " : " WHERE name ")
<< (query.case_insensitive_like ? "ILIKE " : "LIKE ")
<< DB::quote << query.like;
}
return rewritten_query.str();
}
if (query.temporary && !query.from.empty())
throw Exception("The `FROM` and `TEMPORARY` cannot be used together in `SHOW TABLES`", ErrorCodes::SYNTAX_ERROR);

View File

@ -55,6 +55,12 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW CLUSTER" << (settings.hilite ? hilite_none : "");
settings.ostr << " " << backQuoteIfNeed(cluster_str);
}
else if (m_settings)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW " << (changed ? "CHANGED " : "") << "SETTINGS" <<
(settings.hilite ? hilite_none : "");
formatLike(settings);
}
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW " << (temporary ? "TEMPORARY " : "") <<

View File

@ -18,6 +18,8 @@ public:
bool clusters{false};
bool cluster{false};
bool dictionaries{false};
bool m_settings{false};
bool changed{false};
bool temporary{false};
String cluster_str;

View File

@ -24,6 +24,8 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
ParserKeyword s_clusters("CLUSTERS");
ParserKeyword s_cluster("CLUSTER");
ParserKeyword s_dictionaries("DICTIONARIES");
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_changed("CHANGED");
ParserKeyword s_from("FROM");
ParserKeyword s_in("IN");
ParserKeyword s_not("NOT");
@ -99,6 +101,29 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
query->cluster_str = std::move(cluster_str);
}
else if (bool changed = s_changed.ignore(pos); changed || s_settings.ignore(pos))
{
query->m_settings = true;
if (changed)
{
query->changed = true;
if (!s_settings.ignore(pos, expected))
return false;
}
/// Not expected due to "SHOW SETTINGS PROFILES"
if (bool insensitive = s_ilike.ignore(pos); insensitive || s_like.ignore(pos))
{
if (insensitive)
query->case_insensitive_like = true;
if (!like_p.parse(pos, like, expected))
return false;
}
else
return false;
}
else
{
if (s_temporary.ignore(pos))

View File

@ -793,6 +793,7 @@ void AlterCommands::apply(StorageInMemoryMetadata & metadata, const Context & co
throw DB::Exception("Alter commands is not prepared. Cannot apply. It's a bug", ErrorCodes::LOGICAL_ERROR);
auto metadata_copy = metadata;
for (const AlterCommand & command : *this)
if (!command.ignore)
command.apply(metadata_copy, context);
@ -823,6 +824,7 @@ void AlterCommands::apply(StorageInMemoryMetadata & metadata, const Context & co
/// Changes in columns may lead to changes in TTL expressions.
auto column_ttl_asts = metadata_copy.columns.getColumnTTLs();
metadata_copy.column_ttls_by_name.clear();
for (const auto & [name, ast] : column_ttl_asts)
{
auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, metadata_copy.columns, context, metadata_copy.primary_key);
@ -830,7 +832,7 @@ void AlterCommands::apply(StorageInMemoryMetadata & metadata, const Context & co
}
if (metadata_copy.table_ttl.definition_ast != nullptr)
metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(
metadata_copy.table_ttl = TTLTableDescription::getTTLForTableFromAST(
metadata_copy.table_ttl.definition_ast, metadata_copy.columns, context, metadata_copy.primary_key);
metadata = std::move(metadata_copy);

View File

@ -15,16 +15,6 @@ install (
COMPONENT clickhouse
PATTERN "CMakeLists.txt" EXCLUDE
PATTERN ".gitignore" EXCLUDE
PATTERN "top_level_domains" EXCLUDE
)
# Dereference symlink
get_filename_component(TOP_LEVEL_DOMAINS_ABS_DIR config/top_level_domains REALPATH)
install (
DIRECTORY "${TOP_LEVEL_DOMAINS_ABS_DIR}"
DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse-test/config
USE_SOURCE_PERMISSIONS
COMPONENT clickhouse
)
install (FILES server-test.xml DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhouse-server COMPONENT clickhouse)

View File

@ -1 +0,0 @@
../../docker/test/performance-comparison/config/top_level_domains

View File

@ -0,0 +1,5 @@
send_timeout Seconds 300
connect_timeout Seconds 10
connect_timeout_with_failover_ms Milliseconds 50
connect_timeout_with_failover_secure_ms Milliseconds 100
max_memory_usage UInt64 10000000000

View File

@ -0,0 +1,3 @@
show settings like 'send_timeout';
SHOW SETTINGS ILIKE '%CONNECT_timeout%';
SHOW CHANGED SETTINGS ILIKE '%MEMORY%';

View File

@ -0,0 +1,6 @@
1 32
2 0
CREATE TABLE default.table_with_column_ttl\n(\n `EventTime` DateTime,\n `UserID` UInt64,\n `Age` UInt8\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
1 32
2 0
3 27

View File

@ -0,0 +1,30 @@
DROP TABLE IF EXISTS table_with_column_ttl;
CREATE TABLE table_with_column_ttl
(
EventTime DateTime,
UserID UInt64,
Age UInt8 TTL EventTime + INTERVAL 3 MONTH
)
ENGINE MergeTree()
ORDER BY tuple()
SETTINGS min_bytes_for_wide_part = 0; -- column TTL doesn't work for compact parts
INSERT INTO table_with_column_ttl VALUES (now(), 1, 32);
INSERT INTO table_with_column_ttl VALUES (now() - INTERVAL 4 MONTH, 2, 45);
OPTIMIZE TABLE table_with_column_ttl FINAL;
SELECT UserID, Age FROM table_with_column_ttl ORDER BY UserID;
ALTER TABLE table_with_column_ttl MODIFY COLUMN Age REMOVE TTL;
SHOW CREATE TABLE table_with_column_ttl;
INSERT INTO table_with_column_ttl VALUES (now() - INTERVAL 10 MONTH, 3, 27);
OPTIMIZE TABLE table_with_column_ttl FINAL;
SELECT UserID, Age FROM table_with_column_ttl ORDER BY UserID;
DROP TABLE table_with_column_ttl;