mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
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:
commit
18eb141ea1
@ -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}
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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) پرس و جو یک ادغام انجام نمی.
|
||||
|
@ -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.
|
||||
|
@ -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) クエリがマージを実行しませんでした。
|
||||
|
@ -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
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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) 查询未执行合并。
|
||||
|
@ -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) \
|
||||
|
@ -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())
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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";
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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;
|
@ -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 }
|
Loading…
Reference in New Issue
Block a user