Merge pull request #11715 from azat/dist-optimize_skip_unused_shards-fixes

Control nesting level for shards skipping and disallow non-deterministic functions
This commit is contained in:
alexey-milovidov 2020-06-24 12:54:58 +03:00 committed by GitHub
commit 18eb141ea1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 155 additions and 90 deletions

View File

@ -1129,6 +1129,18 @@ Possible values:
Default value: 0
## optimize\_skip\_unused\_shards\_nesting {#optimize-skip-unused-shards-nesting}
Controls [`optimize_skip_unused_shards`](#optimize-skip-unused-shards) (hence still requires [`optimize_skip_unused_shards`](#optimize-skip-unused-shards)) depends on the nesting level of the distributed query (case when you have `Distributed` table that look into another `Distributed` table).
Possible values:
- 0 — Disabled, `optimize_skip_unused_shards` works always.
- 1 — Enables `optimize_skip_unused_shards` only for the first level.
- 2 — Enables `optimize_skip_unused_shards` up to the second level.
Default value: 0
## force\_optimize\_skip\_unused\_shards {#force-optimize-skip-unused-shards}
Enables or disables query execution if [optimize\_skip\_unused\_shards](#optimize-skip-unused-shards) is enabled and skipping of unused shards is not possible. If the skipping is not possible and the setting is enabled, an exception will be thrown.
@ -1141,16 +1153,17 @@ Possible values:
Default value: 0
## force\_optimize\_skip\_unused\_shards\_no\_nested {#settings-force_optimize_skip_unused_shards_no_nested}
## force\_optimize\_skip\_unused\_shards\_nesting {#settings-force_optimize_skip_unused_shards_nesting}
Reset [`optimize_skip_unused_shards`](#optimize-skip-unused-shards) for nested `Distributed` table
Controls [`force_optimize_skip_unused_shards`](#force-optimize-skip-unused-shards) (hence still requires [`force_optimize_skip_unused_shards`](#force-optimize-skip-unused-shards)) depends on the nesting level of the distributed query (case when you have `Distributed` table that look into another `Distributed` table).
Possible values:
- 1 — Enabled.
- 0 — Disabled.
- 0 - Disabled, `force_optimize_skip_unused_shards` works always.
- 1 — Enables `force_optimize_skip_unused_shards` only for the first level.
- 2 — Enables `force_optimize_skip_unused_shards` up to the second level.
Default value: 0.
Default value: 0
## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop}

View File

@ -1048,17 +1048,6 @@ Valores posibles:
Valor predeterminado: 0
## force\_optimize\_skip\_unused\_shards\_no\_nested {#settings-force_optimize_skip_unused_shards_no_nested}
Restablecer [`optimize_skip_unused_shards`](#settings-force_optimize_skip_unused_shards) para anidados `Distributed` tabla
Valores posibles:
- 1 — Enabled.
- 0 — Disabled.
Valor predeterminado: 0.
## Optize\_throw\_if\_noop {#setting-optimize_throw_if_noop}
Habilita o deshabilita el lanzamiento de una excepción [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) la consulta no realizó una fusión.

View File

@ -1048,17 +1048,6 @@ The results of the compilation are saved in the build directory in the form of .
مقدار پیشفرض: 0
## به زور \_بهتیتیتیتی\_سکیپ\_اس\_ش\_شارد\_مایش داده میشود {#settings-force_optimize_skip_unused_shards_no_nested}
بازنشانی [`optimize_skip_unused_shards`](#settings-force_optimize_skip_unused_shards) برای تو در تو `Distributed` جدول
مقادیر ممکن:
- 1 — Enabled.
- 0 — Disabled.
مقدار پیش فرض: 0.
## ا\_فزون\_ف\_کوپ {#setting-optimize_throw_if_noop}
را قادر می سازد و یا غیر فعال پرتاب یک استثنا اگر یک [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) پرس و جو یک ادغام انجام نمی.

View File

@ -1048,17 +1048,6 @@ Valeurs possibles:
Valeur par défaut: 0
## force\_optimize\_skip\_unused\_shards\_no\_nested {#settings-force_optimize_skip_unused_shards_no_nested}
Réinitialiser [`optimize_skip_unused_shards`](#settings-force_optimize_skip_unused_shards) pour imbriquée `Distributed` table
Valeurs possibles:
- 1 — Enabled.
- 0 — Disabled.
Valeur par défaut: 0.
## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop}
Active ou désactive le lancement d'une exception si [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) la requête n'a pas effectué de fusion.

View File

@ -1048,17 +1048,6 @@ PREWHERE/WHEREにシャーディングキー条件があるSELECTクエリの未
デフォルト値:0
## force\_optimize\_skip\_unused\_shards\_no\_nested {#settings-force_optimize_skip_unused_shards_no_nested}
リセット [`optimize_skip_unused_shards`](#settings-force_optimize_skip_unused_shards) 入れ子の場合 `Distributed` テーブル
可能な値:
- 1 — Enabled.
- 0 — Disabled.
デフォルト値は0です。
## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop}
例外のスローを有効または無効にします。 [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) クエリがマージを実行しませんでした。

View File

@ -1025,7 +1025,7 @@ ClickHouse генерирует исключение
Значение по умолчанию: 0.
## optimize_skip_unused_shards {#optimize-skip-unused-shards}
## optimize\_skip\_unused\_shards {#optimize-skip-unused-shards}
Включает или отключает пропуск неиспользуемых шардов для запросов [SELECT](../../sql-reference/statements/select/index.md) , в которых условие ключа шардирования задано в секции `WHERE/PREWHERE`. Предполагается, что данные распределены с помощью ключа шардирования, в противном случае настройка ничего не делает.
@ -1036,15 +1036,39 @@ ClickHouse генерирует исключение
Значение по умолчанию: 0
## force_optimize_skip_unused_shards {#force-optimize-skip-unused-shards}
## optimize\_skip\_unused\_shards\_nesting {#optimize-skip-unused-shards-nesting}
Контролирует настройку [`optimize_skip_unused_shards`](#optimize-skip-unused-shards) (поэтому все еще требует `optimize_skip_unused_shards`) в зависимости от вложенности распределенного запроса (когда у вас есть `Distributed` таблица которая смотрит на другую `Distributed` таблицу).
Возможные значения:
- 0 — Выключена, `optimize_skip_unused_shards` работает всегда.
- 1 — Включает `optimize_skip_unused_shards` только для 1-ого уровня вложенности.
- 2 — Включает `optimize_skip_unused_shards` для 1-ого и 2-ого уровня вложенности.
Значение по умолчанию: 0
## force\_optimize\_skip\_unused\_shards {#force-optimize-skip-unused-shards}
Разрешает или запрещает выполнение запроса, если настройка [optimize_skip_unused_shards](#optimize-skip-unused-shards) включена, а пропуск неиспользуемых шардов невозможен. Если данная настройка включена и пропуск невозможен, ClickHouse генерирует исключение.
Возможные значения:
- 0 — Выключена. ClickHouse не генерирует исключение.
- 1 — Включена. Выполнение запроса запрещается, только если у таблицы есть ключ шардирования.
- 2 — Включена. Выполнение запроса запрещается, даже если для таблицы не определен ключ шардирования.
- 0 — Выключена, `force_optimize_skip_unused_shards` работает всегда.
- 1 — Включает `force_optimize_skip_unused_shards` только для 1-ого уровня вложенности.
- 2 — Включает `force_optimize_skip_unused_shards` для 1-ого и 2-ого уровня вложенности.
Значение по умолчанию: 0
## force\_optimize\_skip\_unused\_shards\_nesting {#settings-force_optimize_skip_unused_shards_nesting}
Контролирует настройку [`force_optimize_skip_unused_shards`](#force-optimize-skip-unused-shards) (поэтому все еще требует `optimize_skip_unused_shards`) в зависимости от вложенности распределенного запроса (когда у вас есть `Distributed` таблица которая смотрит на другую `Distributed` таблицу).
Возможные значения:
- 0 - Disabled, `force_optimize_skip_unused_shards` works on all levels.
- 1 — Enables `force_optimize_skip_unused_shards` only for the first level.
- 2 — Enables `force_optimize_skip_unused_shards` up to the second level.
Значение по умолчанию: 0

View File

@ -1048,17 +1048,6 @@ Olası değerler:
Varsayılan değer: 0
## force\_optimize\_skip\_unused\_shards\_no\_nested {#settings-force_optimize_skip_unused_shards_no_nested}
Sıfırlamak [`optimize_skip_unused_shards`](#settings-force_optimize_skip_unused_shards) iç içe geçmiş için `Distributed` Tablo
Olası değerler:
- 1 — Enabled.
- 0 — Disabled.
Varsayılan değer: 0.
## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop}
Bir özel durum atmayı etkinleştirir veya devre dışı bırakır. [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) sorgu birleştirme gerçekleştirmedi.

View File

@ -1048,17 +1048,6 @@ ClickHouse生成异常
默认值0
## force\_optimize\_skip\_unused\_shards\_no\_nested {#settings-force_optimize_skip_unused_shards_no_nested}
重置 [`optimize_skip_unused_shards`](#settings-force_optimize_skip_unused_shards) 对于嵌套 `Distributed`
可能的值:
- 1 — Enabled.
- 0 — Disabled.
默认值0。
## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop}
启用或禁用抛出异常,如果 [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) 查询未执行合并。

View File

@ -121,10 +121,11 @@ struct Settings : public SettingsCollection<Settings>
\
M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.", 0) \
M(SettingBool, parallel_distributed_insert_select, false, "If true, distributed insert select query in the same cluster will be processed on local tables on every shard", 0) \
M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \
M(SettingBool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avodiing costly aggregation on the initiator server).", 0) \
M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \
M(SettingUInt64, force_optimize_skip_unused_shards, 0, "Throw an exception if unused shards cannot be skipped (1 - throw only if the table has the sharding key, 2 - always throw.", 0) \
M(SettingBool, force_optimize_skip_unused_shards_no_nested, false, "Do not apply force_optimize_skip_unused_shards for nested Distributed tables.", 0) \
M(SettingUInt64, optimize_skip_unused_shards_nesting, 0, "Same as optimize_skip_unused_shards, but accept nesting level until which it will work.", 0) \
M(SettingUInt64, force_optimize_skip_unused_shards_nesting, 0, "Same as force_optimize_skip_unused_shards, but accept nesting level until which it will work.", 0) \
\
M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \
M(SettingUInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \
@ -397,6 +398,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \
M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \
\
M(SettingBool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \
M(SettingBool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0)
#define FORMAT_FACTORY_SETTINGS(M) \

View File

@ -15,7 +15,7 @@ namespace DB
namespace ClusterProxy
{
Context removeUserRestrictionsFromSettings(const Context & context, const Settings & settings)
Context removeUserRestrictionsFromSettings(const Context & context, const Settings & settings, Poco::Logger * log)
{
Settings new_settings = settings;
new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time);
@ -28,10 +28,44 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin
new_settings.max_concurrent_queries_for_user.changed = false;
new_settings.max_memory_usage_for_user.changed = false;
if (settings.force_optimize_skip_unused_shards_no_nested)
if (settings.force_optimize_skip_unused_shards_nesting)
{
new_settings.force_optimize_skip_unused_shards = 0;
new_settings.force_optimize_skip_unused_shards.changed = false;
if (new_settings.force_optimize_skip_unused_shards_nesting == 1)
{
new_settings.force_optimize_skip_unused_shards = false;
new_settings.force_optimize_skip_unused_shards.changed = false;
if (log)
LOG_TRACE(log, "Disabling force_optimize_skip_unused_shards for nested queries (force_optimize_skip_unused_shards_nesting exceeded)");
}
else
{
--new_settings.force_optimize_skip_unused_shards_nesting.value;
new_settings.force_optimize_skip_unused_shards_nesting.changed = true;
if (log)
LOG_TRACE(log, "force_optimize_skip_unused_shards_nesting is now {}", new_settings.force_optimize_skip_unused_shards_nesting);
}
}
if (settings.optimize_skip_unused_shards_nesting)
{
if (new_settings.optimize_skip_unused_shards_nesting == 1)
{
new_settings.optimize_skip_unused_shards = false;
new_settings.optimize_skip_unused_shards.changed = false;
if (log)
LOG_TRACE(log, "Disabling optimize_skip_unused_shards for nested queries (optimize_skip_unused_shards_nesting exceeded)");
}
else
{
--new_settings.optimize_skip_unused_shards_nesting.value;
new_settings.optimize_skip_unused_shards_nesting.changed = true;
if (log)
LOG_TRACE(log, "optimize_skip_unused_shards_nesting is now {}", new_settings.optimize_skip_unused_shards_nesting);
}
}
Context new_context(context);
@ -41,14 +75,16 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin
}
Pipes executeQuery(
IStreamFactory & stream_factory, const ClusterPtr & cluster,
IStreamFactory & stream_factory, const ClusterPtr & cluster, Poco::Logger * log,
const ASTPtr & query_ast, const Context & context, const Settings & settings, const SelectQueryInfo & query_info)
{
assert(log);
Pipes res;
const std::string query = queryToString(query_ast);
Context new_context = removeUserRestrictionsFromSettings(context, settings);
Context new_context = removeUserRestrictionsFromSettings(context, settings, log);
ThrottlerPtr user_level_throttler;
if (auto * process_list_element = context.getProcessListElement())

View File

@ -21,13 +21,13 @@ class IStreamFactory;
/// removes different restrictions (like max_concurrent_queries_for_user, max_memory_usage_for_user, etc.)
/// from settings and creates new context with them
Context removeUserRestrictionsFromSettings(const Context & context, const Settings & settings);
Context removeUserRestrictionsFromSettings(const Context & context, const Settings & settings, Poco::Logger * log = nullptr);
/// Execute a distributed query, creating a vector of BlockInputStreams, from which the result can be read.
/// `stream_factory` object encapsulates the logic of creating streams for a different type of query
/// (currently SELECT, DESCRIBE).
Pipes executeQuery(
IStreamFactory & stream_factory, const ClusterPtr & cluster,
IStreamFactory & stream_factory, const ClusterPtr & cluster, Poco::Logger * log,
const ASTPtr & query_ast, const Context & context, const Settings & settings, const SelectQueryInfo & query_info);
}

View File

@ -45,6 +45,7 @@
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/getClusterName.h>
#include <Interpreters/getTableExpressions.h>
#include <Functions/IFunction.h>
#include <Core/Field.h>
#include <Core/Settings.h>
@ -188,6 +189,18 @@ ExpressionActionsPtr buildShardingKeyExpression(const ASTPtr & sharding_key, con
return ExpressionAnalyzer(query, syntax_result, context).getActions(project);
}
bool isExpressionActionsDeterministics(const ExpressionActionsPtr & actions)
{
for (const auto & action : actions->getActions())
{
if (action.type != ExpressionAction::APPLY_FUNCTION)
continue;
if (!action.function_base->isDeterministic())
return false;
}
return true;
}
class ReplacingConstantExpressionsMatcher
{
public:
@ -299,6 +312,7 @@ StorageDistributed::StorageDistributed(
{
sharding_key_expr = buildShardingKeyExpression(sharding_key_, *global_context, storage_metadata.getColumns().getAllPhysical(), false);
sharding_key_column_name = sharding_key_->getColumnName();
sharding_key_is_deterministic = isExpressionActionsDeterministics(sharding_key_expr);
}
if (!relative_data_path.empty())
@ -514,8 +528,8 @@ Pipes StorageDistributed::read(
: ClusterProxy::SelectStreamFactory(
header, processed_stage, StorageID{remote_database, remote_table}, scalars, has_virtual_shard_num_column, context.getExternalTables());
return ClusterProxy::executeQuery(
select_stream_factory, cluster, modified_query_ast, context, context.getSettingsRef(), query_info);
return ClusterProxy::executeQuery(select_stream_factory, cluster, log,
modified_query_ast, context, context.getSettingsRef(), query_info);
}
@ -695,7 +709,7 @@ ClusterPtr StorageDistributed::getOptimizedCluster(const Context & context, cons
ClusterPtr cluster = getCluster();
const Settings & settings = context.getSettingsRef();
if (has_sharding_key)
if (has_sharding_key && sharding_key_is_deterministic)
{
ClusterPtr optimized = skipUnusedShards(cluster, query_ptr, metadata_snapshot, context);
if (optimized)
@ -708,6 +722,8 @@ ClusterPtr StorageDistributed::getOptimizedCluster(const Context & context, cons
std::stringstream exception_message;
if (!has_sharding_key)
exception_message << "No sharding key";
else if (sharding_key_is_deterministic)
exception_message << "Sharding key is not deterministic";
else
exception_message << "Sharding key " << sharding_key_column_name << " is not used";

View File

@ -143,6 +143,7 @@ public:
const String cluster_name;
bool has_sharding_key;
bool sharding_key_is_deterministic = false;
ExpressionActionsPtr sharding_key_expr;
String sharding_key_column_name;

View File

@ -24,6 +24,12 @@ set force_optimize_skip_unused_shards=1;
select * from dist_01071; -- { serverError 507 }
set force_optimize_skip_unused_shards=2;
select * from dist_01071; -- { serverError 507 }
drop table if exists dist_01071;
-- non deterministic function (i.e. rand())
create table dist_01071 as data_01071 Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01071, key + rand());
set force_optimize_skip_unused_shards=1;
select * from dist_01071 where key = 0; -- { serverError 507 }
drop table if exists data_01071;
drop table if exists dist_01071;
@ -35,7 +41,7 @@ create table data2_01071 (key Int, sub_key Int) Engine=Null();
create table dist2_layer_01071 as data2_01071 Engine=Distributed(test_cluster_two_shards, currentDatabase(), data2_01071, sub_key%2);
create table dist2_01071 as data2_01071 Engine=Distributed(test_cluster_two_shards, currentDatabase(), dist2_layer_01071, key%2);
select * from dist2_01071 where key = 1; -- { serverError 507 }
set force_optimize_skip_unused_shards_no_nested=1;
set force_optimize_skip_unused_shards_nesting=1;
select * from dist2_01071 where key = 1;
drop table if exists data2_01071;
drop table if exists dist2_layer_01071;

View File

@ -3,6 +3,7 @@ DROP TABLE IF EXISTS distributed_table_1;
DROP TABLE IF EXISTS distributed_table_2;
DROP TABLE IF EXISTS local_table_1;
DROP TABLE IF EXISTS local_table_2;
DROP TABLE IF EXISTS local_table_merged;
CREATE TABLE local_table_1 (id String) ENGINE = MergeTree ORDER BY (id);
CREATE TABLE local_table_2(id String) ENGINE = MergeTree ORDER BY (id);

View File

@ -0,0 +1,22 @@
drop table if exists data_01319;
drop table if exists dist_01319;
drop table if exists dist_layer_01319;
create table data_01319 (key Int, sub_key Int) Engine=Null();
create table dist_layer_01319 as data_01319 Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01319, sub_key);
-- test_unavailable_shard here to check that optimize_skip_unused_shards always
-- remove some nodes from the cluster for the first nesting level
create table dist_01319 as data_01319 Engine=Distributed(test_unavailable_shard, currentDatabase(), dist_layer_01319, key+1);
set optimize_skip_unused_shards=1;
set force_optimize_skip_unused_shards=1;
set force_optimize_skip_unused_shards_nesting=2;
set optimize_skip_unused_shards_nesting=2;
select * from dist_01319 where key = 1; -- { serverError 507 }
set force_optimize_skip_unused_shards_nesting=1;
select * from dist_01319 where key = 1;
set force_optimize_skip_unused_shards_nesting=2;
set optimize_skip_unused_shards_nesting=1;
select * from dist_01319 where key = 1;

View File

@ -0,0 +1,10 @@
drop table if exists data_01320;
drop table if exists dist_01320;
create table data_01320 (key Int) Engine=Null();
-- non deterministic function (i.e. rand())
create table dist_01320 as data_01320 Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01320, key + rand());
set optimize_skip_unused_shards=1;
set force_optimize_skip_unused_shards=1;
select * from dist_01320 where key = 0; -- { serverError 507 }