remove experimental_allow_extended_storage_definition_syntax Setting (turn on by default) [#CLICKHOUSE-3000]

This commit is contained in:
Alexey Zatelepin 2017-12-04 22:45:18 +03:00
parent b7096daed9
commit fb9c7ab81d
9 changed files with 7 additions and 34 deletions

View File

@ -300,12 +300,6 @@ struct Settings
M(SettingUInt64, insert_distributed_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled.") \
/* Timeout for DDL query responses from all hosts in cluster. Negative value means infinite. */ \
M(SettingInt64, distributed_ddl_task_timeout, 120, "Timeout for DDL query responses from all hosts in cluster. Negative value means infinite.") \
\
/** If true, allow parameters of storage engines such as partitioning expression, primary key, etc. \
* to be set not in the engine parameters but as separate clauses (PARTITION BY, ORDER BY...) \
* Enable this setting to allow custom MergeTree partitions. \
*/ \
M(SettingBool, experimental_allow_extended_storage_definition_syntax, false, "If true, allow parameters of storage engines such as partitioning expression, primary key, etc.") \
/* Timeout for flushing data from streaming storages. */ \
M(SettingMilliseconds, stream_flush_interval_ms, DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS, "Timeout for flushing data from streaming storages.") \
/* Schema identifier (used by schema-based formats) */ \

View File

@ -790,14 +790,16 @@ StoragePtr StorageFactory::get(
* GraphiteMergeTree(date, [sample_key], primary_key, index_granularity, 'config_element')
* UnsortedMergeTree(date, index_granularity) TODO Add description below.
*
* Alternatively, if experimental_allow_extended_storage_definition_syntax setting is specified,
* you can specify:
* Alternatively, you can specify:
* - Partitioning expression in the PARTITION BY clause;
* - Primary key in the ORDER BY clause;
* - Sampling expression in the SAMPLE BY clause;
* - Additional MergeTreeSettings in the SETTINGS clause;
*/
bool is_extended_storage_def =
storage_def.partition_by || storage_def.order_by || storage_def.sample_by || storage_def.settings;
String name_part = name.substr(0, name.size() - strlen("MergeTree"));
bool replicated = startsWith(name_part, "Replicated");
@ -807,9 +809,6 @@ StoragePtr StorageFactory::get(
MergeTreeData::MergingParams merging_params;
merging_params.mode = MergeTreeData::MergingParams::Ordinary;
const bool allow_extended_storage_def =
attach || local_context.getSettingsRef().experimental_allow_extended_storage_definition_syntax;
if (name_part == "Collapsing")
merging_params.mode = MergeTreeData::MergingParams::Collapsing;
else if (name_part == "Summing")
@ -824,7 +823,7 @@ StoragePtr StorageFactory::get(
merging_params.mode = MergeTreeData::MergingParams::Graphite;
else if (!name_part.empty())
throw Exception(
"Unknown storage " + name + getMergeTreeVerboseHelp(allow_extended_storage_def),
"Unknown storage " + name + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::UNKNOWN_STORAGE);
ASTs args;
@ -833,14 +832,6 @@ StoragePtr StorageFactory::get(
/// NOTE Quite complicated.
bool is_extended_storage_def =
storage_def.partition_by || storage_def.order_by || storage_def.sample_by || storage_def.settings;
if (is_extended_storage_def && !allow_extended_storage_def)
throw Exception(
"Extended storage definition syntax (PARTITION BY, ORDER BY, SAMPLE BY and SETTINGS clauses) "
"is disabled. Enable it with experimental_allow_extended_storage_definition_syntax user setting");
size_t min_num_params = 0;
size_t max_num_params = 0;
String needed_params;
@ -870,7 +861,7 @@ StoragePtr StorageFactory::get(
{
if (merging_params.mode == MergeTreeData::MergingParams::Unsorted)
{
if (args.size() == min_num_params && allow_extended_storage_def)
if (args.size() == min_num_params)
is_extended_storage_def = true;
else
{

View File

@ -1,5 +1,3 @@
SET experimental_allow_extended_storage_definition_syntax = 1;
SELECT '*** Not partitioned ***';
DROP TABLE IF EXISTS test.not_partitioned;

View File

@ -1,5 +1,3 @@
SET experimental_allow_extended_storage_definition_syntax = 1;
SET replication_alter_partitions_sync = 2;
SELECT '*** Not partitioned ***';

View File

@ -1,5 +1,3 @@
SET experimental_allow_extended_storage_definition_syntax = 1;
SELECT '*** Without PARTITION BY and ORDER BY ***';
DROP TABLE IF EXISTS test.unsorted;

View File

@ -1,5 +1,3 @@
SET experimental_allow_extended_storage_definition_syntax=1;
DROP TABLE IF EXISTS test.with_deduplication;
DROP TABLE IF EXISTS test.without_deduplication;
DROP TABLE IF EXISTS test.with_deduplication_mv;

View File

@ -1,5 +1,3 @@
SET experimental_allow_extended_storage_definition_syntax = 1;
DROP TABLE IF EXISTS test.multidimensional;
CREATE TABLE test.multidimensional (x UInt64, arr Array(Array(String))) ENGINE = MergeTree ORDER BY x;

View File

@ -1,5 +1,3 @@
SET experimental_allow_extended_storage_definition_syntax = 1;
DROP TABLE IF EXISTS test.multidimensional;
CREATE TABLE test.multidimensional ENGINE = MergeTree ORDER BY number AS SELECT number, arrayMap(x -> (x, [x], [[x]], (x, toString(x))), arrayMap(x -> range(x), range(number % 10))) AS value FROM system.numbers LIMIT 100000;

View File

@ -3,7 +3,7 @@
Произвольный ключ партиционирования
===================================
Начиная с версии 1.1.54310 доступна возможность создания таблиц семейства MergeTree с произвольным выражением партиционирования (не только по месяцу). Для того, чтобы эта возможность была доступна, нужно включить пользовательскую настройку `experimental_allow_extended_storage_definition_syntax=1` в сессии, которая будет создавать таблицу.
Начиная с версии 1.1.54310 доступна возможность создания таблиц семейства MergeTree с произвольным выражением партиционирования (не только по месяцу).
Ключ партиционирования может представлять собой произвольное выражение из столбцов таблицы, а также кортеж из таких выражений (аналогично первичному ключу). Ключ партиционирования может отсутствовать. При создании таблицы ключ партиционирования указывается в описании движка (ENGINE) с новым синтаксисом: