From 4f13521aa6de2fc6813214307da5e4c83ded9ea5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 15 Jun 2022 14:50:47 +0800 Subject: [PATCH 01/72] struct type support for storage hive --- .../hive_server/prepare_hive_data.sh | 5 +++ src/DataTypes/NestedUtils.cpp | 22 +++++++++++-- src/Storages/Hive/StorageHive.cpp | 21 +++++++++--- src/Storages/Hive/StorageHive.h | 2 ++ tests/integration/test_hive_query/test.py | 33 +++++++++++++++++++ 5 files changed, 75 insertions(+), 8 deletions(-) diff --git a/docker/test/integration/hive_server/prepare_hive_data.sh b/docker/test/integration/hive_server/prepare_hive_data.sh index 8126b975612..cee5581e2db 100755 --- a/docker/test/integration/hive_server/prepare_hive_data.sh +++ b/docker/test/integration/hive_server/prepare_hive_data.sh @@ -8,3 +8,8 @@ hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo pa hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '00' as hour from test.demo;" hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '01' as hour from test.demo;" + +hive -e "CREATE TABLE test.test_hive_types( f_tinyint tinyint, f_smallint smallint, f_int int, f_integer int, f_bigint bigint, f_float float, f_double double, f_decimal decimal(10,0), f_timestamp timestamp, f_date date, f_string string, f_varchar varchar(100), f_char char(100), f_bool boolean, f_array_int array, f_array_string array, f_array_float array, f_map_int map, f_map_string map, f_map_float map, f_struct struct) PARTITIONED BY( day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat';" + +hive -e "insert into test.test_hive_types partition(day='2022-02-20') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-20 14:47:04', '2022-02-20', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3)); insert into test.test_hive_types partition(day='2022-02-19') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-19 14:47:04', '2022-02-19', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3));" + diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index 20aae08f363..db66d4d979b 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -17,7 +17,6 @@ #include - namespace DB { @@ -76,8 +75,7 @@ Block flatten(const Block & block) for (const auto & elem : block) { - const DataTypeArray * type_arr = typeid_cast(elem.type.get()); - if (type_arr) + if (const DataTypeArray * type_arr = typeid_cast(elem.type.get())) { const DataTypeTuple * type_tuple = typeid_cast(type_arr->getNestedType().get()); if (type_tuple && type_tuple->haveExplicitNames()) @@ -114,6 +112,24 @@ Block flatten(const Block & block) else res.insert(elem); } + else if (const DataTypeTuple * type_tuple = typeid_cast(elem.type.get())) + { + if (type_tuple->haveExplicitNames()) + { + const DataTypes & element_types = type_tuple->getElements(); + const Strings & names = type_tuple->getElementNames(); + const ColumnTuple * column_tuple = typeid_cast(elem.column.get()); + size_t tuple_size = column_tuple->tupleSize(); + for (size_t i = 0; i < tuple_size; ++i) + { + const auto & element_column = column_tuple->getColumn(i); + String nested_name = concatenateName(elem.name, names[i]); + res.insert(ColumnWithTypeAndName(element_column.getPtr(), element_types[i], nested_name)); + } + } + else + res.insert(elem); + } else res.insert(elem); } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 6046dd58677..bc65ab2fd97 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -37,6 +37,8 @@ #include #include #include +#include +#include namespace DB { @@ -563,8 +565,8 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( const ContextPtr & context_, PruneLevel prune_level) const { - LOG_DEBUG( - log, "Collect hive files from partition {}, prune_level:{}", boost::join(partition.values, ","), pruneLevelToString(prune_level)); + //LOG_DEBUG( + // log, "Collect hive files from partition {}, prune_level:{}", boost::join(partition.values, ","), pruneLevelToString(prune_level)); /// Skip partition "__HIVE_DEFAULT_PARTITION__" bool has_default_partition = false; @@ -766,8 +768,13 @@ Pipe StorageHive::read( sources_info->hive_metastore_client = hive_metastore_client; sources_info->partition_name_types = partition_name_types; - Block sample_block; const auto header_block = storage_snapshot->metadata->getSampleBlock(); + bool support_subset_columns = supportsSubsetOfColumns(); + Block flatten_block; + if (support_subset_columns) + flatten_block = Nested::flatten(header_block); + + Block sample_block; for (const auto & column : column_names) { if (header_block.has(column)) @@ -775,13 +782,17 @@ Pipe StorageHive::read( sample_block.insert(header_block.getByName(column)); continue; } - + else if (support_subset_columns && flatten_block.has(column)) + { + sample_block.insert(flatten_block.getByName(column)); + continue; + } if (column == "_path") sources_info->need_path_column = true; if (column == "_file") sources_info->need_file_column = true; } - + LOG_TRACE(&Poco::Logger::get("StorageHive"), "sample_block={}", sample_block.dumpNames()); if (num_streams > sources_info->hive_files.size()) num_streams = sources_info->hive_files.size(); diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index d92d2dbd745..2f69ae00b6e 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -42,6 +42,8 @@ public: String getName() const override { return "Hive"; } bool supportsIndexForIn() const override { return true; } + + bool supportsSubcolumns() const override { return true; } bool mayBenefitFromIndexForIn( const ASTPtr & /* left_in_operand */, ContextPtr /* query_context */, diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index fd4d91d6f78..00aaaec7afe 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -400,3 +400,36 @@ def test_cache_dir_use(started_cluster): ["bash", "-c", "ls /tmp/clickhouse_local_cache1 | wc -l"] ) assert result0 != "0" and result1 != "0" + +def test_cache_dir_use(started_cluster): + node = started_cluster.instances["h0_0_0"] + result0 = node.exec_in_container( + ["bash", "-c", "ls /tmp/clickhouse_local_cache | wc -l"] + ) + result1 = node.exec_in_container( + ["bash", "-c", "ls /tmp/clickhouse_local_cache1 | wc -l"] + ) + assert result0 != "0" and result1 != "0" + +def test_hive_struct_type(started_cluster): + node = started_cluster.instances["h0_0_0"] + result = node.query( + """ + CREATE TABLE IF NOT EXISTS default.test_hive_types (`f_tinyint` Int8, `f_smallint` Int16, `f_int` Int32, `f_integer` Int32, `f_bigint` Int64, `f_float` Float32, `f_double` Float64, `f_decimal` Float64, `f_timestamp` DateTime, `f_date` Date, `f_string` String, `f_varchar` String, `f_char` String, `f_bool` Boolean, `f_array_int` Array(Int32), `f_array_string` Array(String), `f_array_float` Array(Float32), `f_map_int` Map(String, Int32), `f_map_string` Map(String, String), `f_map_float` Map(String, Float32), `f_struct` Tuple(a String, b Int32, c Float32), `day` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'test_hive_types') PARTITION BY (day) + """ + ) + result = node.query( + """ + SELECT * FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 + """ + ) + expected_result = """1 2 3 4 5 6.11 7.22 8 2022-02-20 14:47:04 2022-02-20 hello world hello world hello world true [1,2,3] ['hello world','hello world'] [1.1,1.2] {'a':100,'b':200,'c':300} {'a':'aa','b':'bb','c':'cc'} {'a':111.1,'b':222.2,'c':333.3} ('aaa',200,333.3) 2022-02-20""" + assert result.strip() == expected_result + + result = node.query( + """ + SELECT day, f_struct.a FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 + """ + ) + expected_result = """2022-02-20 aaa""" + assert result.strip() == expected_result From 655e42c9bcd9c4fe409f0d72f30b6697f2a57eeb Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 16 Jun 2022 09:44:41 +0800 Subject: [PATCH 02/72] remove trace logs --- src/Storages/Hive/StorageHive.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index bc65ab2fd97..0135da97445 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -792,7 +792,7 @@ Pipe StorageHive::read( if (column == "_file") sources_info->need_file_column = true; } - LOG_TRACE(&Poco::Logger::get("StorageHive"), "sample_block={}", sample_block.dumpNames()); + if (num_streams > sources_info->hive_files.size()) num_streams = sources_info->hive_files.size(); From e115e3f7311400d0e3c1af2b4dfb13bd3ad617f5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 16 Jun 2022 09:53:04 +0800 Subject: [PATCH 03/72] remove unused header --- src/Storages/Hive/StorageHive.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 0135da97445..ab2b1ef7a09 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -38,7 +38,6 @@ #include #include #include -#include namespace DB { From 35d534c213faf5abde296dc041737d8970d0d2f3 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 16 Jun 2022 16:16:34 +0800 Subject: [PATCH 04/72] nested struct in struct --- .../hive_server/prepare_hive_data.sh | 4 +- src/DataTypes/NestedUtils.cpp | 68 +++++++++++++++- src/DataTypes/NestedUtils.h | 13 +++ .../Formats/Impl/ArrowColumnToCHColumn.cpp | 81 ++++++++++--------- src/Storages/Hive/StorageHive.cpp | 24 ++++-- tests/integration/test_hive_query/test.py | 6 +- 6 files changed, 145 insertions(+), 51 deletions(-) diff --git a/docker/test/integration/hive_server/prepare_hive_data.sh b/docker/test/integration/hive_server/prepare_hive_data.sh index cee5581e2db..39d435eb05a 100755 --- a/docker/test/integration/hive_server/prepare_hive_data.sh +++ b/docker/test/integration/hive_server/prepare_hive_data.sh @@ -9,7 +9,7 @@ hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo pa hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '00' as hour from test.demo;" hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '01' as hour from test.demo;" -hive -e "CREATE TABLE test.test_hive_types( f_tinyint tinyint, f_smallint smallint, f_int int, f_integer int, f_bigint bigint, f_float float, f_double double, f_decimal decimal(10,0), f_timestamp timestamp, f_date date, f_string string, f_varchar varchar(100), f_char char(100), f_bool boolean, f_array_int array, f_array_string array, f_array_float array, f_map_int map, f_map_string map, f_map_float map, f_struct struct) PARTITIONED BY( day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat';" +hive -e "CREATE TABLE test.test_hive_types( f_tinyint tinyint, f_smallint smallint, f_int int, f_integer int, f_bigint bigint, f_float float, f_double double, f_decimal decimal(10,0), f_timestamp timestamp, f_date date, f_string string, f_varchar varchar(100), f_char char(100), f_bool boolean, f_array_int array, f_array_string array, f_array_float array, f_map_int map, f_map_string map, f_map_float map, f_struct struct>) PARTITIONED BY( day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat';" -hive -e "insert into test.test_hive_types partition(day='2022-02-20') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-20 14:47:04', '2022-02-20', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3)); insert into test.test_hive_types partition(day='2022-02-19') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-19 14:47:04', '2022-02-19', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3));" +hive -e "insert into test.test_hive_types partition(day='2022-02-20') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-20 14:47:04', '2022-02-20', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 10, 'y', 'xyz')); insert into test.test_hive_types partition(day='2022-02-19') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-19 14:47:04', '2022-02-19', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 11, 'y', 'abc'));" diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index db66d4d979b..0df664ad408 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -259,7 +260,72 @@ std::unordered_set getAllTableNames(const Block & block, bool to_lower_c } return nested_table_names; } - } +NestedColumnExtractHelper::NestedColumnExtractHelper(const Block & block_, bool case_insentive_) + : block(block_) + , case_insentive(case_insentive_) +{} + +std::optional NestedColumnExtractHelper::extractColumn(const String & column_name) +{ + if (block.has(column_name, case_insentive)) + return {block.getByName(column_name, case_insentive)}; + + auto nested_names = Nested::splitName(column_name); + if (case_insentive) + { + boost::to_lower(nested_names.first); + boost::to_lower(nested_names.second); + } + if (!block.has(nested_names.first, case_insentive)) + return {}; + + if (!nested_tables.contains(nested_names.first)) + { + ColumnsWithTypeAndName columns = {block.getByName(nested_names.first, case_insentive)}; + nested_tables[nested_names.first] = std::make_shared(Nested::flatten(columns)); + } + + return extractColumn(column_name, nested_names.first, nested_names.second); +} + +std::optional NestedColumnExtractHelper::extractColumn( + const String & original_column_name, const String & column_name_prefix, const String & column_name_suffix) +{ + auto table_iter = nested_tables.find(column_name_prefix); + if (table_iter == nested_tables.end()) + { + return {}; + } + + auto & nested_table = table_iter->second; + auto nested_names = Nested::splitName(column_name_suffix); + auto new_column_name_prefix = Nested::concatenateName(column_name_prefix, nested_names.first); + if (nested_names.second.empty()) + { + if (nested_table->has(new_column_name_prefix, case_insentive)) + { + ColumnWithTypeAndName column = nested_table->getByName(new_column_name_prefix, case_insentive); + if (case_insentive) + column.name = original_column_name; + return {column}; + } + else + { + return {}; + } + } + + if (!nested_table->has(new_column_name_prefix, case_insentive)) + { + return {}; + } + + ColumnsWithTypeAndName columns = {nested_table->getByName(new_column_name_prefix, case_insentive)}; + Block sub_block(columns); + nested_tables[new_column_name_prefix] = std::make_shared(Nested::flatten(sub_block)); + return extractColumn(original_column_name, new_column_name_prefix, nested_names.second); + +} } diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index f6dc42d5c58..39f73b65100 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -35,4 +35,17 @@ namespace Nested std::unordered_set getAllTableNames(const Block & block, bool to_lower_case = false); } +class NestedColumnExtractHelper +{ +public: + explicit NestedColumnExtractHelper(const Block & block_, bool case_insentive_); + std::optional extractColumn(const String & column_name); +private: + std::optional + extractColumn(const String & original_column_name, const String & column_name_prefix, const String & column_name_suffix); + const Block & block; + bool case_insentive; + std::map nested_tables; +}; + } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index c792d828e44..da89d6f7321 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -1,4 +1,6 @@ #include "ArrowColumnToCHColumn.h" +#include +#include #if USE_ARROW || USE_ORC || USE_PARQUET @@ -602,8 +604,8 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & Columns columns_list; UInt64 num_rows = name_to_column_ptr.begin()->second->length(); - columns_list.reserve(header.rows()); - std::unordered_map nested_tables; + columns_list.reserve(header.columns()); + std::unordered_map>> nested_tables; bool skipped = false; for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) { @@ -613,55 +615,57 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & if (case_insensitive_matching) boost::to_lower(search_column_name); - bool read_from_nested = false; - String nested_table_name = Nested::extractTableName(header_column.name); - String search_nested_table_name = nested_table_name; - if (case_insensitive_matching) - boost::to_lower(search_nested_table_name); - + ColumnWithTypeAndName column; if (!name_to_column_ptr.contains(search_column_name)) { + bool read_from_nested = false; /// Check if it's a column from nested table. - if (import_nested && name_to_column_ptr.contains(search_nested_table_name)) + if (import_nested) { - if (!nested_tables.contains(search_nested_table_name)) + String nested_table_name = Nested::extractTableName(header_column.name); + String search_nested_table_name = nested_table_name; + if (case_insensitive_matching) + boost::to_lower(search_nested_table_name); + if (name_to_column_ptr.contains(search_nested_table_name)) { - std::shared_ptr arrow_column = name_to_column_ptr[search_nested_table_name]; - ColumnsWithTypeAndName cols - = {readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_values, true, true, false, skipped)}; - Block block(cols); - nested_tables[search_nested_table_name] = std::make_shared(Nested::flatten(block)); + if (!nested_tables.contains(search_nested_table_name)) + { + std::shared_ptr arrow_column = name_to_column_ptr[search_nested_table_name]; + ColumnsWithTypeAndName cols = {readColumnFromArrowColumn( + arrow_column, nested_table_name, format_name, false, dictionary_values, true, true, false, skipped)}; + BlockPtr block_ptr = std::make_shared(cols); + auto column_extractor = std::make_shared(*block_ptr, case_insensitive_matching); + nested_tables[search_nested_table_name] = {block_ptr, column_extractor}; + } + auto nested_column = nested_tables[search_nested_table_name].second->extractColumn(search_column_name); + if (nested_column) + { + column = *nested_column; + if (case_insensitive_matching) + column.name = header_column.name; + read_from_nested = true; + } } - - read_from_nested = nested_tables[search_nested_table_name]->has(header_column.name, case_insensitive_matching); } - if (!read_from_nested) { if (!allow_missing_columns) throw Exception{ErrorCodes::THERE_IS_NO_COLUMN, "Column '{}' is not presented in input data.", header_column.name}; - - ColumnWithTypeAndName column; - column.name = header_column.name; - column.type = header_column.type; - column.column = header_column.column->cloneResized(num_rows); - columns_list.push_back(std::move(column.column)); - continue; + else + { + column.name = header_column.name; + column.type = header_column.type; + column.column = header_column.column->cloneResized(num_rows); + columns_list.push_back(std::move(column.column)); + continue; + } } } - - - ColumnWithTypeAndName column; - if (read_from_nested) - { - column = nested_tables[search_nested_table_name]->getByName(header_column.name, case_insensitive_matching); - if (case_insensitive_matching) - column.name = header_column.name; - } else { auto arrow_column = name_to_column_ptr[search_column_name]; - column = readColumnFromArrowColumn(arrow_column, header_column.name, format_name, false, dictionary_values, true, true, false, skipped); + column = readColumnFromArrowColumn( + arrow_column, header_column.name, format_name, false, dictionary_values, true, true, false, skipped); } try @@ -689,17 +693,16 @@ std::vector ArrowColumnToCHColumn::getMissingColumns(const arrow::Schema { std::vector missing_columns; auto block_from_arrow = arrowSchemaToCHHeader(schema, format_name, false, &header, case_insensitive_matching); - auto flatten_block_from_arrow = Nested::flatten(block_from_arrow); + NestedColumnExtractHelper nested_columns_extractor(block_from_arrow, case_insensitive_matching); for (size_t i = 0, columns = header.columns(); i < columns; ++i) { const auto & header_column = header.getByPosition(i); bool read_from_nested = false; - String nested_table_name = Nested::extractTableName(header_column.name); if (!block_from_arrow.has(header_column.name, case_insensitive_matching)) { - if (import_nested && block_from_arrow.has(nested_table_name, case_insensitive_matching)) - read_from_nested = flatten_block_from_arrow.has(header_column.name, case_insensitive_matching); + if (import_nested && nested_columns_extractor.extractColumn(header_column.name)) + read_from_nested = true; if (!read_from_nested) { diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index ab2b1ef7a09..9dbfd1119dc 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -1,3 +1,4 @@ +#include #include #if USE_HIVE @@ -769,11 +770,18 @@ Pipe StorageHive::read( const auto header_block = storage_snapshot->metadata->getSampleBlock(); bool support_subset_columns = supportsSubsetOfColumns(); - Block flatten_block; - if (support_subset_columns) - flatten_block = Nested::flatten(header_block); + auto settings = context_->getSettingsRef(); + auto case_insensitive_matching = [&]() -> bool + { + if (format_name == "Parquet") + return settings.input_format_parquet_case_insensitive_column_matching; + else if (format_name == "ORC") + return settings.input_format_orc_case_insensitive_column_matching; + return false; + }; Block sample_block; + NestedColumnExtractHelper nested_columns_extractor(header_block, case_insensitive_matching()); for (const auto & column : column_names) { if (header_block.has(column)) @@ -781,10 +789,14 @@ Pipe StorageHive::read( sample_block.insert(header_block.getByName(column)); continue; } - else if (support_subset_columns && flatten_block.has(column)) + else if (support_subset_columns) { - sample_block.insert(flatten_block.getByName(column)); - continue; + auto subset_column = nested_columns_extractor.extractColumn(column); + if (subset_column) + { + sample_block.insert(*subset_column); + continue; + } } if (column == "_path") sources_info->need_path_column = true; diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 00aaaec7afe..4bac09320ce 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -415,7 +415,7 @@ def test_hive_struct_type(started_cluster): node = started_cluster.instances["h0_0_0"] result = node.query( """ - CREATE TABLE IF NOT EXISTS default.test_hive_types (`f_tinyint` Int8, `f_smallint` Int16, `f_int` Int32, `f_integer` Int32, `f_bigint` Int64, `f_float` Float32, `f_double` Float64, `f_decimal` Float64, `f_timestamp` DateTime, `f_date` Date, `f_string` String, `f_varchar` String, `f_char` String, `f_bool` Boolean, `f_array_int` Array(Int32), `f_array_string` Array(String), `f_array_float` Array(Float32), `f_map_int` Map(String, Int32), `f_map_string` Map(String, String), `f_map_float` Map(String, Float32), `f_struct` Tuple(a String, b Int32, c Float32), `day` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'test_hive_types') PARTITION BY (day) + CREATE TABLE IF NOT EXISTS default.test_hive_types (`f_tinyint` Int8, `f_smallint` Int16, `f_int` Int32, `f_integer` Int32, `f_bigint` Int64, `f_float` Float32, `f_double` Float64, `f_decimal` Float64, `f_timestamp` DateTime, `f_date` Date, `f_string` String, `f_varchar` String, `f_char` String, `f_bool` Boolean, `f_array_int` Array(Int32), `f_array_string` Array(String), `f_array_float` Array(Float32), `f_map_int` Map(String, Int32), `f_map_string` Map(String, String), `f_map_float` Map(String, Float32), `f_struct` Tuple(a String, b Int32, c Float32, d Tuple(x Int32, y String)), `day` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'test_hive_types') PARTITION BY (day) """ ) result = node.query( @@ -423,7 +423,7 @@ def test_hive_struct_type(started_cluster): SELECT * FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 """ ) - expected_result = """1 2 3 4 5 6.11 7.22 8 2022-02-20 14:47:04 2022-02-20 hello world hello world hello world true [1,2,3] ['hello world','hello world'] [1.1,1.2] {'a':100,'b':200,'c':300} {'a':'aa','b':'bb','c':'cc'} {'a':111.1,'b':222.2,'c':333.3} ('aaa',200,333.3) 2022-02-20""" + expected_result = """1 2 3 4 5 6.11 7.22 8 2022-02-20 14:47:04 2022-02-20 hello world hello world hello world true [1,2,3] ['hello world','hello world'] [1.1,1.2] {'a':100,'b':200,'c':300} {'a':'aa','b':'bb','c':'cc'} {'a':111.1,'b':222.2,'c':333.3} ('aaa',200,333.3,(10,'xyz')) 2022-02-20""" assert result.strip() == expected_result result = node.query( @@ -431,5 +431,5 @@ def test_hive_struct_type(started_cluster): SELECT day, f_struct.a FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 """ ) - expected_result = """2022-02-20 aaa""" + expected_result = """2022-02-20 aaa 10""" assert result.strip() == expected_result From a8b17fec848a698dc8d0f6886b041089c991a923 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 16 Jun 2022 16:53:11 +0800 Subject: [PATCH 05/72] fixed a bug --- src/Storages/Hive/StorageHive.cpp | 2 +- src/Storages/Hive/StorageHive.h | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 9dbfd1119dc..60936f6a3f4 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -769,7 +769,7 @@ Pipe StorageHive::read( sources_info->partition_name_types = partition_name_types; const auto header_block = storage_snapshot->metadata->getSampleBlock(); - bool support_subset_columns = supportsSubsetOfColumns(); + bool support_subset_columns = supportsSubcolumns(); auto settings = context_->getSettingsRef(); auto case_insensitive_matching = [&]() -> bool diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 2f69ae00b6e..fd806553a86 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -44,6 +44,7 @@ public: bool supportsIndexForIn() const override { return true; } bool supportsSubcolumns() const override { return true; } + bool mayBenefitFromIndexForIn( const ASTPtr & /* left_in_operand */, ContextPtr /* query_context */, From 8c629085e485f15a221f2e443f3b89e95bc0d9e0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 17 Jun 2022 09:36:59 +0800 Subject: [PATCH 06/72] simplified code --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index da89d6f7321..305009a070f 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -698,18 +698,13 @@ std::vector ArrowColumnToCHColumn::getMissingColumns(const arrow::Schema for (size_t i = 0, columns = header.columns(); i < columns; ++i) { const auto & header_column = header.getByPosition(i); - bool read_from_nested = false; if (!block_from_arrow.has(header_column.name, case_insensitive_matching)) { - if (import_nested && nested_columns_extractor.extractColumn(header_column.name)) - read_from_nested = true; - - if (!read_from_nested) + if (!import_nested || !nested_columns_extractor.extractColumn(header_column.name)) { if (!allow_missing_columns) throw Exception{ErrorCodes::THERE_IS_NO_COLUMN, "Column '{}' is not presented in input data.", header_column.name}; - - missing_columns.push_back(i); + missing_columns.push_back(i); } } } From c13bf03fe066950a48d85625d55d7d0552fad636 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 20 Jun 2022 09:56:21 +0800 Subject: [PATCH 07/72] fixed code style --- src/DataTypes/NestedUtils.cpp | 4 ++-- tests/integration/test_hive_query/test.py | 12 +++++++----- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index 0df664ad408..2e429bcff10 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -311,7 +311,7 @@ std::optional NestedColumnExtractHelper::extractColumn( column.name = original_column_name; return {column}; } - else + else { return {}; } @@ -325,7 +325,7 @@ std::optional NestedColumnExtractHelper::extractColumn( ColumnsWithTypeAndName columns = {nested_table->getByName(new_column_name_prefix, case_insentive)}; Block sub_block(columns); nested_tables[new_column_name_prefix] = std::make_shared(Nested::flatten(sub_block)); - return extractColumn(original_column_name, new_column_name_prefix, nested_names.second); + return extractColumn(original_column_name, new_column_name_prefix, nested_names.second); } } diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 4bac09320ce..538c99d1a65 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -401,6 +401,7 @@ def test_cache_dir_use(started_cluster): ) assert result0 != "0" and result1 != "0" + def test_cache_dir_use(started_cluster): node = started_cluster.instances["h0_0_0"] result0 = node.exec_in_container( @@ -411,6 +412,7 @@ def test_cache_dir_use(started_cluster): ) assert result0 != "0" and result1 != "0" + def test_hive_struct_type(started_cluster): node = started_cluster.instances["h0_0_0"] result = node.query( @@ -419,17 +421,17 @@ def test_hive_struct_type(started_cluster): """ ) result = node.query( - """ + """ SELECT * FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 - """ + """ ) - expected_result = """1 2 3 4 5 6.11 7.22 8 2022-02-20 14:47:04 2022-02-20 hello world hello world hello world true [1,2,3] ['hello world','hello world'] [1.1,1.2] {'a':100,'b':200,'c':300} {'a':'aa','b':'bb','c':'cc'} {'a':111.1,'b':222.2,'c':333.3} ('aaa',200,333.3,(10,'xyz')) 2022-02-20""" + expected_result = """1 2 3 4 5 6.11 7.22 8 2022-02-20 14:47:04 2022-02-20 hello world hello world hello world true [1,2,3] ['hello world','hello world'] [1.1,1.2] {'a':100,'b':200,'c':300} {'a':'aa','b':'bb','c':'cc'} {'a':111.1,'b':222.2,'c':333.3} ('aaa',200,333.3,(10,'xyz')) 2022-02-20""" assert result.strip() == expected_result result = node.query( - """ + """ SELECT day, f_struct.a FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 - """ + """ ) expected_result = """2022-02-20 aaa 10""" assert result.strip() == expected_result From fc641d9ce49bbd0e256ddb45d01cee02405f2f2d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 21 Jun 2022 11:29:55 +0800 Subject: [PATCH 08/72] some changes --- src/DataTypes/NestedUtils.cpp | 9 +++++++-- src/DataTypes/NestedUtils.h | 8 ++++++-- src/Storages/Hive/StorageHive.cpp | 6 +++--- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index 2e429bcff10..9cab49a509d 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -4,6 +4,7 @@ #include #include #include +#include "Columns/IColumn.h" #include #include @@ -119,7 +120,11 @@ Block flatten(const Block & block) { const DataTypes & element_types = type_tuple->getElements(); const Strings & names = type_tuple->getElementNames(); - const ColumnTuple * column_tuple = typeid_cast(elem.column.get()); + const ColumnTuple * column_tuple; + if(isColumnConst(*elem.column)) + column_tuple = typeid_cast(&assert_cast(*elem.column).getDataColumn()); + else + column_tuple = typeid_cast(elem.column.get()); size_t tuple_size = column_tuple->tupleSize(); for (size_t i = 0; i < tuple_size; ++i) { @@ -306,7 +311,7 @@ std::optional NestedColumnExtractHelper::extractColumn( { if (nested_table->has(new_column_name_prefix, case_insentive)) { - ColumnWithTypeAndName column = nested_table->getByName(new_column_name_prefix, case_insentive); + ColumnWithTypeAndName column = *nested_table->findByName(new_column_name_prefix, case_insentive); if (case_insentive) column.name = original_column_name; return {column}; diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index 39f73b65100..9473d30497a 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -18,8 +18,9 @@ namespace Nested /// Returns the prefix of the name to the first '.'. Or the name is unchanged if there is no dot. std::string extractTableName(const std::string & nested_name); - /// Replace Array(Tuple(...)) columns to a multiple of Array columns in a form of `column_name.element_name`. - /// only for named tuples that actually represent Nested structures. + /// Flat a column of nested type into columns + /// 1) For named tuples,t Tuple(x .., y ..., ...), replace it with t.x ..., t.y ... , ... + /// 2) For an Array with named Tuple element column, a Array(Tuple(x ..., y ..., ...)), replace it with multiple Array Columns, a.x ..., a.y ..., ... Block flatten(const Block & block); /// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column. @@ -35,6 +36,9 @@ namespace Nested std::unordered_set getAllTableNames(const Block & block, bool to_lower_case = false); } +/// Use this class to extract element columns from columns of nested type in a block, e.g. named Tuple. +/// It can extract a column from a multiple nested type column, e.g. named Tuple in named Tuple +/// Keeps some intermediate datas to avoid rebuild them multi-times. class NestedColumnExtractHelper { public: diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 60936f6a3f4..6d298c0033c 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -565,8 +565,8 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( const ContextPtr & context_, PruneLevel prune_level) const { - //LOG_DEBUG( - // log, "Collect hive files from partition {}, prune_level:{}", boost::join(partition.values, ","), pruneLevelToString(prune_level)); + LOG_DEBUG( + log, "Collect hive files from partition {}, prune_level:{}", boost::join(partition.values, ","), pruneLevelToString(prune_level)); /// Skip partition "__HIVE_DEFAULT_PARTITION__" bool has_default_partition = false; @@ -794,7 +794,7 @@ Pipe StorageHive::read( auto subset_column = nested_columns_extractor.extractColumn(column); if (subset_column) { - sample_block.insert(*subset_column); + sample_block.insert(std::move(*subset_column)); continue; } } From 31bf1203d337e3127a319671d704fb324b6cfb01 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Jun 2022 15:57:19 +0800 Subject: [PATCH 09/72] update codes --- src/DataTypes/NestedUtils.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index 9cab49a509d..dbdba39fa84 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -121,7 +121,7 @@ Block flatten(const Block & block) const DataTypes & element_types = type_tuple->getElements(); const Strings & names = type_tuple->getElementNames(); const ColumnTuple * column_tuple; - if(isColumnConst(*elem.column)) + if (isColumnConst(*elem.column)) column_tuple = typeid_cast(&assert_cast(*elem.column).getDataColumn()); else column_tuple = typeid_cast(elem.column.get()); @@ -309,12 +309,12 @@ std::optional NestedColumnExtractHelper::extractColumn( auto new_column_name_prefix = Nested::concatenateName(column_name_prefix, nested_names.first); if (nested_names.second.empty()) { - if (nested_table->has(new_column_name_prefix, case_insentive)) + if (auto * column_ref = nested_table->findByName(new_column_name_prefix, case_insentive)) { - ColumnWithTypeAndName column = *nested_table->findByName(new_column_name_prefix, case_insentive); + ColumnWithTypeAndName column = *column_ref; if (case_insentive) column.name = original_column_name; - return {column}; + return {std::move(column)}; } else { From 96e6f9a2d02ba6cf560703f73d6acc971b3dd445 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Jun 2022 16:10:01 +0800 Subject: [PATCH 10/72] fixed code style --- src/DataTypes/NestedUtils.h | 4 ++-- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index 9473d30497a..e7cda541f47 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -36,9 +36,9 @@ namespace Nested std::unordered_set getAllTableNames(const Block & block, bool to_lower_case = false); } -/// Use this class to extract element columns from columns of nested type in a block, e.g. named Tuple. +/// Use this class to extract element columns from columns of nested type in a block, e.g. named Tuple. /// It can extract a column from a multiple nested type column, e.g. named Tuple in named Tuple -/// Keeps some intermediate datas to avoid rebuild them multi-times. +/// Keeps some intermediate data to avoid rebuild them multi-times. class NestedColumnExtractHelper { public: diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 305009a070f..eefe4231b89 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -1,4 +1,3 @@ -#include "ArrowColumnToCHColumn.h" #include #include @@ -704,7 +703,7 @@ std::vector ArrowColumnToCHColumn::getMissingColumns(const arrow::Schema { if (!allow_missing_columns) throw Exception{ErrorCodes::THERE_IS_NO_COLUMN, "Column '{}' is not presented in input data.", header_column.name}; - missing_columns.push_back(i); + missing_columns.push_back(i); } } } From 2c5a88fabaefb7b865e8cf6d08ad8609c4342b64 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Jun 2022 09:59:13 +0200 Subject: [PATCH 11/72] deprecate Ordinary database --- docker/test/stateful/run.sh | 4 ++++ docker/test/stateless/run.sh | 4 ++++ src/Core/Settings.h | 3 ++- src/Databases/DatabaseAtomic.cpp | 6 +++--- src/Databases/DatabaseAtomic.h | 2 +- src/Databases/DatabaseFactory.cpp | 7 ++++++- src/Databases/DatabaseLazy.cpp | 4 ++-- src/Databases/DatabaseLazy.h | 2 +- src/Databases/DatabaseMemory.cpp | 2 +- src/Databases/DatabaseMemory.h | 2 +- src/Databases/DatabaseOnDisk.cpp | 2 +- src/Databases/DatabaseOnDisk.h | 2 +- src/Databases/DatabaseReplicated.cpp | 4 ++-- src/Databases/DatabaseReplicated.h | 2 +- src/Databases/IDatabase.h | 2 +- .../MySQL/DatabaseMaterializedMySQL.cpp | 4 ++-- .../MySQL/DatabaseMaterializedMySQL.h | 2 +- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- src/Databases/MySQL/DatabaseMySQL.h | 2 +- .../DatabaseMaterializedPostgreSQL.cpp | 4 ++-- .../DatabaseMaterializedPostgreSQL.h | 2 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 2 +- src/Interpreters/AsynchronousInsertQueue.h | 1 - src/Interpreters/HashJoin.h | 6 +++--- src/Interpreters/InterpreterCreateQuery.cpp | 6 ++---- src/Interpreters/InterpreterDropQuery.cpp | 18 +++++++++--------- src/Interpreters/InterpreterDropQuery.h | 2 +- src/Interpreters/loadMetadata.cpp | 7 ------- src/Parsers/ASTDropQuery.cpp | 4 ++-- src/Parsers/ASTDropQuery.h | 2 +- src/Parsers/ParserDropQuery.cpp | 6 +++--- src/Storages/IStorage.h | 2 +- .../StorageMaterializedPostgreSQL.cpp | 4 ++-- .../PostgreSQL/StorageMaterializedPostgreSQL.h | 2 +- src/Storages/StorageMaterializedView.cpp | 4 ++-- src/Storages/StorageMaterializedView.h | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 6 +++--- src/Storages/WindowView/StorageWindowView.h | 2 +- tests/clickhouse-test | 4 ++-- tests/config/users.d/database_ordinary.xml | 2 +- .../configs/users.xml | 1 - .../configs/users_disable_bytes_settings.xml | 1 - .../configs/users_disable_rows_settings.xml | 1 - .../01047_window_view_parser_inner_table.sql | 1 + .../0_stateless/01048_window_view_parser.sql | 1 + .../01053_drop_database_mat_view.sql | 1 + .../0_stateless/01085_window_view_attach.sql | 1 + .../0_stateless/01086_window_view_cleanup.sh | 2 +- .../0_stateless/01109_exchange_tables.sql | 4 +--- .../0_stateless/01114_database_atomic.sh | 4 ++-- .../01148_zookeeper_path_macros_unfolding.sql | 1 + .../01155_rename_move_materialized_view.sql | 2 ++ .../0_stateless/01162_strange_mutations.sh | 10 +++++----- .../0_stateless/01190_full_attach_syntax.sql | 1 + .../01192_rename_database_zookeeper.sh | 8 ++++---- .../01224_no_superfluous_dict_reload.sql | 1 + ...01225_show_create_table_from_dictionary.sql | 1 + .../01249_bad_arguments_for_bloom_filter.sql | 1 + ...320_create_sync_race_condition_zookeeper.sh | 2 +- .../01516_create_table_primary_key.sql | 1 + .../01517_drop_mv_with_inner_table.sql | 1 + .../0_stateless/01600_detach_permanently.sh | 2 +- .../0_stateless/01601_detach_permanently.sql | 1 + .../0_stateless/01603_rename_overwrite_bug.sql | 1 + .../01810_max_part_removal_threads_long.sh | 2 +- .../0_stateless/02096_rename_atomic_hang.sql | 1 + 67 files changed, 108 insertions(+), 93 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 5f55bb9fa21..d77978c904b 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -120,6 +120,10 @@ function run_tests() ADDITIONAL_OPTIONS+=('--replicated-database') fi + if [[ -n "$USE_DATABASE_ORDINARY" ]] && [[ "$USE_DATABASE_ORDINARY" -eq 1 ]]; then + ADDITIONAL_OPTIONS+=('--db-engine=Ordinary') + fi + set +e clickhouse-test -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time \ --skip 00168_parallel_processing_on_replicas "${ADDITIONAL_OPTIONS[@]}" \ diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 52bf8a60669..075f588cae3 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -115,6 +115,10 @@ function run_tests() ADDITIONAL_OPTIONS+=("$RUN_BY_HASH_TOTAL") fi + if [[ -n "$USE_DATABASE_ORDINARY" ]] && [[ "$USE_DATABASE_ORDINARY" -eq 1 ]]; then + ADDITIONAL_OPTIONS+=('--db-engine=Ordinary') + fi + set +e clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f1fd9d20f00..883281c94f3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -445,7 +445,6 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Seconds, wait_for_window_view_fire_signal_timeout, 10, "Timeout for waiting for window view fire signal in event time processing", 0) \ M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ - M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic, "Default database engine.", 0) \ M(DefaultTableEngine, default_table_engine, DefaultTableEngine::None, "Default table engine used when ENGINE is not set in CREATE statement.",0) \ M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \ M(Bool, database_atomic_wait_for_drop_and_detach_synchronously, false, "When executing DROP or DETACH TABLE in Atomic database, wait for table data to be finally dropped or detached.", 0) \ @@ -591,6 +590,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, use_local_cache_for_remote_storage, true, "Use local cache for remote storage like HDFS or S3, it's used for remote table engine only", 0) \ \ M(Bool, allow_unrestricted_reads_from_keeper, false, "Allow unrestricted (without condition on path) reads from system.zookeeper table, can be handy, but is not safe for zookeeper", 0) \ + M(Bool, allow_deprecated_database_ordinary, false, "Allow to create databases with deprecated Ordinary engine", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ @@ -637,6 +637,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) MAKE_OBSOLETE(M, UInt64, background_schedule_pool_size, 128) \ MAKE_OBSOLETE(M, UInt64, background_message_broker_schedule_pool_size, 16) \ MAKE_OBSOLETE(M, UInt64, background_distributed_schedule_pool_size, 16) \ + MAKE_OBSOLETE(M, DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic) \ /** The section above is for obsolete settings. Do not add anything there. */ diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ea887c84111..9e7d29d89b1 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -109,13 +109,13 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String & return table; } -void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool no_delay) +void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool sync) { auto table = tryGetTable(table_name, local_context); /// Remove the inner table (if any) to avoid deadlock /// (due to attempt to execute DROP from the worker thread) if (table) - table->dropInnerTableIfAny(no_delay, local_context); + table->dropInnerTableIfAny(sync, local_context); else throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuote(database_name), backQuote(table_name)); @@ -145,7 +145,7 @@ void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_na /// Notify DatabaseCatalog that table was dropped. It will remove table data in background. /// Cleanup is performed outside of database to allow easily DROP DATABASE without waiting for cleanup to complete. - DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay); + DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, sync); } void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_name, IDatabase & to_database, diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index b748e53244d..cba9593a601 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -35,7 +35,7 @@ public: bool exchange, bool dictionary) override; - void dropTable(ContextPtr context, const String & table_name, bool no_delay) override; + void dropTable(ContextPtr context, const String & table_name, bool sync) override; void attachTable(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path) override; StoragePtr detachTable(ContextPtr context, const String & name) override; diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 5cc334eaad4..182d9fe686b 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -138,8 +138,13 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have table overrides", engine_name); if (engine_name == "Ordinary") + { + if (!create.attach && !context->getSettingsRef().allow_deprecated_database_ordinary) + throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Ordinary database engine is deprecated"); return std::make_shared(database_name, metadata_path, context); - else if (engine_name == "Atomic") + } + + if (engine_name == "Atomic") return std::make_shared(database_name, metadata_path, uuid, context); else if (engine_name == "Memory") return std::make_shared(database_name, context); diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index b024c73d578..84e1a132669 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -77,10 +77,10 @@ void DatabaseLazy::createTable( void DatabaseLazy::dropTable( ContextPtr local_context, const String & table_name, - bool no_delay) + bool sync) { SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); }); - DatabaseOnDisk::dropTable(local_context, table_name, no_delay); + DatabaseOnDisk::dropTable(local_context, table_name, sync); } void DatabaseLazy::renameTable( diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 3a7d7b14be1..8d4799945fb 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -37,7 +37,7 @@ public: void dropTable( ContextPtr context, const String & table_name, - bool no_delay) override; + bool sync) override; void renameTable( ContextPtr context, diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 6df5b70c827..664bb015925 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -52,7 +52,7 @@ void DatabaseMemory::createTable( void DatabaseMemory::dropTable( ContextPtr /*context*/, const String & table_name, - bool /*no_delay*/) + bool /*sync*/) { std::unique_lock lock{mutex}; auto table = detachTableUnlocked(table_name, lock); diff --git a/src/Databases/DatabaseMemory.h b/src/Databases/DatabaseMemory.h index b854d9be1f3..6b22b5a4ec2 100644 --- a/src/Databases/DatabaseMemory.h +++ b/src/Databases/DatabaseMemory.h @@ -32,7 +32,7 @@ public: void dropTable( ContextPtr context, const String & table_name, - bool no_delay) override; + bool sync) override; ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override; ASTPtr getCreateDatabaseQuery() const override; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 9484da8ec2d..67aa712d557 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -281,7 +281,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri } } -void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_name, bool /*no_delay*/) +void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_name, bool /*sync*/) { String table_metadata_path = getObjectMetadataPath(table_name); String table_metadata_path_drop = table_metadata_path + drop_suffix; diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index a118c8da678..462b602a8b8 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -43,7 +43,7 @@ public: void dropTable( ContextPtr context, const String & table_name, - bool no_delay) override; + bool sync) override; void renameTable( ContextPtr context, diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 5c701c8d90c..0d20dfb2770 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -811,7 +811,7 @@ void DatabaseReplicated::shutdown() } -void DatabaseReplicated::dropTable(ContextPtr local_context, const String & table_name, bool no_delay) +void DatabaseReplicated::dropTable(ContextPtr local_context, const String & table_name, bool sync) { auto txn = local_context->getZooKeeperMetadataTransaction(); assert(!ddl_worker->isCurrentlyActive() || txn || startsWith(table_name, ".inner_id.")); @@ -820,7 +820,7 @@ void DatabaseReplicated::dropTable(ContextPtr local_context, const String & tabl String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name); txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1)); } - DatabaseAtomic::dropTable(local_context, table_name, no_delay); + DatabaseAtomic::dropTable(local_context, table_name, sync); } void DatabaseReplicated::renameTable(ContextPtr local_context, const String & table_name, IDatabase & to_database, diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 45a9d12981c..3aa2aa378b7 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -30,7 +30,7 @@ public: String getEngineName() const override { return "Replicated"; } /// If current query is initial, then the following methods add metadata updating ZooKeeper operations to current ZooKeeperMetadataTransaction. - void dropTable(ContextPtr, const String & table_name, bool no_delay) override; + void dropTable(ContextPtr, const String & table_name, bool sync) override; void renameTable(ContextPtr context, const String & table_name, IDatabase & to_database, const String & to_table_name, bool exchange, bool dictionary) override; void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 38c85cf3d05..4de64a2b24f 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -198,7 +198,7 @@ public: virtual void dropTable( /// NOLINT ContextPtr /*context*/, const String & /*name*/, - [[maybe_unused]] bool no_delay = false) + [[maybe_unused]] bool sync = false) { throw Exception("There is no DROP TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 13f55eab9e2..a81e07f0173 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -80,10 +80,10 @@ void DatabaseMaterializedMySQL::createTable(ContextPtr context_, const String & DatabaseAtomic::createTable(context_, name, table, query); } -void DatabaseMaterializedMySQL::dropTable(ContextPtr context_, const String & name, bool no_delay) +void DatabaseMaterializedMySQL::dropTable(ContextPtr context_, const String & name, bool sync) { checkIsInternalQuery(context_, "DROP TABLE"); - DatabaseAtomic::dropTable(context_, name, no_delay); + DatabaseAtomic::dropTable(context_, name, sync); } void DatabaseMaterializedMySQL::attachTable(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path) diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index 32686784f2a..a6810f29d87 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -52,7 +52,7 @@ public: void createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) override; - void dropTable(ContextPtr context_, const String & name, bool no_delay) override; + void dropTable(ContextPtr context_, const String & name, bool sync) override; void attachTable(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path) override; diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 58be682bd73..95098ba9cbd 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -447,7 +447,7 @@ void DatabaseMySQL::detachTablePermanently(ContextPtr, const String & table_name table_iter->second.second->is_dropped = true; } -void DatabaseMySQL::dropTable(ContextPtr local_context, const String & table_name, bool /*no_delay*/) +void DatabaseMySQL::dropTable(ContextPtr local_context, const String & table_name, bool /*sync*/) { detachTablePermanently(local_context, table_name); } diff --git a/src/Databases/MySQL/DatabaseMySQL.h b/src/Databases/MySQL/DatabaseMySQL.h index 1ee090ecd52..6a5105e2dba 100644 --- a/src/Databases/MySQL/DatabaseMySQL.h +++ b/src/Databases/MySQL/DatabaseMySQL.h @@ -82,7 +82,7 @@ public: void detachTablePermanently(ContextPtr context, const String & table_name) override; - void dropTable(ContextPtr context, const String & table_name, bool no_delay) override; + void dropTable(ContextPtr context, const String & table_name, bool sync) override; void attachTable(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 60d2fa0d2c8..8c4be49ef5c 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -389,10 +389,10 @@ void DatabaseMaterializedPostgreSQL::stopReplication() } -void DatabaseMaterializedPostgreSQL::dropTable(ContextPtr local_context, const String & table_name, bool no_delay) +void DatabaseMaterializedPostgreSQL::dropTable(ContextPtr local_context, const String & table_name, bool sync) { /// Modify context into nested_context and pass query to Atomic database. - DatabaseAtomic::dropTable(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context), table_name, no_delay); + DatabaseAtomic::dropTable(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context), table_name, sync); } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index 08420f4ba5e..ac2bcedca60 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -55,7 +55,7 @@ public: StoragePtr detachTable(ContextPtr context, const String & table_name) override; - void dropTable(ContextPtr local_context, const String & name, bool no_delay) override; + void dropTable(ContextPtr local_context, const String & name, bool sync) override; void drop(ContextPtr local_context) override; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 1bcc203beb9..fed4d1ee64d 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -264,7 +264,7 @@ void DatabasePostgreSQL::createTable(ContextPtr local_context, const String & ta } -void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool /* no_delay */) +void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool /* sync */) { std::lock_guard lock{mutex}; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 3397dcc8076..3d5e7bd444e 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -53,7 +53,7 @@ public: StoragePtr tryGetTable(const String & name, ContextPtr context) const override; void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; - void dropTable(ContextPtr, const String & table_name, bool no_delay) override; + void dropTable(ContextPtr, const String & table_name, bool sync) override; void attachTable(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; StoragePtr detachTable(ContextPtr context, const String & table_name) override; diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index db3cb3049fd..8a4e8dad8dd 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index b2a7aedaa5a..17d09b25ea1 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -16,7 +16,7 @@ #include #include #include -#include +#include #include #include @@ -339,7 +339,7 @@ public: /// We keep correspondence between used_flags and hash table internal buffer. /// Hash table cannot be modified during HashJoin lifetime and must be protected with lock. - void setLock(RWLockImpl::LockHolder rwlock_holder) + void setLock(TableLockHolder rwlock_holder) { storage_join_lock = rwlock_holder; } @@ -394,7 +394,7 @@ private: /// Should be set via setLock to protect hash table from modification from StorageJoin /// If set HashJoin instance is not available for modification (addJoinedBlock) - RWLockImpl::LockHolder storage_join_lock = nullptr; + TableLockHolder storage_join_lock = nullptr; void dataMapInit(MapsVariant &); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index b29f7372d38..7c5c6e15638 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -150,10 +150,9 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) /// When attaching old-style database during server startup, we must always use Ordinary engine if (create.attach) throw Exception("Database engine must be specified for ATTACH DATABASE query", ErrorCodes::UNKNOWN_DATABASE_ENGINE); - bool old_style_database = getContext()->getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary; auto engine = std::make_shared(); auto storage = std::make_shared(); - engine->name = old_style_database ? "Ordinary" : "Atomic"; + engine->name = "Atomic"; engine->no_empty_args = true; storage->set(storage->engine, engine); create.set(create.storage, storage); @@ -196,8 +195,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) if (create_from_user) { - const auto & default_engine = getContext()->getSettingsRef().default_database_engine.value; - if (create.uuid == UUIDHelpers::Nil && default_engine == DefaultDatabaseEngine::Atomic) + if (create.uuid == UUIDHelpers::Nil) create.uuid = UUIDHelpers::generateV4(); /// Will enable Atomic engine for nested database } else if (attach_from_user && create.uuid == UUIDHelpers::Nil) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 41b65e73efa..ac731ec6f4b 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -62,7 +62,7 @@ BlockIO InterpreterDropQuery::execute() } if (getContext()->getSettingsRef().database_atomic_wait_for_drop_and_detach_synchronously) - drop.no_delay = true; + drop.sync = true; if (drop.table) return executeToTable(drop); @@ -89,7 +89,7 @@ BlockIO InterpreterDropQuery::executeToTable(ASTDropQuery & query) DatabasePtr database; UUID table_to_wait_on = UUIDHelpers::Nil; auto res = executeToTableImpl(getContext(), query, database, table_to_wait_on); - if (query.no_delay) + if (query.sync) waitForTableToBeActuallyDroppedOrDetached(query, database, table_to_wait_on); return res; } @@ -244,7 +244,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue DatabaseCatalog::instance().tryRemoveLoadingDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, is_drop_or_detach_database); - database->dropTable(context_, table_id.table_name, query.no_delay); + database->dropTable(context_, table_id.table_name, query.sync); } db = database; @@ -300,7 +300,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(const ASTDropQuery & query) } catch (...) { - if (query.no_delay) + if (query.sync) { for (const auto & table_uuid : tables_to_wait) waitForTableToBeActuallyDroppedOrDetached(query, database, table_uuid); @@ -308,7 +308,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(const ASTDropQuery & query) throw; } - if (query.no_delay) + if (query.sync) { for (const auto & table_uuid : tables_to_wait) waitForTableToBeActuallyDroppedOrDetached(query, database, table_uuid); @@ -345,7 +345,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, query_for_table.kind = query.kind; query_for_table.if_exists = true; query_for_table.setDatabase(database_name); - query_for_table.no_delay = query.no_delay; + query_for_table.sync = query.sync; /// Flush should not be done if shouldBeEmptyOnDetach() == false, /// since in this case getTablesIterator() may do some additional work, @@ -368,7 +368,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, } } - if (!drop && query.no_delay) + if (!drop && query.sync) { /// Avoid "some tables are still in use" when sync mode is enabled for (const auto & table_uuid : uuids_to_wait) @@ -428,7 +428,7 @@ void InterpreterDropQuery::extendQueryLogElemImpl(QueryLogElement & elem, const elem.query_kind = "Drop"; } -void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool no_delay) +void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool sync) { if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context)) { @@ -437,7 +437,7 @@ void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr drop_query->setDatabase(target_table_id.database_name); drop_query->setTable(target_table_id.table_name); drop_query->kind = kind; - drop_query->no_delay = no_delay; + drop_query->sync = sync; drop_query->if_exists = true; ASTPtr ast_drop_query = drop_query; /// FIXME We have to use global context to execute DROP query for inner table diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index 1a38abcdff9..2b65039954b 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -26,7 +26,7 @@ public: void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const override; - static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool no_delay); + static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool sync); private: AccessRightsElements getRequiredAccessForDDLOnCluster() const; diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index de920eaddbf..f6cfbe452c6 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -18,7 +18,6 @@ #include #include -#include #include #include @@ -71,12 +70,6 @@ static void loadDatabase( ReadBufferFromFile in(database_metadata_file, 1024); readStringUntilEOF(database_attach_query, in); } - else if (fs::exists(fs::path(database_path))) - { - /// TODO Remove this code (it's required for compatibility with versions older than 20.7) - /// Database exists, but .sql file is absent. It's old-style Ordinary database (e.g. system or default) - database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database) + " ENGINE = Ordinary"; - } else { /// It's first server run and we need create default and system databases. diff --git a/src/Parsers/ASTDropQuery.cpp b/src/Parsers/ASTDropQuery.cpp index 9e815ee75de..11c1dd4c47a 100644 --- a/src/Parsers/ASTDropQuery.cpp +++ b/src/Parsers/ASTDropQuery.cpp @@ -72,8 +72,8 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState if (permanently) settings.ostr << " PERMANENTLY"; - if (no_delay) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " NO DELAY" << (settings.hilite ? hilite_none : ""); + if (sync) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " SYNC" << (settings.hilite ? hilite_none : ""); } } diff --git a/src/Parsers/ASTDropQuery.h b/src/Parsers/ASTDropQuery.h index ef2b609fbac..b4c96353a09 100644 --- a/src/Parsers/ASTDropQuery.h +++ b/src/Parsers/ASTDropQuery.h @@ -31,7 +31,7 @@ public: /// Same as above bool is_view{false}; - bool no_delay{false}; + bool sync{false}; // We detach the object permanently, so it will not be reattached back during server restart. bool permanently{false}; diff --git a/src/Parsers/ParserDropQuery.cpp b/src/Parsers/ParserDropQuery.cpp index e7263aa47f5..f40a39e6b2f 100644 --- a/src/Parsers/ParserDropQuery.cpp +++ b/src/Parsers/ParserDropQuery.cpp @@ -31,7 +31,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons bool temporary = false; bool is_dictionary = false; bool is_view = false; - bool no_delay = false; + bool sync = false; bool permanently = false; if (s_database.ignore(pos, expected)) @@ -83,7 +83,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons /// actually for TRUNCATE NO DELAY / SYNC means nothing if (s_no_delay.ignore(pos, expected) || s_sync.ignore(pos, expected)) - no_delay = true; + sync = true; auto query = std::make_shared(); node = query; @@ -93,7 +93,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons query->temporary = temporary; query->is_dictionary = is_dictionary; query->is_view = is_view; - query->no_delay = no_delay; + query->sync = sync; query->permanently = permanently; query->database = database; query->table = table; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index a655da4473b..6dd329db02b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -396,7 +396,7 @@ public: */ virtual void drop() {} - virtual void dropInnerTableIfAny(bool /* no_delay */, ContextPtr /* context */) {} + virtual void dropInnerTableIfAny(bool /* sync */, ContextPtr /* context */) {} /** Clear the table data and leave it empty. * Must be called under exclusive lock (lockExclusively). diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 190ffabe2c1..cc80d567d1d 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -240,7 +240,7 @@ void StorageMaterializedPostgreSQL::shutdown() } -void StorageMaterializedPostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPtr local_context) +void StorageMaterializedPostgreSQL::dropInnerTableIfAny(bool sync, ContextPtr local_context) { /// If it is a table with database engine MaterializedPostgreSQL - return, because delition of /// internal tables is managed there. @@ -252,7 +252,7 @@ void StorageMaterializedPostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPt auto nested_table = tryGetNested() != nullptr; if (nested_table) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, getNestedStorageID(), no_delay); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, getNestedStorageID(), sync); } diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index bb3836a8853..f1eea33d4b0 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -84,7 +84,7 @@ public: void shutdown() override; /// Used only for single MaterializedPostgreSQL storage. - void dropInnerTableIfAny(bool no_delay, ContextPtr local_context) override; + void dropInnerTableIfAny(bool sync, ContextPtr local_context) override; NamesAndTypesList getVirtuals() const override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index d0685c263f8..2ece0af3359 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -215,10 +215,10 @@ void StorageMaterializedView::drop() dropInnerTableIfAny(true, getContext()); } -void StorageMaterializedView::dropInnerTableIfAny(bool no_delay, ContextPtr local_context) +void StorageMaterializedView::dropInnerTableIfAny(bool sync, ContextPtr local_context) { if (has_inner_table && tryGetTargetTable()) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, no_delay); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, sync); } void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 8aec0313ecb..0adf394876c 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -42,7 +42,7 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; void drop() override; - void dropInnerTableIfAny(bool no_delay, ContextPtr local_context) override; + void dropInnerTableIfAny(bool sync, ContextPtr local_context) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index cfb19869074..d9780eb8446 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1606,7 +1606,7 @@ void StorageWindowView::drop() dropInnerTableIfAny(true, getContext()); } -void StorageWindowView::dropInnerTableIfAny(bool no_delay, ContextPtr local_context) +void StorageWindowView::dropInnerTableIfAny(bool sync, ContextPtr local_context) { if (!std::exchange(has_inner_table, false)) return; @@ -1614,10 +1614,10 @@ void StorageWindowView::dropInnerTableIfAny(bool no_delay, ContextPtr local_cont try { InterpreterDropQuery::executeDropQuery( - ASTDropQuery::Kind::Drop, getContext(), local_context, inner_table_id, no_delay); + ASTDropQuery::Kind::Drop, getContext(), local_context, inner_table_id, sync); if (has_inner_target_table) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, no_delay); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, sync); } catch (...) { diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 86cc80ee8ea..96c034b9590 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -120,7 +120,7 @@ public: void checkTableCanBeDropped() const override; - void dropInnerTableIfAny(bool no_delay, ContextPtr context) override; + void dropInnerTableIfAny(bool sync, ContextPtr context) override; void drop() override; diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 3e0d4e822b4..189bb1e687f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1489,9 +1489,9 @@ def collect_build_flags(args): result.append(BuildFlags.RELEASE) value = clickhouse_execute( - args, "SELECT value FROM system.settings WHERE name = 'default_database_engine'" + args, "SELECT value FROM system.settings WHERE name = 'allow_deprecated_database_ordinary'" ) - if value == b"Ordinary": + if value == b"1": result.append(BuildFlags.ORDINARY_DATABASE) value = int( diff --git a/tests/config/users.d/database_ordinary.xml b/tests/config/users.d/database_ordinary.xml index b3b81ee25ff..8ffd2f27a62 100644 --- a/tests/config/users.d/database_ordinary.xml +++ b/tests/config/users.d/database_ordinary.xml @@ -1,7 +1,7 @@ - Ordinary + 1 diff --git a/tests/integration/test_materialized_mysql_database/configs/users.xml b/tests/integration/test_materialized_mysql_database/configs/users.xml index 4b7f5a1b109..0e116f115fe 100644 --- a/tests/integration/test_materialized_mysql_database/configs/users.xml +++ b/tests/integration/test_materialized_mysql_database/configs/users.xml @@ -3,7 +3,6 @@ 1 - Atomic 1 0 diff --git a/tests/integration/test_materialized_mysql_database/configs/users_disable_bytes_settings.xml b/tests/integration/test_materialized_mysql_database/configs/users_disable_bytes_settings.xml index f590ebff6b4..a00b6ca6b9a 100644 --- a/tests/integration/test_materialized_mysql_database/configs/users_disable_bytes_settings.xml +++ b/tests/integration/test_materialized_mysql_database/configs/users_disable_bytes_settings.xml @@ -3,7 +3,6 @@ 1 - Atomic 1 0 diff --git a/tests/integration/test_materialized_mysql_database/configs/users_disable_rows_settings.xml b/tests/integration/test_materialized_mysql_database/configs/users_disable_rows_settings.xml index e0fa0a9097b..3a7cc2537e5 100644 --- a/tests/integration/test_materialized_mysql_database/configs/users_disable_rows_settings.xml +++ b/tests/integration/test_materialized_mysql_database/configs/users_disable_rows_settings.xml @@ -3,7 +3,6 @@ 1 - Atomic 0 1 diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql index 8b978e6094c..2d9911287a3 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql @@ -2,6 +2,7 @@ SET allow_experimental_window_view = 1; DROP DATABASE IF EXISTS test_01047; +set allow_deprecated_database_ordinary=1; CREATE DATABASE test_01047 ENGINE=Ordinary; DROP TABLE IF EXISTS test_01047.mt; diff --git a/tests/queries/0_stateless/01048_window_view_parser.sql b/tests/queries/0_stateless/01048_window_view_parser.sql index 95190ddafa1..4c329f99f6e 100644 --- a/tests/queries/0_stateless/01048_window_view_parser.sql +++ b/tests/queries/0_stateless/01048_window_view_parser.sql @@ -2,6 +2,7 @@ SET allow_experimental_window_view = 1; DROP DATABASE IF EXISTS test_01048; +set allow_deprecated_database_ordinary=1; CREATE DATABASE test_01048 ENGINE=Ordinary; DROP TABLE IF EXISTS test_01048.mt; diff --git a/tests/queries/0_stateless/01053_drop_database_mat_view.sql b/tests/queries/0_stateless/01053_drop_database_mat_view.sql index 67a488f7245..e9936d7d3b2 100644 --- a/tests/queries/0_stateless/01053_drop_database_mat_view.sql +++ b/tests/queries/0_stateless/01053_drop_database_mat_view.sql @@ -1,6 +1,7 @@ -- Tags: no-parallel DROP DATABASE IF EXISTS some_tests; +set allow_deprecated_database_ordinary=1; CREATE DATABASE some_tests ENGINE=Ordinary; -- Different inner table name with Atomic create table some_tests.my_table ENGINE = MergeTree(day, (day), 8192) as select today() as day, 'mystring' as str; diff --git a/tests/queries/0_stateless/01085_window_view_attach.sql b/tests/queries/0_stateless/01085_window_view_attach.sql index 604bf5dd198..bb47e0dc6b9 100644 --- a/tests/queries/0_stateless/01085_window_view_attach.sql +++ b/tests/queries/0_stateless/01085_window_view_attach.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP DATABASE IF EXISTS test_01085; +set allow_deprecated_database_ordinary=1; CREATE DATABASE test_01085 ENGINE=Ordinary; DROP TABLE IF EXISTS test_01085.mt; diff --git a/tests/queries/0_stateless/01086_window_view_cleanup.sh b/tests/queries/0_stateless/01086_window_view_cleanup.sh index a7b976bf4e0..c85455616e1 100755 --- a/tests/queries/0_stateless/01086_window_view_cleanup.sh +++ b/tests/queries/0_stateless/01086_window_view_cleanup.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery <&1| grep -Ev "Removing leftovers from table|removed by another replica" $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" if [[ $engine == *"ReplicatedMergeTree"* ]]; then - $CLICKHOUSE_CLIENT --enable_positional_arguments 0 -q "ALTER TABLE test - UPDATE test = (SELECT groupArray(id) FROM t1)[n - 99] WHERE 1" 2>&1| grep -Fa "DB::Exception: " | grep -Fv "statement with subquery may be nondeterministic" - $CLICKHOUSE_CLIENT --enable_positional_arguments 0 --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "ALTER TABLE test + $CLICKHOUSE_CLIENT 0 -q "ALTER TABLE test + UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 'dummy')[n - 99] WHERE 1" 2>&1| grep -Fa "DB::Exception: " | grep -Fv "statement with subquery may be nondeterministic" + $CLICKHOUSE_CLIENT --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1)[n - 99] WHERE 1" elif [[ $engine == *"Join"* ]]; then - $CLICKHOUSE_CLIENT --enable_positional_arguments 0 -q "ALTER TABLE test + $CLICKHOUSE_CLIENT -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1)[n - 99] WHERE 1" 2>&1| grep -Fa "DB::Exception: " | grep -Fv "Table engine Join supports only DELETE mutations" else - $CLICKHOUSE_CLIENT --enable_positional_arguments 0 --mutations_sync=1 -q "ALTER TABLE test + $CLICKHOUSE_CLIENT --mutations_sync=1 -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1)[n - 99] WHERE 1" fi $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.sql b/tests/queries/0_stateless/01190_full_attach_syntax.sql index ed05950ff98..e0ffe7ede66 100644 --- a/tests/queries/0_stateless/01190_full_attach_syntax.sql +++ b/tests/queries/0_stateless/01190_full_attach_syntax.sql @@ -1,6 +1,7 @@ -- Tags: no-parallel DROP DATABASE IF EXISTS test_01190; +set allow_deprecated_database_ordinary=1; CREATE DATABASE test_01190 ENGINE=Ordinary; -- Full ATTACH requires UUID with Atomic USE test_01190; diff --git a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh index c6686bde672..641e26870d4 100755 --- a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh +++ b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh @@ -11,8 +11,8 @@ $CLICKHOUSE_CLIENT --check_table_dependencies=0 -q "DROP DATABASE IF EXISTS test $CLICKHOUSE_CLIENT --check_table_dependencies=0 -q "DROP DATABASE IF EXISTS test_01192_renamed" $CLICKHOUSE_CLIENT --check_table_dependencies=0 -q "DROP DATABASE IF EXISTS test_01192_atomic" -$CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'" 2>&1| grep -F "does not support" > /dev/null && echo "ok" -$CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'" +$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001' ENGINE=Ordinary" 2>&1| grep -F "does not support" > /dev/null && echo "ok" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'" # 2. check metadata $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE DATABASE test_01192" @@ -35,14 +35,14 @@ $CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE test_01192_renamed" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01192_renamed.mt" # 5. check moving tables from Ordinary to Atomic (can be used to "alter" database engine) -$CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01192" +$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -q "CREATE DATABASE test_01192 ENGINE=Ordinary" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01192.mt AS test_01192_renamed.mt ENGINE=MergeTree ORDER BY n" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01192.rmt AS test_01192_renamed.mt ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '1') ORDER BY n" $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW test_01192.mv TO test_01192.rmt AS SELECT * FROM test_01192.mt" $CLICKHOUSE_CLIENT -q "INSERT INTO test_01192.mt SELECT number FROM numbers(10)" && echo "inserted" -$CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01192_atomic" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01192_atomic" $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01192_renamed" # it's blocking $CLICKHOUSE_CLIENT -q "RENAME TABLE test_01192.mt TO test_01192_atomic.mt, test_01192.rmt TO test_01192_atomic.rmt, test_01192.mv TO test_01192_atomic.mv" && echo "renamed" diff --git a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql index 6f5deb91ee4..5db92e70650 100644 --- a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql +++ b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql @@ -2,6 +2,7 @@ DROP DATABASE IF EXISTS dict_db_01224; DROP DATABASE IF EXISTS dict_db_01224_dictionary; +set allow_deprecated_database_ordinary=1; CREATE DATABASE dict_db_01224 ENGINE=Ordinary; -- Different internal dictionary name with Atomic CREATE DATABASE dict_db_01224_dictionary Engine=Dictionary; diff --git a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql index 006ff952ee9..09cde642ed2 100644 --- a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql +++ b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql @@ -2,6 +2,7 @@ DROP DATABASE IF EXISTS dict_db_01225; DROP DATABASE IF EXISTS dict_db_01225_dictionary; +set allow_deprecated_database_ordinary=1; CREATE DATABASE dict_db_01225 ENGINE=Ordinary; -- Different internal dictionary name with Atomic CREATE DATABASE dict_db_01225_dictionary Engine=Dictionary; diff --git a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql index c2be4d04e5f..d187a2e4d4e 100644 --- a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql +++ b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql @@ -1,6 +1,7 @@ -- Tags: no-parallel DROP DATABASE IF EXISTS test_01249; +set allow_deprecated_database_ordinary=1; CREATE DATABASE test_01249 ENGINE=Ordinary; -- Full ATTACH requires UUID with Atomic USE test_01249; diff --git a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh index 758ec4825e0..ef45e8e63bc 100755 --- a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh +++ b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e $CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS test_01320" -$CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01320 ENGINE=Ordinary" # Different bahaviour of DROP with Atomic +$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 --query "CREATE DATABASE test_01320 ENGINE=Ordinary" # Different bahaviour of DROP with Atomic function thread1() { diff --git a/tests/queries/0_stateless/01516_create_table_primary_key.sql b/tests/queries/0_stateless/01516_create_table_primary_key.sql index b3c4acd50ff..b2b9f288eab 100644 --- a/tests/queries/0_stateless/01516_create_table_primary_key.sql +++ b/tests/queries/0_stateless/01516_create_table_primary_key.sql @@ -1,6 +1,7 @@ -- Tags: no-parallel DROP DATABASE IF EXISTS test_01516; +set allow_deprecated_database_ordinary=1; CREATE DATABASE test_01516 ENGINE=Ordinary; -- Full ATTACH requires UUID with Atomic USE test_01516; diff --git a/tests/queries/0_stateless/01517_drop_mv_with_inner_table.sql b/tests/queries/0_stateless/01517_drop_mv_with_inner_table.sql index 5e31faa70da..67a2009b913 100644 --- a/tests/queries/0_stateless/01517_drop_mv_with_inner_table.sql +++ b/tests/queries/0_stateless/01517_drop_mv_with_inner_table.sql @@ -32,6 +32,7 @@ show tables from db_01517_atomic_sync; -- Ordinary --- drop database if exists db_01517_ordinary; +set allow_deprecated_database_ordinary=1; create database db_01517_ordinary Engine=Ordinary; create table db_01517_ordinary.source (key Int) engine=Null; diff --git a/tests/queries/0_stateless/01600_detach_permanently.sh b/tests/queries/0_stateless/01600_detach_permanently.sh index 949fe0e6ea2..c32a255448e 100755 --- a/tests/queries/0_stateless/01600_detach_permanently.sh +++ b/tests/queries/0_stateless/01600_detach_permanently.sh @@ -15,7 +15,7 @@ mkdir -p "${WORKING_FOLDER_01600}" clickhouse_local() { local query="$1" shift - ${CLICKHOUSE_LOCAL} --query "$query" "$@" --path="${WORKING_FOLDER_01600}" + ${CLICKHOUSE_LOCAL} --allow_deprecated_database_ordinary=1 --query "$query" "$@" --path="${WORKING_FOLDER_01600}" } test_detach_attach_sequence() { diff --git a/tests/queries/0_stateless/01601_detach_permanently.sql b/tests/queries/0_stateless/01601_detach_permanently.sql index 97797a59af5..95c80e77213 100644 --- a/tests/queries/0_stateless/01601_detach_permanently.sql +++ b/tests/queries/0_stateless/01601_detach_permanently.sql @@ -72,6 +72,7 @@ SELECT '-----------------------'; SELECT 'database ordinary tests'; DROP DATABASE IF EXISTS test1601_detach_permanently_ordinary; +set allow_deprecated_database_ordinary=1; CREATE DATABASE test1601_detach_permanently_ordinary Engine=Ordinary; create table test1601_detach_permanently_ordinary.test_name_reuse (number UInt64) engine=MergeTree order by tuple(); diff --git a/tests/queries/0_stateless/01603_rename_overwrite_bug.sql b/tests/queries/0_stateless/01603_rename_overwrite_bug.sql index 82f9996991f..acf9f520709 100644 --- a/tests/queries/0_stateless/01603_rename_overwrite_bug.sql +++ b/tests/queries/0_stateless/01603_rename_overwrite_bug.sql @@ -1,6 +1,7 @@ -- Tags: no-parallel DROP database IF EXISTS test_1603_rename_bug_ordinary; +set allow_deprecated_database_ordinary=1; create database test_1603_rename_bug_ordinary engine=Ordinary; create table test_1603_rename_bug_ordinary.foo engine=Memory as select * from numbers(100); create table test_1603_rename_bug_ordinary.bar engine=Log as select * from numbers(200); diff --git a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh index f5ab71d8d34..b1f30a41924 100755 --- a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh +++ b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh @@ -10,7 +10,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q "create database ordinary_$CLICKHOUSE_DATABASE engine=Ordinary" +$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -nm -q "create database ordinary_$CLICKHOUSE_DATABASE engine=Ordinary" # MergeTree $CLICKHOUSE_CLIENT -nm -q """ diff --git a/tests/queries/0_stateless/02096_rename_atomic_hang.sql b/tests/queries/0_stateless/02096_rename_atomic_hang.sql index 96261bfe127..dec5f3f9506 100644 --- a/tests/queries/0_stateless/02096_rename_atomic_hang.sql +++ b/tests/queries/0_stateless/02096_rename_atomic_hang.sql @@ -2,6 +2,7 @@ drop database if exists db_hang; drop database if exists db_hang_temp; +set allow_deprecated_database_ordinary=1; create database db_hang engine=Ordinary; use db_hang; create table db_hang.test(A Int64) Engine=MergeTree order by A; From f00e6b5a7a50bc0dc616b5df81034e6cd27e03d2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Jun 2022 10:37:52 +0200 Subject: [PATCH 12/72] deprecate old MergeTree syntax --- src/Core/Settings.h | 1 + .../MergeTree/registerStorageMergeTree.cpp | 5 ++++ .../test_cluster_copier/configs/users.xml | 1 + .../configs_three_nodes/users.xml | 1 + .../configs_two_nodes/users.xml | 1 + .../configs/config_allow_databases.xml | 5 ++++ .../test_cross_replication/test.py | 2 +- .../test_delayed_replica_failover/test.py | 2 +- .../configs/users.d/query_log.xml | 1 + .../configs_secure/users.d/query_log.xml | 1 + .../test_extreme_deduplication/test.py | 4 ++-- .../test_https_replication/test.py | 2 +- .../test_insert_into_distributed/test.py | 24 +++++++++---------- .../test.py | 2 +- .../enable_distributed_inserts_batching.xml | 1 + .../test_merge_tree_empty_parts/test.py | 4 ++-- .../test_mutations_with_merge_tree/test.py | 2 +- tests/integration/test_random_inserts/test.py | 2 +- .../test_replace_partition/test.py | 6 ++--- .../test_replicated_database/test.py | 4 ++-- .../test_replication_credentials/test.py | 2 +- .../test.py | 4 ++-- tests/integration/test_union_header/test.py | 2 +- .../integration/test_zookeeper_config/test.py | 4 ++-- .../test_zookeeper_config/test_password.py | 4 ++-- tests/performance/parallel_final.xml | 1 + tests/performance/scalar2.xml | 4 ++++ .../queries/0_stateless/00030_alter_table.sql | 1 + .../0_stateless/00043_summing_empty_part.sql | 1 + .../00046_stored_aggregates_simple.sql | 1 + .../00047_stored_aggregates_complex.sql | 1 + .../00048_a_stored_aggregates_merge.sql | 1 + .../00048_b_stored_aggregates_merge.sql | 1 + .../0_stateless/00061_merge_tree_alter.sql | 1 + ...icated_merge_tree_alter_zookeeper_long.sql | 1 + .../0_stateless/00079_defaulted_columns.sql | 1 + .../0_stateless/00084_summing_merge_tree.sql | 1 + .../0_stateless/00098_shard_i_union_all.sql | 1 + ...s_and_insert_without_explicit_database.sql | 1 + .../00121_drop_column_zookeeper.sql | 1 + ...4_shard_distributed_with_many_replicas.sql | 1 + .../00140_prewhere_column_order.sql | 1 + .../00141_parse_timestamp_as_datetime.sql | 1 + .../00146_summing_merge_tree_nested_map.sql | 1 + .../00147_alter_nested_default.sql | 1 + ..._summing_merge_tree_aggregate_function.sql | 1 + ..._merge_tree_nested_map_multiple_values.sql | 1 + .../queries/0_stateless/00155_long_merges.sh | 4 ++-- .../0_stateless/00168_buffer_defaults.sql | 1 + ...00191_aggregating_merge_tree_and_final.sql | 1 + .../0_stateless/00193_parallel_replicas.sql | 1 + ...ated_drop_on_non_leader_zookeeper_long.sql | 1 + .../00253_insert_recursive_defaults.sql | 1 + .../00278_insert_already_sorted.sql | 1 + tests/queries/0_stateless/00282_merging.sql | 1 + .../00345_index_accurate_comparison.sql | 1 + ..._column_aggregate_function_insert_from.sql | 1 + .../00394_new_nested_column_keeps_offsets.sql | 1 + .../00394_replaceall_vector_fixed.sql | 1 + tests/queries/0_stateless/00395_nullable.sql | 1 + .../0_stateless/00427_alter_primary_key.sh | 1 + .../0_stateless/00440_nulls_merge_tree.sql | 1 + ...olumn_in_partition_concurrent_zookeeper.sh | 2 +- tests/queries/0_stateless/00453_cast_enum.sql | 1 + .../00495_reading_const_zero_column.sql | 1 + .../0_stateless/00504_mergetree_arrays_rw.sql | 1 + .../0_stateless/00506_union_distributed.sql | 1 + ...rd_desc_table_functions_and_subqueries.sql | 1 + ...ication_after_drop_partition_zookeeper.sql | 1 + .../00531_aggregate_over_nullable.sql | 1 + .../0_stateless/00543_null_and_prewhere.sql | 2 ++ .../0_stateless/00561_storage_join.sql | 1 + .../00563_complex_in_expression.sql | 1 + ..._column_values_with_default_expression.sql | 1 + ...t_database_when_create_materializ_view.sql | 1 + ...column_exception_when_drop_depen_column.sh | 2 +- ...75_merge_and_index_with_function_in_in.sql | 1 + .../0_stateless/00584_view_union_all.sql | 1 + .../00594_alias_in_distributed.sql | 1 + .../00597_push_down_predicate_long.sql | 1 + .../0_stateless/00609_mv_index_in_in.sql | 1 + ...iew_forward_alter_partition_statements.sql | 1 + ...l_and_remote_node_in_distributed_query.sql | 1 + .../00615_nullable_alter_optimize.sql | 1 + .../00621_regression_for_in_operator.sql | 1 + .../0_stateless/00623_in_partition_key.sql | 1 + ...plicated_truncate_table_zookeeper_long.sql | 1 + ...0648_replacing_empty_set_from_prewhere.sql | 1 + .../0_stateless/00652_mergetree_mutations.sh | 2 +- .../00652_replicated_mutations_zookeeper.sh | 4 ++-- .../0_stateless/00678_shard_funnel_window.sql | 1 + .../00712_prewhere_with_alias_bug_2.sql | 1 + .../0_stateless/00712_prewhere_with_final.sql | 1 + ...00712_prewhere_with_sampling_and_alias.sql | 1 + .../00717_merge_and_distributed.sql | 1 + .../0_stateless/00729_prewhere_array_join.sql | 1 + ...731_long_merge_tree_select_opened_files.sh | 2 +- .../00732_decimal_summing_merge_tree.sql | 1 + .../00748_insert_array_with_null.sql | 1 + .../00752_low_cardinality_mv_2.sql | 1 + ..._system_columns_and_system_tables_long.sql | 1 + .../00754_alter_modify_order_by.sql | 1 + ...ify_order_by_replicated_zookeeper_long.sql | 1 + ...materialized_view_with_column_defaults.sql | 1 + ...800_low_cardinality_distributed_insert.sql | 1 + .../0_stateless/00806_alter_update.sql | 1 + .../0_stateless/00829_bitmap_function.sql | 1 + .../00834_hints_for_type_function_typos.sh | 4 ++-- ...crash_when_distributed_modify_order_by.sql | 1 + ...plicated_merge_tree_optimize_final_long.sh | 4 ++-- ...o_distributed_with_materialized_column.sql | 1 + .../00981_in_subquery_with_tuple.sh | 2 +- ...1008_materialized_view_henyihanwobushi.sql | 1 + .../01053_drop_database_mat_view.sql | 1 + ...1062_alter_on_mutataion_zookeeper_long.sql | 1 + ...ndary_index_with_old_format_merge_tree.sql | 1 + .../01076_predicate_optimizer_with_view.sql | 1 + .../01089_alter_settings_old_format.sql | 1 + .../01101_prewhere_after_alter.sql | 1 + .../01125_generate_random_qoega.sql | 1 + ...126_month_partitioning_consistent_code.sql | 1 + ..._month_partitioning_consistency_select.sql | 1 + ...ery_in_aggregate_function_JustStranger.sql | 1 + .../01355_alter_column_with_order.sql | 1 + .../01387_clear_column_default_depends.sql | 1 + .../01417_freeze_partition_verbose.sh | 2 +- .../01430_modify_sample_by_zookeeper_long.sql | 1 + ...1455_shard_leaf_max_rows_bytes_to_read.sql | 1 + .../01648_mutations_and_escaping.sql | 1 + .../01713_table_ttl_old_syntax_zookeeper.sql | 1 + ...778_test_LowCardinality_FixedString_pk.sql | 1 + .../0_stateless/01798_uniq_theta_sketch.sql | 1 + .../01925_broken_partition_id_zookeeper.sql | 1 + .../0_stateless/02250_hints_for_columns.sh | 2 +- .../00040_aggregating_materialized_view.sql | 1 + .../00041_aggregating_materialized_view.sql | 1 + .../00054_merge_tree_partitions.sql | 1 + .../00071_merge_tree_optimize_aio.sql | 1 + 138 files changed, 173 insertions(+), 50 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 883281c94f3..1ee46ed4caa 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -591,6 +591,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(Bool, allow_unrestricted_reads_from_keeper, false, "Allow unrestricted (without condition on path) reads from system.zookeeper table, can be handy, but is not safe for zookeeper", 0) \ M(Bool, allow_deprecated_database_ordinary, false, "Allow to create databases with deprecated Ordinary engine", 0) \ + M(Bool, allow_deprecated_syntax_for_merge_tree, false, "Allow to create *MergeTree tables with deprecated engine definition syntax", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 43e1af21eac..5ad39aa826a 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -303,6 +303,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) arg_idx, e.message(), getMergeTreeVerboseHelp(is_extended_storage_def)); } } + else if (!args.attach && !args.local_context.lock()->getSettingsRef().allow_deprecated_syntax_for_merge_tree) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "This syntax for *MergeTree engine is deprecated. " + "Use extended storage definition syntax with ORDER BY/PRIMARY KEY clause."); + } /// For Replicated. String zookeeper_path; diff --git a/tests/integration/test_cluster_copier/configs/users.xml b/tests/integration/test_cluster_copier/configs/users.xml index 492cf4d7ee6..2542642f6df 100644 --- a/tests/integration/test_cluster_copier/configs/users.xml +++ b/tests/integration/test_cluster_copier/configs/users.xml @@ -5,6 +5,7 @@ 1 5 + 1 diff --git a/tests/integration/test_cluster_copier/configs_three_nodes/users.xml b/tests/integration/test_cluster_copier/configs_three_nodes/users.xml index ce3538a31b8..f017daff974 100644 --- a/tests/integration/test_cluster_copier/configs_three_nodes/users.xml +++ b/tests/integration/test_cluster_copier/configs_three_nodes/users.xml @@ -3,6 +3,7 @@ 1 + 1 diff --git a/tests/integration/test_cluster_copier/configs_two_nodes/users.xml b/tests/integration/test_cluster_copier/configs_two_nodes/users.xml index ce3538a31b8..f017daff974 100644 --- a/tests/integration/test_cluster_copier/configs_two_nodes/users.xml +++ b/tests/integration/test_cluster_copier/configs_two_nodes/users.xml @@ -3,6 +3,7 @@ 1 + 1 diff --git a/tests/integration/test_config_substitutions/configs/config_allow_databases.xml b/tests/integration/test_config_substitutions/configs/config_allow_databases.xml index 98008306787..be727360dcf 100644 --- a/tests/integration/test_config_substitutions/configs/config_allow_databases.xml +++ b/tests/integration/test_config_substitutions/configs/config_allow_databases.xml @@ -1,4 +1,9 @@ + + + 1 + + diff --git a/tests/integration/test_cross_replication/test.py b/tests/integration/test_cross_replication/test.py index 143b8823bf2..2a73acadafd 100644 --- a/tests/integration/test_cross_replication/test.py +++ b/tests/integration/test_cross_replication/test.py @@ -37,7 +37,7 @@ def started_cluster(): CREATE DATABASE shard_{shard}; CREATE TABLE shard_{shard}.replicated(date Date, id UInt32, shard_id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated', '{replica}', date, id, 8192); + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; """.format( shard=shard, replica=node.name ) diff --git a/tests/integration/test_delayed_replica_failover/test.py b/tests/integration/test_delayed_replica_failover/test.py index 387d6a12f48..a480ee3f278 100644 --- a/tests/integration/test_delayed_replica_failover/test.py +++ b/tests/integration/test_delayed_replica_failover/test.py @@ -32,7 +32,7 @@ def started_cluster(): node.query( """ CREATE TABLE replicated (d Date, x UInt32) ENGINE = - ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated', '{instance}', d, d, 8192)""".format( + ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated', '{instance}') PARTITION BY toYYYYMM(d) ORDER BY d""".format( shard=shard, instance=node.name ) ) diff --git a/tests/integration/test_distributed_ddl/configs/users.d/query_log.xml b/tests/integration/test_distributed_ddl/configs/users.d/query_log.xml index 26db7f54514..ef8abbd9174 100644 --- a/tests/integration/test_distributed_ddl/configs/users.d/query_log.xml +++ b/tests/integration/test_distributed_ddl/configs/users.d/query_log.xml @@ -3,6 +3,7 @@ 1 + 1 diff --git a/tests/integration/test_distributed_ddl/configs_secure/users.d/query_log.xml b/tests/integration/test_distributed_ddl/configs_secure/users.d/query_log.xml index 26db7f54514..ef8abbd9174 100644 --- a/tests/integration/test_distributed_ddl/configs_secure/users.d/query_log.xml +++ b/tests/integration/test_distributed_ddl/configs_secure/users.d/query_log.xml @@ -3,6 +3,7 @@ 1 + 1 diff --git a/tests/integration/test_extreme_deduplication/test.py b/tests/integration/test_extreme_deduplication/test.py index 2c8772aad4e..71f783d37c9 100644 --- a/tests/integration/test_extreme_deduplication/test.py +++ b/tests/integration/test_extreme_deduplication/test.py @@ -40,7 +40,7 @@ def test_deduplication_window_in_seconds(started_cluster): node1.query( """ CREATE TABLE simple ON CLUSTER test_cluster (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}', date, id, 8192)""" + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id""" ) node.query("INSERT INTO simple VALUES (0, 0)") @@ -77,7 +77,7 @@ def test_deduplication_works_in_case_of_intensive_inserts(started_cluster): node1.query( """ CREATE TABLE simple ON CLUSTER test_cluster (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}', date, id, 8192)""" + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id""" ) node1.query("INSERT INTO simple VALUES (0, 0)") diff --git a/tests/integration/test_https_replication/test.py b/tests/integration/test_https_replication/test.py index 4cf9f19b870..301487aa6cf 100644 --- a/tests/integration/test_https_replication/test.py +++ b/tests/integration/test_https_replication/test.py @@ -19,7 +19,7 @@ def _fill_nodes(nodes, shard): CREATE DATABASE test; CREATE TABLE test_table(date Date, id UInt32, dummy UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}', date, id, 8192); + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; """.format( shard=shard, replica=node.name ) diff --git a/tests/integration/test_insert_into_distributed/test.py b/tests/integration/test_insert_into_distributed/test.py index b8d94d2a043..aff31ec365c 100644 --- a/tests/integration/test_insert_into_distributed/test.py +++ b/tests/integration/test_insert_into_distributed/test.py @@ -52,7 +52,7 @@ CREATE TABLE distributed (x UInt32) ENGINE = Distributed('test_cluster', 'defaul ) remote.query( - "CREATE TABLE local2 (d Date, x UInt32, s String) ENGINE = MergeTree(d, x, 8192)" + "CREATE TABLE local2 (d Date, x UInt32, s String) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x" ) instance_test_inserts_batching.query( """ @@ -61,7 +61,7 @@ CREATE TABLE distributed (d Date, x UInt32) ENGINE = Distributed('test_cluster', ) instance_test_inserts_local_cluster.query( - "CREATE TABLE local (d Date, x UInt32) ENGINE = MergeTree(d, x, 8192)" + "CREATE TABLE local (d Date, x UInt32) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x" ) instance_test_inserts_local_cluster.query( """ @@ -71,12 +71,12 @@ CREATE TABLE distributed_on_local (d Date, x UInt32) ENGINE = Distributed('test_ node1.query( """ -CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node1', date, id, 8192) +CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node1') PARTITION BY toYYYYMM(date) ORDER BY id """ ) node2.query( """ -CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node2', date, id, 8192) +CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node2') PARTITION BY toYYYYMM(date) ORDER BY id """ ) @@ -94,12 +94,12 @@ CREATE TABLE distributed (date Date, id UInt32) ENGINE = Distributed('shard_with shard1.query( """ -CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE = MergeTree(d, x, 8192)""" +CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x""" ) shard2.query( """ -CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE = MergeTree(d, x, 8192)""" +CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x""" ) shard1.query( @@ -143,7 +143,7 @@ CREATE TABLE distributed_one_replica_no_internal_replication (date Date, id UInt node2.query( """ -CREATE TABLE single_replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/single_replicated', 'node2', date, id, 8192) +CREATE TABLE single_replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/single_replicated', 'node2') PARTITION BY toYYYYMM(date) ORDER BY id """ ) @@ -228,11 +228,11 @@ def test_inserts_batching(started_cluster): # 4. Full batch of inserts after ALTER (that have different block structure). # 5. What was left to insert with the column structure before ALTER. expected = """\ -20000101_20000101_1_1_0\t[1] -20000101_20000101_2_2_0\t[2,3,4] -20000101_20000101_3_3_0\t[5,6,7] -20000101_20000101_4_4_0\t[10,11,12] -20000101_20000101_5_5_0\t[8,9] +20000101__1_1_0\t[1] +20000101__2_2_0\t[2,3,4] +20000101__3_3_0\t[5,6,7] +20000101__4_4_0\t[10,11,12] +20000101__5_5_0\t[8,9] """ assert TSV(result) == TSV(expected) diff --git a/tests/integration/test_insert_into_distributed_sync_async/test.py b/tests/integration/test_insert_into_distributed_sync_async/test.py index e0c454feee6..12423cc4747 100755 --- a/tests/integration/test_insert_into_distributed_sync_async/test.py +++ b/tests/integration/test_insert_into_distributed_sync_async/test.py @@ -23,7 +23,7 @@ def started_cluster(): for node in (node1, node2): node.query( """ -CREATE TABLE local_table(date Date, val UInt64) ENGINE = MergeTree(date, (date, val), 8192); +CREATE TABLE local_table(date Date, val UInt64) ENGINE = MergeTree() PARTITION BY toYYYYMM(date) ORDER BY (date, val); """ ) diff --git a/tests/integration/test_insert_into_distributed_through_materialized_view/configs/enable_distributed_inserts_batching.xml b/tests/integration/test_insert_into_distributed_through_materialized_view/configs/enable_distributed_inserts_batching.xml index de0c930b8ab..295d1c8d3cc 100644 --- a/tests/integration/test_insert_into_distributed_through_materialized_view/configs/enable_distributed_inserts_batching.xml +++ b/tests/integration/test_insert_into_distributed_through_materialized_view/configs/enable_distributed_inserts_batching.xml @@ -3,6 +3,7 @@ 1 3 + 1 diff --git a/tests/integration/test_merge_tree_empty_parts/test.py b/tests/integration/test_merge_tree_empty_parts/test.py index 7ca275e96de..57bf49e6803 100644 --- a/tests/integration/test_merge_tree_empty_parts/test.py +++ b/tests/integration/test_merge_tree_empty_parts/test.py @@ -25,7 +25,7 @@ def started_cluster(): def test_empty_parts_alter_delete(started_cluster): node1.query( "CREATE TABLE empty_parts_delete (d Date, key UInt64, value String) \ - ENGINE = ReplicatedMergeTree('/clickhouse/tables/empty_parts_delete', 'r1', d, key, 8192)" + ENGINE = ReplicatedMergeTree('/clickhouse/tables/empty_parts_delete', 'r1') PARTITION BY toYYYYMM(d) ORDER BY key" ) node1.query("INSERT INTO empty_parts_delete VALUES (toDate('2020-10-10'), 1, 'a')") @@ -44,7 +44,7 @@ def test_empty_parts_alter_delete(started_cluster): def test_empty_parts_summing(started_cluster): node1.query( "CREATE TABLE empty_parts_summing (d Date, key UInt64, value Int64) \ - ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/empty_parts_summing', 'r1', d, key, 8192)" + ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/empty_parts_summing', 'r1') PARTITION BY toYYYYMM(d) ORDER BY key" ) node1.query("INSERT INTO empty_parts_summing VALUES (toDate('2020-10-10'), 1, 1)") diff --git a/tests/integration/test_mutations_with_merge_tree/test.py b/tests/integration/test_mutations_with_merge_tree/test.py index d1843017b9f..7831cde7dea 100644 --- a/tests/integration/test_mutations_with_merge_tree/test.py +++ b/tests/integration/test_mutations_with_merge_tree/test.py @@ -17,7 +17,7 @@ def started_cluster(): try: cluster.start() instance_test_mutations.query( - """CREATE TABLE test_mutations_with_ast_elements(date Date, a UInt64, b String) ENGINE = MergeTree(date, (a, date), 8192)""" + """CREATE TABLE test_mutations_with_ast_elements(date Date, a UInt64, b String) ENGINE = MergeTree PARTITION BY toYYYYMM(date) ORDER BY (a, date)""" ) instance_test_mutations.query( """INSERT INTO test_mutations_with_ast_elements SELECT '2019-07-29' AS date, 1, toString(number) FROM numbers(1) SETTINGS force_index_by_date = 0, force_primary_key = 0""" diff --git a/tests/integration/test_random_inserts/test.py b/tests/integration/test_random_inserts/test.py index 4d6aaa9276d..9ac0c5b024c 100644 --- a/tests/integration/test_random_inserts/test.py +++ b/tests/integration/test_random_inserts/test.py @@ -44,7 +44,7 @@ def test_random_inserts(started_cluster): node1.query( """ CREATE TABLE simple ON CLUSTER test_cluster (date Date, i UInt32, s String) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}', date, i, 8192)""" + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY i""" ) with PartitionManager() as pm_random_drops: diff --git a/tests/integration/test_replace_partition/test.py b/tests/integration/test_replace_partition/test.py index 7ce79d9aca8..579b22286b9 100644 --- a/tests/integration/test_replace_partition/test.py +++ b/tests/integration/test_replace_partition/test.py @@ -20,13 +20,13 @@ def _fill_nodes(nodes, shard): CREATE DATABASE test; CREATE TABLE real_table(date Date, id UInt32, dummy UInt32) - ENGINE = MergeTree(date, id, 8192); + ENGINE = MergeTree PARTITION BY toYYYYMM(date) ORDER BY id; CREATE TABLE other_table(date Date, id UInt32, dummy UInt32) - ENGINE = MergeTree(date, id, 8192); + ENGINE = MergeTree PARTITION BY toYYYYMM(date) ORDER BY id; CREATE TABLE test_table(date Date, id UInt32, dummy UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}', date, id, 8192); + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; """.format( shard=shard, replica=node.name ) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 92c109974e1..99fb5bc1e89 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -393,7 +393,7 @@ def test_alters_from_different_replicas(started_cluster): main_node.query( "CREATE TABLE testdb.concurrent_test " "(CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) " - "ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192);" + "ENGINE = MergeTree PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID);" ) main_node.query( @@ -443,7 +443,7 @@ def test_alters_from_different_replicas(started_cluster): " `Added0` UInt32,\\n `Added1` UInt32,\\n `Added2` UInt32,\\n `AddedNested1.A` Array(UInt32),\\n" " `AddedNested1.B` Array(UInt64),\\n `AddedNested1.C` Array(String),\\n `AddedNested2.A` Array(UInt32),\\n" " `AddedNested2.B` Array(UInt64)\\n)\\n" - "ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192)" + "ENGINE = MergeTree\\nPARTITION BY toYYYYMM(StartDate)\\nORDER BY (CounterID, StartDate, intHash32(UserID), VisitID)\\nSETTINGS index_granularity = 8192" ) assert_create_query([main_node, competing_node], "testdb.concurrent_test", expected) diff --git a/tests/integration/test_replication_credentials/test.py b/tests/integration/test_replication_credentials/test.py index e5313cb6bd4..79588fcd38b 100644 --- a/tests/integration/test_replication_credentials/test.py +++ b/tests/integration/test_replication_credentials/test.py @@ -10,7 +10,7 @@ def _fill_nodes(nodes, shard): """ CREATE DATABASE test; CREATE TABLE test_table(date Date, id UInt32, dummy UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}', date, id, 8192); + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; """.format( shard=shard, replica=node.name ) diff --git a/tests/integration/test_send_request_to_leader_replica/test.py b/tests/integration/test_send_request_to_leader_replica/test.py index 60df18bf7d3..b56e1315672 100644 --- a/tests/integration/test_send_request_to_leader_replica/test.py +++ b/tests/integration/test_send_request_to_leader_replica/test.py @@ -40,7 +40,7 @@ def started_cluster(): node.query( """ CREATE TABLE sometable(date Date, id UInt32, value Int32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '{replica}', date, id, 8192); + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; """.format( replica=node.name ), @@ -51,7 +51,7 @@ def started_cluster(): node.query( """ CREATE TABLE someothertable(date Date, id UInt32, value Int32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/someothertable', '{replica}', date, id, 8192); + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/someothertable', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; """.format( replica=node.name ), diff --git a/tests/integration/test_union_header/test.py b/tests/integration/test_union_header/test.py index f883057c1d8..2e7f6cb399a 100644 --- a/tests/integration/test_union_header/test.py +++ b/tests/integration/test_union_header/test.py @@ -27,7 +27,7 @@ def started_cluster(): log_type UInt32, account_id String ) - ENGINE = MergeTree(event_date, (event_time, account_id), 8192); + ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_time, account_id); """ ) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index d3d90ca0d4f..65f82c2286b 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -48,7 +48,7 @@ def test_chroot_with_same_root(started_cluster): node.query( """ CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; """.format( replica=node.name ) @@ -68,7 +68,7 @@ def test_chroot_with_different_root(started_cluster): node.query( """ CREATE TABLE simple_different (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple_different', '{replica}', date, id, 8192); + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple_different', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; """.format( replica=node.name ) diff --git a/tests/integration/test_zookeeper_config/test_password.py b/tests/integration/test_zookeeper_config/test_password.py index 580b426db6f..71f059b3277 100644 --- a/tests/integration/test_zookeeper_config/test_password.py +++ b/tests/integration/test_zookeeper_config/test_password.py @@ -35,7 +35,7 @@ def test_identity(started_cluster): node1.query( """ CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; """.format( replica=node1.name ) @@ -45,6 +45,6 @@ def test_identity(started_cluster): node2.query( """ CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '1', date, id, 8192); + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '1') PARTITION BY toYYYYMM(date) ORDER BY id; """ ) diff --git a/tests/performance/parallel_final.xml b/tests/performance/parallel_final.xml index ca84ed52a04..d7ea0240105 100644 --- a/tests/performance/parallel_final.xml +++ b/tests/performance/parallel_final.xml @@ -4,6 +4,7 @@ 1024 1 20G + 1 diff --git a/tests/performance/scalar2.xml b/tests/performance/scalar2.xml index eb427536646..7a122dbec95 100644 --- a/tests/performance/scalar2.xml +++ b/tests/performance/scalar2.xml @@ -1,4 +1,8 @@ + + 1 + + CREATE TABLE tbl0 (`ds` Date, `x1` String, `x2` UInt32, `x3` UInt32, `x4` UInt32, `bm` AggregateFunction(groupBitmap, UInt32)) ENGINE = MergeTree PARTITION BY (ds, x1) ORDER BY (x2, x3, x4) SETTINGS index_granularity = 1 CREATE TABLE tbl (`ds` Date, `y1` UInt32, `x4` UInt32, `y2` UInt32, `y3` UInt32, `bm` AggregateFunction(groupBitmap, UInt32), `y4` UInt32 DEFAULT 0) ENGINE = MergeTree PARTITION BY (ds) ORDER BY (x4, y2, y3) SETTINGS index_granularity = 8192, max_parts_in_total = 10000000 diff --git a/tests/queries/0_stateless/00030_alter_table.sql b/tests/queries/0_stateless/00030_alter_table.sql index 5fc45575a4a..fb9b3de4067 100644 --- a/tests/queries/0_stateless/00030_alter_table.sql +++ b/tests/queries/0_stateless/00030_alter_table.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS alter_test; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alter_test (CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192); INSERT INTO alter_test VALUES (1, '2014-01-01', 2, 3, [1,2,3], ['a','b','c'], 4); diff --git a/tests/queries/0_stateless/00043_summing_empty_part.sql b/tests/queries/0_stateless/00043_summing_empty_part.sql index 68fc4b5b1c4..40cecabf378 100644 --- a/tests/queries/0_stateless/00043_summing_empty_part.sql +++ b/tests/queries/0_stateless/00043_summing_empty_part.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS empty_summing; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE empty_summing (d Date, k UInt64, v Int8) ENGINE=SummingMergeTree(d, k, 8192); INSERT INTO empty_summing VALUES ('2015-01-01', 1, 10); diff --git a/tests/queries/0_stateless/00046_stored_aggregates_simple.sql b/tests/queries/0_stateless/00046_stored_aggregates_simple.sql index 8b1ef5ba48d..2a4ee9fa5d3 100644 --- a/tests/queries/0_stateless/00046_stored_aggregates_simple.sql +++ b/tests/queries/0_stateless/00046_stored_aggregates_simple.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS stored_aggregates; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE stored_aggregates ( d Date, diff --git a/tests/queries/0_stateless/00047_stored_aggregates_complex.sql b/tests/queries/0_stateless/00047_stored_aggregates_complex.sql index 63728f131b0..2e416f91d5d 100644 --- a/tests/queries/0_stateless/00047_stored_aggregates_complex.sql +++ b/tests/queries/0_stateless/00047_stored_aggregates_complex.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS stored_aggregates; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE stored_aggregates ( d Date, diff --git a/tests/queries/0_stateless/00048_a_stored_aggregates_merge.sql b/tests/queries/0_stateless/00048_a_stored_aggregates_merge.sql index 0138a75c19a..0213ebf46d6 100644 --- a/tests/queries/0_stateless/00048_a_stored_aggregates_merge.sql +++ b/tests/queries/0_stateless/00048_a_stored_aggregates_merge.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS stored_aggregates; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE stored_aggregates ( d Date, diff --git a/tests/queries/0_stateless/00048_b_stored_aggregates_merge.sql b/tests/queries/0_stateless/00048_b_stored_aggregates_merge.sql index 79617692ebd..708794eabff 100644 --- a/tests/queries/0_stateless/00048_b_stored_aggregates_merge.sql +++ b/tests/queries/0_stateless/00048_b_stored_aggregates_merge.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS stored_aggregates; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE stored_aggregates ( d Date, diff --git a/tests/queries/0_stateless/00061_merge_tree_alter.sql b/tests/queries/0_stateless/00061_merge_tree_alter.sql index 822386baa47..ee5694518d9 100644 --- a/tests/queries/0_stateless/00061_merge_tree_alter.sql +++ b/tests/queries/0_stateless/00061_merge_tree_alter.sql @@ -1,6 +1,7 @@ -- Tags: no-backward-compatibility-check DROP TABLE IF EXISTS alter_00061; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alter_00061 (d Date, k UInt64, i32 Int32) ENGINE=MergeTree(d, k, 8192); INSERT INTO alter_00061 VALUES ('2015-01-01', 10, 42); diff --git a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.sql b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.sql index e8d1a713c21..4475421d36f 100644 --- a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.sql +++ b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.sql @@ -6,6 +6,7 @@ DROP TABLE IF EXISTS replicated_alter2; SET replication_alter_partitions_sync = 2; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00062/alter', 'r1', d, k, 8192); CREATE TABLE replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00062/alter', 'r2', d, k, 8192); diff --git a/tests/queries/0_stateless/00079_defaulted_columns.sql b/tests/queries/0_stateless/00079_defaulted_columns.sql index 617dc2435c4..04dfb7057d2 100644 --- a/tests/queries/0_stateless/00079_defaulted_columns.sql +++ b/tests/queries/0_stateless/00079_defaulted_columns.sql @@ -13,6 +13,7 @@ drop table defaulted; create table defaulted (col1 Int8, col2 UInt64 default (SELECT dummy+99 from system.one)) engine=Memory; --{serverError 116} +set allow_deprecated_syntax_for_merge_tree=1; create table defaulted (payload String, date materialized today(), key materialized 0 * rand()) engine=MergeTree(date, key, 8192); desc table defaulted; insert into defaulted (payload) values ('hello clickhouse'); diff --git a/tests/queries/0_stateless/00084_summing_merge_tree.sql b/tests/queries/0_stateless/00084_summing_merge_tree.sql index 5be1371fbcc..429fde5c2b5 100644 --- a/tests/queries/0_stateless/00084_summing_merge_tree.sql +++ b/tests/queries/0_stateless/00084_summing_merge_tree.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS summing_merge_tree; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE summing_merge_tree (d Date, a String, x UInt32, y UInt64, z Float64) ENGINE = SummingMergeTree(d, a, 8192); INSERT INTO summing_merge_tree VALUES ('2000-01-01', 'Hello', 1, 2, 3); diff --git a/tests/queries/0_stateless/00098_shard_i_union_all.sql b/tests/queries/0_stateless/00098_shard_i_union_all.sql index 5151f9ad0e5..58db30a8f9a 100644 --- a/tests/queries/0_stateless/00098_shard_i_union_all.sql +++ b/tests/queries/0_stateless/00098_shard_i_union_all.sql @@ -3,6 +3,7 @@ DROP TABLE IF EXISTS report1; DROP TABLE IF EXISTS report2; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE report1(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192); CREATE TABLE report2(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192); diff --git a/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql b/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql index b05b49ba33a..95d46032601 100644 --- a/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql +++ b/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql @@ -8,6 +8,7 @@ DROP TABLE IF EXISTS test_table; DROP TABLE IF EXISTS test_view; DROP TABLE IF EXISTS test_view_filtered; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test_table (EventDate Date, CounterID UInt32, UserID UInt64, EventTime DateTime('America/Los_Angeles'), UTCEventTime DateTime('UTC')) ENGINE = MergeTree(EventDate, CounterID, 8192); CREATE MATERIALIZED VIEW test_view (Rows UInt64, MaxHitTime DateTime('America/Los_Angeles')) ENGINE = Memory AS SELECT count() AS Rows, max(UTCEventTime) AS MaxHitTime FROM test_table; CREATE MATERIALIZED VIEW test_view_filtered (EventDate Date, CounterID UInt32) ENGINE = Memory POPULATE AS SELECT CounterID, EventDate FROM test_table WHERE EventDate < '2013-01-01'; diff --git a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql index fd32f3d32d2..f62f11c60fd 100644 --- a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql +++ b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql @@ -2,6 +2,7 @@ -- Tag no-replicated-database: Old syntax is not allowed DROP TABLE IF EXISTS alter_00121; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t1', 'r1', d, (d), 8192); INSERT INTO alter_00121 VALUES ('2014-01-01', 1); diff --git a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql index e29a166c1ee..901b818cbc0 100644 --- a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql +++ b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql @@ -4,6 +4,7 @@ SET allow_experimental_parallel_reading_from_replicas = 0; SET max_parallel_replicas = 2; DROP TABLE IF EXISTS report; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE report(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192); INSERT INTO report(id,event_date,priority,description) VALUES (1, '2015-01-01', 1, 'foo')(2, '2015-02-01', 2, 'bar')(3, '2015-03-01', 3, 'foo')(4, '2015-04-01', 4, 'bar')(5, '2015-05-01', 5, 'foo'); diff --git a/tests/queries/0_stateless/00140_prewhere_column_order.sql b/tests/queries/0_stateless/00140_prewhere_column_order.sql index d949b6f780b..61c2fbcf39c 100644 --- a/tests/queries/0_stateless/00140_prewhere_column_order.sql +++ b/tests/queries/0_stateless/00140_prewhere_column_order.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS prewhere; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE prewhere (d Date, a String, b String) ENGINE = MergeTree(d, d, 8192); INSERT INTO prewhere VALUES ('2015-01-01', 'hello', 'world'); diff --git a/tests/queries/0_stateless/00141_parse_timestamp_as_datetime.sql b/tests/queries/0_stateless/00141_parse_timestamp_as_datetime.sql index 4780884ca02..dbd251f878c 100644 --- a/tests/queries/0_stateless/00141_parse_timestamp_as_datetime.sql +++ b/tests/queries/0_stateless/00141_parse_timestamp_as_datetime.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS default; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE default (d Date DEFAULT toDate(t), t DateTime) ENGINE = MergeTree(d, t, 8192); INSERT INTO default (t) VALUES ('1234567890'); SELECT toStartOfMonth(d), toUInt32(t) FROM default; diff --git a/tests/queries/0_stateless/00146_summing_merge_tree_nested_map.sql b/tests/queries/0_stateless/00146_summing_merge_tree_nested_map.sql index ab9e8c9b177..e759fbd85d4 100644 --- a/tests/queries/0_stateless/00146_summing_merge_tree_nested_map.sql +++ b/tests/queries/0_stateless/00146_summing_merge_tree_nested_map.sql @@ -1,5 +1,6 @@ drop table if exists nested_map; +set allow_deprecated_syntax_for_merge_tree=1; create table nested_map (d default today(), k UInt64, payload default rand(), SomeMap Nested(ID UInt32, Num Int64)) engine=SummingMergeTree(d, k, 8192); insert into nested_map (k, `SomeMap.ID`, `SomeMap.Num`) values (0,[1],[100]),(1,[1],[100]),(2,[1],[100]),(3,[1,2],[100,150]); diff --git a/tests/queries/0_stateless/00147_alter_nested_default.sql b/tests/queries/0_stateless/00147_alter_nested_default.sql index 54c99545364..070204aef65 100644 --- a/tests/queries/0_stateless/00147_alter_nested_default.sql +++ b/tests/queries/0_stateless/00147_alter_nested_default.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS alter_00147; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alter_00147 (d Date DEFAULT toDate('2015-01-01'), n Nested(x String)) ENGINE = MergeTree(d, d, 8192); INSERT INTO alter_00147 (`n.x`) VALUES (['Hello', 'World']); diff --git a/tests/queries/0_stateless/00148_summing_merge_tree_aggregate_function.sql b/tests/queries/0_stateless/00148_summing_merge_tree_aggregate_function.sql index eab6b9da465..04edf709bde 100644 --- a/tests/queries/0_stateless/00148_summing_merge_tree_aggregate_function.sql +++ b/tests/queries/0_stateless/00148_summing_merge_tree_aggregate_function.sql @@ -29,6 +29,7 @@ select count() from summing_merge_tree_aggregate_function; drop table summing_merge_tree_aggregate_function; ---- sum + uniq + uniqExact +set allow_deprecated_syntax_for_merge_tree=1; create table summing_merge_tree_aggregate_function ( d materialized today(), k UInt64, diff --git a/tests/queries/0_stateless/00148_summing_merge_tree_nested_map_multiple_values.sql b/tests/queries/0_stateless/00148_summing_merge_tree_nested_map_multiple_values.sql index 68c27cd726f..7c5757cd526 100644 --- a/tests/queries/0_stateless/00148_summing_merge_tree_nested_map_multiple_values.sql +++ b/tests/queries/0_stateless/00148_summing_merge_tree_nested_map_multiple_values.sql @@ -1,5 +1,6 @@ drop table if exists nested_map_multiple_values; +set allow_deprecated_syntax_for_merge_tree=1; create table nested_map_multiple_values (d materialized today(), k UInt64, payload materialized rand(), SomeMap Nested(ID UInt32, Num1 Int64, Num2 Float64)) engine=SummingMergeTree(d, k, 8192); insert into nested_map_multiple_values values (0,[1],[100],[1.0]),(1,[1],[100],[1.0]),(2,[1],[100],[1.0]),(3,[1,2],[100,150],[1.0,1.5]); diff --git a/tests/queries/0_stateless/00155_long_merges.sh b/tests/queries/0_stateless/00155_long_merges.sh index 15ad0892a42..88f9daf6798 100755 --- a/tests/queries/0_stateless/00155_long_merges.sh +++ b/tests/queries/0_stateless/00155_long_merges.sh @@ -11,9 +11,9 @@ function create { $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS aggregating_00155" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS replacing_00155" - $CLICKHOUSE_CLIENT --query="CREATE TABLE summing_00155 (d Date DEFAULT today(), x UInt64, s UInt64 DEFAULT 1) ENGINE = SummingMergeTree(d, x, 8192)" + $CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE summing_00155 (d Date DEFAULT today(), x UInt64, s UInt64 DEFAULT 1) ENGINE = SummingMergeTree(d, x, 8192)" $CLICKHOUSE_CLIENT --query="CREATE TABLE collapsing_00155 (d Date DEFAULT today(), x UInt64, s Int8 DEFAULT 1) ENGINE = CollapsingMergeTree(d, x, 8192, s)" - $CLICKHOUSE_CLIENT --query="CREATE TABLE aggregating_00155 (d Date DEFAULT today(), x UInt64, s AggregateFunction(sum, UInt64)) ENGINE = AggregatingMergeTree(d, x, 8192)" + $CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE aggregating_00155 (d Date DEFAULT today(), x UInt64, s AggregateFunction(sum, UInt64)) ENGINE = AggregatingMergeTree(d, x, 8192)" $CLICKHOUSE_CLIENT --query="CREATE TABLE replacing_00155 (d Date DEFAULT today(), x UInt64, s Int8 DEFAULT 1, v UInt64) ENGINE = ReplacingMergeTree(d, (x), 8192, v)" } diff --git a/tests/queries/0_stateless/00168_buffer_defaults.sql b/tests/queries/0_stateless/00168_buffer_defaults.sql index 8e0008adf4d..ce1dea8aaa8 100644 --- a/tests/queries/0_stateless/00168_buffer_defaults.sql +++ b/tests/queries/0_stateless/00168_buffer_defaults.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS mt_00168; DROP TABLE IF EXISTS mt_00168_buffer; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE mt_00168 (EventDate Date, UTCEventTime DateTime, MoscowEventDate Date DEFAULT toDate(UTCEventTime)) ENGINE = MergeTree(EventDate, UTCEventTime, 8192); CREATE TABLE mt_00168_buffer AS mt_00168 ENGINE = Buffer(currentDatabase(), mt_00168, 16, 10, 100, 10000, 1000000, 10000000, 100000000); DESC TABLE mt_00168; diff --git a/tests/queries/0_stateless/00191_aggregating_merge_tree_and_final.sql b/tests/queries/0_stateless/00191_aggregating_merge_tree_and_final.sql index 776edeeb43c..8160d4dee9e 100644 --- a/tests/queries/0_stateless/00191_aggregating_merge_tree_and_final.sql +++ b/tests/queries/0_stateless/00191_aggregating_merge_tree_and_final.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS aggregating_00191; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE aggregating_00191 (d Date DEFAULT '2000-01-01', k UInt64, u AggregateFunction(uniq, UInt64)) ENGINE = AggregatingMergeTree(d, k, 8192); INSERT INTO aggregating_00191 (k, u) SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000) GROUP BY k; diff --git a/tests/queries/0_stateless/00193_parallel_replicas.sql b/tests/queries/0_stateless/00193_parallel_replicas.sql index 6c5b50972cc..2549ada2a78 100644 --- a/tests/queries/0_stateless/00193_parallel_replicas.sql +++ b/tests/queries/0_stateless/00193_parallel_replicas.sql @@ -3,6 +3,7 @@ DROP TABLE IF EXISTS parallel_replicas; DROP TABLE IF EXISTS parallel_replicas_backup; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE parallel_replicas (d Date DEFAULT today(), x UInt32, u UInt64, s String) ENGINE = MergeTree(d, cityHash64(u, s), (x, d, cityHash64(u, s)), 8192); INSERT INTO parallel_replicas (x, u, s) VALUES (1, 2, 'A'),(3, 4, 'B'),(5, 6, 'C'),(7, 8, 'D'),(9,10,'E'); INSERT INTO parallel_replicas (x, u, s) VALUES (11, 12, 'F'),(13, 14, 'G'),(15, 16, 'H'),(17, 18, 'I'),(19,20,'J'); diff --git a/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.sql b/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.sql index f6eb4b2f8c1..78319c3edd4 100644 --- a/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.sql +++ b/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.sql @@ -6,6 +6,7 @@ SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS attach_r1; DROP TABLE IF EXISTS attach_r2; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE attach_r1 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00236/01/attach', 'r1', d, d, 8192); CREATE TABLE attach_r2 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00236/01/attach', 'r2', d, d, 8192); diff --git a/tests/queries/0_stateless/00253_insert_recursive_defaults.sql b/tests/queries/0_stateless/00253_insert_recursive_defaults.sql index 37bfe0aa188..c0edc447125 100644 --- a/tests/queries/0_stateless/00253_insert_recursive_defaults.sql +++ b/tests/queries/0_stateless/00253_insert_recursive_defaults.sql @@ -5,6 +5,7 @@ SELECT * FROM defaults; DROP TABLE defaults; DROP TABLE IF EXISTS elog_cut; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE elog_cut ( date Date DEFAULT toDate(uts), diff --git a/tests/queries/0_stateless/00278_insert_already_sorted.sql b/tests/queries/0_stateless/00278_insert_already_sorted.sql index 735b46d22dd..b3de48dc155 100644 --- a/tests/queries/0_stateless/00278_insert_already_sorted.sql +++ b/tests/queries/0_stateless/00278_insert_already_sorted.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS sorted; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE sorted (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192); INSERT INTO sorted (x) SELECT intDiv(number, 100000) AS x FROM system.numbers LIMIT 1000000; diff --git a/tests/queries/0_stateless/00282_merging.sql b/tests/queries/0_stateless/00282_merging.sql index 30aec1bc439..a49cde87134 100644 --- a/tests/queries/0_stateless/00282_merging.sql +++ b/tests/queries/0_stateless/00282_merging.sql @@ -73,6 +73,7 @@ SELECT * FROM merge ORDER BY _part_index, x; DROP TABLE merge; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE IF NOT EXISTS merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192); SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; diff --git a/tests/queries/0_stateless/00345_index_accurate_comparison.sql b/tests/queries/0_stateless/00345_index_accurate_comparison.sql index 138a93990cf..aafe2a0ae69 100644 --- a/tests/queries/0_stateless/00345_index_accurate_comparison.sql +++ b/tests/queries/0_stateless/00345_index_accurate_comparison.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS index; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE index ( key Int32, diff --git a/tests/queries/0_stateless/00384_column_aggregate_function_insert_from.sql b/tests/queries/0_stateless/00384_column_aggregate_function_insert_from.sql index 0c614602866..a723bc9b59c 100644 --- a/tests/queries/0_stateless/00384_column_aggregate_function_insert_from.sql +++ b/tests/queries/0_stateless/00384_column_aggregate_function_insert_from.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS aggregates; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE aggregates (d Date, s AggregateFunction(uniq, UInt64)) ENGINE = MergeTree(d, d, 8192); INSERT INTO aggregates diff --git a/tests/queries/0_stateless/00394_new_nested_column_keeps_offsets.sql b/tests/queries/0_stateless/00394_new_nested_column_keeps_offsets.sql index 7c377529416..f058a852e91 100644 --- a/tests/queries/0_stateless/00394_new_nested_column_keeps_offsets.sql +++ b/tests/queries/0_stateless/00394_new_nested_column_keeps_offsets.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS alter_00394; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alter_00394 (d Date, k UInt64, i32 Int32, n Nested(ui8 UInt8, s String)) ENGINE=MergeTree(d, k, 8192); INSERT INTO alter_00394 VALUES ('2015-01-01', 3, 30, [1,2,3], ['1','12','123']); diff --git a/tests/queries/0_stateless/00394_replaceall_vector_fixed.sql b/tests/queries/0_stateless/00394_replaceall_vector_fixed.sql index 47ab3fe4a1c..ada48add403 100644 --- a/tests/queries/0_stateless/00394_replaceall_vector_fixed.sql +++ b/tests/queries/0_stateless/00394_replaceall_vector_fixed.sql @@ -14,6 +14,7 @@ ORDER BY str ASC; DROP TABLE replaceall; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE replaceall (date Date DEFAULT today(), fs FixedString(16)) ENGINE = MergeTree(date, (date, fs), 8192); INSERT INTO replaceall (fs) VALUES ('54db0d43009d\0\0\0\0'), ('fe2b58224766cf10'), ('54db0d43009d\0\0\0\0'), ('fe2b58224766cf10'); diff --git a/tests/queries/0_stateless/00395_nullable.sql b/tests/queries/0_stateless/00395_nullable.sql index 71dc045ad09..83d27830f4e 100644 --- a/tests/queries/0_stateless/00395_nullable.sql +++ b/tests/queries/0_stateless/00395_nullable.sql @@ -8,6 +8,7 @@ SELECT NULL + NULL; SELECT '----- MergeTree engine -----'; DROP TABLE IF EXISTS test1_00395; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test1_00395( col1 UInt64, col2 Nullable(UInt64), col3 String, col4 Nullable(String), diff --git a/tests/queries/0_stateless/00427_alter_primary_key.sh b/tests/queries/0_stateless/00427_alter_primary_key.sh index 4ad1166bfa4..1269e2ad6e3 100755 --- a/tests/queries/0_stateless/00427_alter_primary_key.sh +++ b/tests/queries/0_stateless/00427_alter_primary_key.sh @@ -8,6 +8,7 @@ function perform() { local query=$1 TZ=UTC $CLICKHOUSE_CLIENT \ + --allow_deprecated_syntax_for_merge_tree=1 \ --use_client_time_zone=1 \ --input_format_values_interpret_expressions=0 \ --query "$query" 2>/dev/null diff --git a/tests/queries/0_stateless/00440_nulls_merge_tree.sql b/tests/queries/0_stateless/00440_nulls_merge_tree.sql index 7281f960509..dd9473027b4 100644 --- a/tests/queries/0_stateless/00440_nulls_merge_tree.sql +++ b/tests/queries/0_stateless/00440_nulls_merge_tree.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS nulls; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE nulls (d Date, x Nullable(UInt64)) ENGINE = MergeTree(d, d, 8192); INSERT INTO nulls SELECT toDate('2000-01-01'), number % 10 != 0 ? number : NULL FROM system.numbers LIMIT 10000; SELECT count() FROM nulls WHERE x IS NULL; diff --git a/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh b/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh index a441c9a6761..2f79365f756 100755 --- a/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh +++ b/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -ch="$CLICKHOUSE_CLIENT --stacktrace -q" +ch="$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 --stacktrace -q" $ch "DROP TABLE IF EXISTS clear_column1" $ch "DROP TABLE IF EXISTS clear_column2" diff --git a/tests/queries/0_stateless/00453_cast_enum.sql b/tests/queries/0_stateless/00453_cast_enum.sql index 70b7e2b1d6d..384db50c7c4 100644 --- a/tests/queries/0_stateless/00453_cast_enum.sql +++ b/tests/queries/0_stateless/00453_cast_enum.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS cast_enums; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE cast_enums ( type Enum8('session' = 1, 'pageview' = 2, 'click' = 3), diff --git a/tests/queries/0_stateless/00495_reading_const_zero_column.sql b/tests/queries/0_stateless/00495_reading_const_zero_column.sql index da529c5b9b7..0af201b66e9 100644 --- a/tests/queries/0_stateless/00495_reading_const_zero_column.sql +++ b/tests/queries/0_stateless/00495_reading_const_zero_column.sql @@ -1,4 +1,5 @@ drop table if exists one_table; +set allow_deprecated_syntax_for_merge_tree=1; create table one_table (date Date, one UInt64) engine = MergeTree(date, (date, one), 8192); insert into one_table select today(), toUInt64(1) from system.numbers limit 100000; SET preferred_block_size_bytes = 8192; diff --git a/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql b/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql index 766f9dfb368..7c939d060ea 100644 --- a/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql +++ b/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql @@ -1,4 +1,5 @@ +set allow_deprecated_syntax_for_merge_tree=1; drop table if exists test_ins_arr; create table test_ins_arr (date Date, val Array(UInt64)) engine = MergeTree(date, (date), 8192); insert into test_ins_arr select toDate('2017-10-02'), [number, 42] from system.numbers limit 10000; diff --git a/tests/queries/0_stateless/00506_union_distributed.sql b/tests/queries/0_stateless/00506_union_distributed.sql index 08282d94d8b..8a5dba00c27 100644 --- a/tests/queries/0_stateless/00506_union_distributed.sql +++ b/tests/queries/0_stateless/00506_union_distributed.sql @@ -8,6 +8,7 @@ DROP TABLE IF EXISTS union1; DROP TABLE IF EXISTS union2; DROP TABLE IF EXISTS union3; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE union1 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = MergeTree(date, (a, date), 8192); CREATE TABLE union2 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost, currentDatabase(), 'union1'); CREATE TABLE union3 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost, currentDatabase(), 'union2'); diff --git a/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.sql b/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.sql index 64ceb394b17..e358072248a 100644 --- a/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.sql +++ b/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.sql @@ -1,6 +1,7 @@ -- Tags: shard drop table if exists tab; +set allow_deprecated_syntax_for_merge_tree=1; create table tab (date Date, val UInt64, val2 UInt8 default 42, val3 UInt8 default val2 + 1, val4 UInt64 alias val) engine = MergeTree(date, (date, val), 8192); desc tab; select '-'; diff --git a/tests/queries/0_stateless/00516_deduplication_after_drop_partition_zookeeper.sql b/tests/queries/0_stateless/00516_deduplication_after_drop_partition_zookeeper.sql index 6c76adb86d0..08dccac9d25 100644 --- a/tests/queries/0_stateless/00516_deduplication_after_drop_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00516_deduplication_after_drop_partition_zookeeper.sql @@ -2,6 +2,7 @@ -- Tag no-replicated-database: Old syntax is not allowed DROP TABLE IF EXISTS deduplication_by_partition; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE deduplication_by_partition(d Date, x UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00516/deduplication_by_partition', 'r1', d, x, 8192); diff --git a/tests/queries/0_stateless/00531_aggregate_over_nullable.sql b/tests/queries/0_stateless/00531_aggregate_over_nullable.sql index ff485b4251a..1680bb90bb1 100644 --- a/tests/queries/0_stateless/00531_aggregate_over_nullable.sql +++ b/tests/queries/0_stateless/00531_aggregate_over_nullable.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS agg_over_nullable; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE agg_over_nullable ( partition Date, timestamp DateTime, diff --git a/tests/queries/0_stateless/00543_null_and_prewhere.sql b/tests/queries/0_stateless/00543_null_and_prewhere.sql index 793d65925a1..5f50397862a 100644 --- a/tests/queries/0_stateless/00543_null_and_prewhere.sql +++ b/tests/queries/0_stateless/00543_null_and_prewhere.sql @@ -1,3 +1,5 @@ + +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test ( dt Date, diff --git a/tests/queries/0_stateless/00561_storage_join.sql b/tests/queries/0_stateless/00561_storage_join.sql index 913ecec6f4a..9927592465a 100644 --- a/tests/queries/0_stateless/00561_storage_join.sql +++ b/tests/queries/0_stateless/00561_storage_join.sql @@ -1,5 +1,6 @@ drop table IF EXISTS joinbug; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE joinbug ( event_date Date MATERIALIZED toDate(created, 'Asia/Istanbul'), id UInt64, diff --git a/tests/queries/0_stateless/00563_complex_in_expression.sql b/tests/queries/0_stateless/00563_complex_in_expression.sql index cd80b9c3a7a..bd053e0d020 100644 --- a/tests/queries/0_stateless/00563_complex_in_expression.sql +++ b/tests/queries/0_stateless/00563_complex_in_expression.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS test_00563; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test_00563 ( dt Date, site_id Int32, site_key String ) ENGINE = MergeTree(dt, (site_id, site_key, dt), 8192); INSERT INTO test_00563 (dt,site_id, site_key) VALUES ('2018-1-29', 100, 'key'); SELECT * FROM test_00563 WHERE toInt32(site_id) IN (100); diff --git a/tests/queries/0_stateless/00564_initial_column_values_with_default_expression.sql b/tests/queries/0_stateless/00564_initial_column_values_with_default_expression.sql index d26621dce7a..3fff20a1fe1 100644 --- a/tests/queries/0_stateless/00564_initial_column_values_with_default_expression.sql +++ b/tests/queries/0_stateless/00564_initial_column_values_with_default_expression.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS test; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE IF NOT EXISTS test( id UInt32, track UInt8, codec String, content String, rdate Date DEFAULT '2018-02-03', track_id String DEFAULT concat(concat(concat(toString(track), '-'), codec), content) ) ENGINE=MergeTree(rdate, (id, track_id), 8192); INSERT INTO test(id, track, codec) VALUES(1, 0, 'h264'); diff --git a/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql b/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql index c384ef37fc7..46fc0dd586d 100644 --- a/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql +++ b/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql @@ -8,6 +8,7 @@ DROP DATABASE IF EXISTS none; DROP TABLE IF EXISTS test_00571; DROP TABLE IF EXISTS test_materialized_00571; +set allow_deprecated_syntax_for_merge_tree=1; CREATE DATABASE none; CREATE TABLE test_00571 ( date Date, platform Enum8('a' = 0, 'b' = 1, 'c' = 2), app Enum8('a' = 0, 'b' = 1) ) ENGINE = MergeTree(date, (platform, app), 8192); CREATE MATERIALIZED VIEW test_materialized_00571 ENGINE = MergeTree(date, (platform, app), 8192) POPULATE AS SELECT date, platform, app FROM (SELECT * FROM test_00571); diff --git a/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh b/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh index 42bf37cbda5..706c0819125 100755 --- a/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh +++ b/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) exception_pattern="Code: 44.*Cannot drop column \`id\`, because column \`id2\` depends on it" ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_00575;" -${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_00575 (dt Date DEFAULT now(), id UInt32, id2 UInt32 DEFAULT id + 1) ENGINE = MergeTree(dt, dt, 8192);" +${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query "CREATE TABLE test_00575 (dt Date DEFAULT now(), id UInt32, id2 UInt32 DEFAULT id + 1) ENGINE = MergeTree(dt, dt, 8192);" ${CLICKHOUSE_CLIENT} --query "INSERT INTO test_00575(dt,id) VALUES ('2018-02-22',3), ('2018-02-22',4), ('2018-02-22',5);" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_00575 ORDER BY id;" echo "$(${CLICKHOUSE_CLIENT} --query "ALTER TABLE test_00575 DROP COLUMN id;" --server_logs_file=/dev/null 2>&1 | grep -c "$exception_pattern")" diff --git a/tests/queries/0_stateless/00575_merge_and_index_with_function_in_in.sql b/tests/queries/0_stateless/00575_merge_and_index_with_function_in_in.sql index e491ef37ada..6f0ddd9fae3 100644 --- a/tests/queries/0_stateless/00575_merge_and_index_with_function_in_in.sql +++ b/tests/queries/0_stateless/00575_merge_and_index_with_function_in_in.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS t_00575; +set allow_deprecated_syntax_for_merge_tree=1; create table t_00575(d Date) engine MergeTree(d, d, 8192); insert into t_00575 values ('2018-02-20'); diff --git a/tests/queries/0_stateless/00584_view_union_all.sql b/tests/queries/0_stateless/00584_view_union_all.sql index 2e4d7ea66db..a86dfaec6c8 100644 --- a/tests/queries/0_stateless/00584_view_union_all.sql +++ b/tests/queries/0_stateless/00584_view_union_all.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS Test_00584; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE Test_00584 ( createdDate Date, str String, diff --git a/tests/queries/0_stateless/00594_alias_in_distributed.sql b/tests/queries/0_stateless/00594_alias_in_distributed.sql index b75dfdf7967..250ede2bb10 100644 --- a/tests/queries/0_stateless/00594_alias_in_distributed.sql +++ b/tests/queries/0_stateless/00594_alias_in_distributed.sql @@ -3,6 +3,7 @@ DROP TABLE IF EXISTS alias_local10; DROP TABLE IF EXISTS alias10; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alias_local10 ( Id Int8, EventDate Date DEFAULT '2000-01-01', diff --git a/tests/queries/0_stateless/00597_push_down_predicate_long.sql b/tests/queries/0_stateless/00597_push_down_predicate_long.sql index 40ea8b48caa..a77f3730ac2 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate_long.sql +++ b/tests/queries/0_stateless/00597_push_down_predicate_long.sql @@ -7,6 +7,7 @@ SET joined_subquery_requires_alias = 0; DROP TABLE IF EXISTS test_00597; DROP TABLE IF EXISTS test_view_00597; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test_00597(date Date, id Int8, name String, value Int64) ENGINE = MergeTree(date, (id, date), 8192); CREATE VIEW test_view_00597 AS SELECT * FROM test_00597; diff --git a/tests/queries/0_stateless/00609_mv_index_in_in.sql b/tests/queries/0_stateless/00609_mv_index_in_in.sql index 002c94dd576..bd9f35350c1 100644 --- a/tests/queries/0_stateless/00609_mv_index_in_in.sql +++ b/tests/queries/0_stateless/00609_mv_index_in_in.sql @@ -6,6 +6,7 @@ DROP TABLE IF EXISTS test_mv_00609; create table test_00609 (a Int8) engine=Memory; insert into test_00609 values (1); +set allow_deprecated_syntax_for_merge_tree=1; create materialized view test_mv_00609 uuid '00000609-1000-4000-8000-000000000001' Engine=MergeTree(date, (a), 8192) populate as select a, toDate('2000-01-01') date from test_00609; select * from test_mv_00609; -- OK diff --git a/tests/queries/0_stateless/00610_materialized_view_forward_alter_partition_statements.sql b/tests/queries/0_stateless/00610_materialized_view_forward_alter_partition_statements.sql index 6f5ba07e5db..8830204ecb5 100644 --- a/tests/queries/0_stateless/00610_materialized_view_forward_alter_partition_statements.sql +++ b/tests/queries/0_stateless/00610_materialized_view_forward_alter_partition_statements.sql @@ -1,6 +1,7 @@ DROP TABLE IF EXISTS tab_00610; DROP TABLE IF EXISTS mv_00610; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE tab_00610(d Date, x UInt32) ENGINE MergeTree(d, x, 8192); CREATE MATERIALIZED VIEW mv_00610(d Date, y UInt64) ENGINE MergeTree(d, y, 8192) AS SELECT d, x + 1 AS y FROM tab_00610; diff --git a/tests/queries/0_stateless/00614_shard_same_header_for_local_and_remote_node_in_distributed_query.sql b/tests/queries/0_stateless/00614_shard_same_header_for_local_and_remote_node_in_distributed_query.sql index 5d373927714..17cc76670b3 100644 --- a/tests/queries/0_stateless/00614_shard_same_header_for_local_and_remote_node_in_distributed_query.sql +++ b/tests/queries/0_stateless/00614_shard_same_header_for_local_and_remote_node_in_distributed_query.sql @@ -1,6 +1,7 @@ -- Tags: distributed drop table if exists tab; +set allow_deprecated_syntax_for_merge_tree=1; create table tab (date Date, time DateTime, data String) ENGINE = MergeTree(date, (time, data), 8192); insert into tab values ('2018-01-21','2018-01-21 15:12:13','test'); select time FROM remote('127.0.0.{1,2}', currentDatabase(), tab) WHERE date = '2018-01-21' limit 2; diff --git a/tests/queries/0_stateless/00615_nullable_alter_optimize.sql b/tests/queries/0_stateless/00615_nullable_alter_optimize.sql index 5c0a192e826..26ff3b78d5e 100644 --- a/tests/queries/0_stateless/00615_nullable_alter_optimize.sql +++ b/tests/queries/0_stateless/00615_nullable_alter_optimize.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS test_00615; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test_00615 ( dt Date, diff --git a/tests/queries/0_stateless/00621_regression_for_in_operator.sql b/tests/queries/0_stateless/00621_regression_for_in_operator.sql index 84355a04df4..273f930a90f 100644 --- a/tests/queries/0_stateless/00621_regression_for_in_operator.sql +++ b/tests/queries/0_stateless/00621_regression_for_in_operator.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS regression_for_in_operator_view; DROP TABLE IF EXISTS regression_for_in_operator; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE regression_for_in_operator (d Date, v UInt32, g String) ENGINE=MergeTree(d, d, 8192); CREATE MATERIALIZED VIEW regression_for_in_operator_view ENGINE=AggregatingMergeTree(d, (d,g), 8192) AS SELECT d, g, maxState(v) FROM regression_for_in_operator GROUP BY d, g; diff --git a/tests/queries/0_stateless/00623_in_partition_key.sql b/tests/queries/0_stateless/00623_in_partition_key.sql index 0141151e369..6cb33f96f13 100644 --- a/tests/queries/0_stateless/00623_in_partition_key.sql +++ b/tests/queries/0_stateless/00623_in_partition_key.sql @@ -1,4 +1,5 @@ drop table if exists test54378; +set allow_deprecated_syntax_for_merge_tree=1; create table test54378 (part_date Date, pk_date Date, date Date) Engine=MergeTree(part_date, pk_date, 8192); insert into test54378 values ('2018-04-19', '2018-04-19', '2018-04-19'); diff --git a/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.sql b/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.sql index 4a3640e2dd7..44a7d8ca60f 100644 --- a/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.sql +++ b/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.sql @@ -4,6 +4,7 @@ DROP TABLE IF EXISTS replicated_truncate1; DROP TABLE IF EXISTS replicated_truncate2; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE replicated_truncate1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00623/truncate', 'r1', d, k, 8192); CREATE TABLE replicated_truncate2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00623/truncate', 'r2', d, k, 8192); diff --git a/tests/queries/0_stateless/00648_replacing_empty_set_from_prewhere.sql b/tests/queries/0_stateless/00648_replacing_empty_set_from_prewhere.sql index 1f873c94b18..bbeb4dd3148 100644 --- a/tests/queries/0_stateless/00648_replacing_empty_set_from_prewhere.sql +++ b/tests/queries/0_stateless/00648_replacing_empty_set_from_prewhere.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS final_test; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE final_test (id String, version Date) ENGINE = ReplacingMergeTree(version, id, 8192); INSERT INTO final_test (id, version) VALUES ('2018-01-01', '2018-01-01'); SELECT * FROM final_test FINAL PREWHERE id == '2018-01-02'; diff --git a/tests/queries/0_stateless/00652_mergetree_mutations.sh b/tests/queries/0_stateless/00652_mergetree_mutations.sh index 0cba8b94040..535cbe99dfe 100755 --- a/tests/queries/0_stateless/00652_mergetree_mutations.sh +++ b/tests/queries/0_stateless/00652_mergetree_mutations.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations(d Date, x UInt32, s String, a UInt32 ALIAS x + 1, m MATERIALIZED x + 2) ENGINE MergeTree(d, intDiv(x, 10), 8192)" +${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations(d Date, x UInt32, s String, a UInt32 ALIAS x + 1, m MATERIALIZED x + 2) ENGINE MergeTree(d, intDiv(x, 10), 8192)" # Test a mutation on empty table ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations DELETE WHERE x = 1" diff --git a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh index f070e34f2a1..7a6c7609660 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh @@ -12,8 +12,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r1" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r2" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r1(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r1', d, intDiv(x, 10), 8192)" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r2', d, intDiv(x, 10), 8192)" +${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r1(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r1', d, intDiv(x, 10), 8192)" +${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r2', d, intDiv(x, 10), 8192)" # Test a mutation on empty table ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" diff --git a/tests/queries/0_stateless/00678_shard_funnel_window.sql b/tests/queries/0_stateless/00678_shard_funnel_window.sql index 8ea07a1c4ba..73e48923283 100644 --- a/tests/queries/0_stateless/00678_shard_funnel_window.sql +++ b/tests/queries/0_stateless/00678_shard_funnel_window.sql @@ -1,6 +1,7 @@ -- Tags: shard DROP TABLE IF EXISTS remote_test; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE remote_test(uid String, its UInt32, action_code String, day Date) ENGINE = MergeTree(day, (uid, its), 8192); INSERT INTO remote_test SELECT toString(number) AS uid, number % 3 AS its, toString(number % 3) AS action_code, '2000-01-01' FROM system.numbers LIMIT 10000; SELECT level, COUNT() FROM (SELECT uid, windowFunnel(3600)(toUInt32(its), action_code != '', action_code = '2') AS level FROM remote('127.0.0.{2,3}', currentDatabase(), remote_test) GROUP BY uid) GROUP BY level; diff --git a/tests/queries/0_stateless/00712_prewhere_with_alias_bug_2.sql b/tests/queries/0_stateless/00712_prewhere_with_alias_bug_2.sql index 97d5e33633a..beb986adebc 100644 --- a/tests/queries/0_stateless/00712_prewhere_with_alias_bug_2.sql +++ b/tests/queries/0_stateless/00712_prewhere_with_alias_bug_2.sql @@ -1,5 +1,6 @@ drop table if exists table; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE table (a UInt32, date Date, b UInt64, c UInt64, str String, d Int8, arr Array(UInt64), arr_alias Array(UInt64) ALIAS arr) ENGINE = MergeTree(date, intHash32(c), (a, date, intHash32(c), b), 8192); SELECT alias2 AS alias3 diff --git a/tests/queries/0_stateless/00712_prewhere_with_final.sql b/tests/queries/0_stateless/00712_prewhere_with_final.sql index 4528d4f61f0..6b49e523f70 100644 --- a/tests/queries/0_stateless/00712_prewhere_with_final.sql +++ b/tests/queries/0_stateless/00712_prewhere_with_final.sql @@ -1,4 +1,5 @@ drop table if exists trepl; +set allow_deprecated_syntax_for_merge_tree=1; create table trepl(d Date,a Int32, b Int32) engine = ReplacingMergeTree(d, (a,b), 8192); insert into trepl values ('2018-09-19', 1, 1); select b from trepl FINAL prewhere a < 1000; diff --git a/tests/queries/0_stateless/00712_prewhere_with_sampling_and_alias.sql b/tests/queries/0_stateless/00712_prewhere_with_sampling_and_alias.sql index d011e1b368d..7c8ae4eed7c 100644 --- a/tests/queries/0_stateless/00712_prewhere_with_sampling_and_alias.sql +++ b/tests/queries/0_stateless/00712_prewhere_with_sampling_and_alias.sql @@ -1,4 +1,5 @@ drop table if exists t_00712_2; +set allow_deprecated_syntax_for_merge_tree=1; create table t_00712_2 (date Date, counter UInt64, sampler UInt64, alias_col alias date + 1) engine = MergeTree(date, intHash32(sampler), (counter, date, intHash32(sampler)), 8192); insert into t_00712_2 values ('2018-01-01', 1, 1); select alias_col from t_00712_2 sample 1 / 2 where date = '2018-01-01' and counter = 1 and sampler = 1; diff --git a/tests/queries/0_stateless/00717_merge_and_distributed.sql b/tests/queries/0_stateless/00717_merge_and_distributed.sql index f7b4a2b24d3..f27f6a75688 100644 --- a/tests/queries/0_stateless/00717_merge_and_distributed.sql +++ b/tests/queries/0_stateless/00717_merge_and_distributed.sql @@ -7,6 +7,7 @@ DROP TABLE IF EXISTS test_local_2; DROP TABLE IF EXISTS test_distributed_1; DROP TABLE IF EXISTS test_distributed_2; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test_local_1 (date Date, value UInt32) ENGINE = MergeTree(date, date, 8192); CREATE TABLE test_local_2 (date Date, value UInt32) ENGINE = MergeTree(date, date, 8192); CREATE TABLE test_distributed_1 AS test_local_1 ENGINE = Distributed('test_shard_localhost', currentDatabase(), test_local_1, rand()); diff --git a/tests/queries/0_stateless/00729_prewhere_array_join.sql b/tests/queries/0_stateless/00729_prewhere_array_join.sql index ba10dd38bd2..5ac79c150c6 100644 --- a/tests/queries/0_stateless/00729_prewhere_array_join.sql +++ b/tests/queries/0_stateless/00729_prewhere_array_join.sql @@ -1,6 +1,7 @@ SET send_logs_level = 'fatal'; drop table if exists t1_00729; +set allow_deprecated_syntax_for_merge_tree=1; create table t1_00729 (id UInt64, val Array(String),nid UInt64, eDate Date)ENGINE = MergeTree(eDate, (id, eDate), 8192); insert into t1_00729 (id,val,nid,eDate) values (1,['background','foreground','heading','image'],1,'2018-09-27'); diff --git a/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh b/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh index d9a3631a7dd..2510517a740 100755 --- a/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh +++ b/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -settings="--log_queries=1 --log_query_threads=1 --log_profile_events=1 --log_query_settings=1" +settings="--log_queries=1 --log_query_threads=1 --log_profile_events=1 --log_query_settings=1 --allow_deprecated_syntax_for_merge_tree=1" # Test insert logging on each block and checkPacket() method diff --git a/tests/queries/0_stateless/00732_decimal_summing_merge_tree.sql b/tests/queries/0_stateless/00732_decimal_summing_merge_tree.sql index 1524776c16e..96e16a1300e 100644 --- a/tests/queries/0_stateless/00732_decimal_summing_merge_tree.sql +++ b/tests/queries/0_stateless/00732_decimal_summing_merge_tree.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS decimal_sum; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE decimal_sum ( date Date, diff --git a/tests/queries/0_stateless/00748_insert_array_with_null.sql b/tests/queries/0_stateless/00748_insert_array_with_null.sql index 78c564abad3..ca36352c2cf 100644 --- a/tests/queries/0_stateless/00748_insert_array_with_null.sql +++ b/tests/queries/0_stateless/00748_insert_array_with_null.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS arraytest; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE arraytest ( created_date Date DEFAULT toDate(created_at), created_at DateTime DEFAULT now(), strings Array(String) DEFAULT emptyArrayString()) ENGINE = MergeTree(created_date, cityHash64(created_at), (created_date, cityHash64(created_at)), 8192); INSERT INTO arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', 'ccccc']); diff --git a/tests/queries/0_stateless/00752_low_cardinality_mv_2.sql b/tests/queries/0_stateless/00752_low_cardinality_mv_2.sql index 71950469a47..83c6b1c1a6b 100644 --- a/tests/queries/0_stateless/00752_low_cardinality_mv_2.sql +++ b/tests/queries/0_stateless/00752_low_cardinality_mv_2.sql @@ -1,6 +1,7 @@ drop table if exists radacct; drop table if exists mv_traffic_by_tadig15min; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE radacct ( radacctid UInt64, f3gppchargingid Nullable(String), f3gppggsnaddress Nullable(String), f3gppggsnmccmnc Nullable(String), f3gppgprsqos Nullable(String), f3gppimeisv Nullable(String), f3gppimsi Nullable(UInt64), f3gppimsimccmnc Nullable(String), f3gpploci Nullable(String), f3gppnsapi Nullable(String), f3gpprattype Nullable(String), f3gppsgsnaddress Nullable(String), f3gppsgsnmccmnc Nullable(String), acctdelaytime Nullable(UInt32), acctinputoctets Nullable(UInt64), acctinputpackets Nullable(UInt64), acctoutputoctets Nullable(UInt64), acctoutputpackets Nullable(UInt64), acctsessionid String, acctstatustype Nullable(String), acctuniqueid String, calledstationid Nullable(String), callingstationid Nullable(String), framedipaddress Nullable(String), nasidentifier Nullable(String), nasipaddress Nullable(String), acctstarttime Nullable(DateTime), acctstoptime Nullable(DateTime), acctsessiontime Nullable(UInt32), acctterminatecause Nullable(String), acctstartdelay Nullable(UInt32), acctstopdelay Nullable(UInt32), connectinfo_start Nullable(String), connectinfo_stop Nullable(String), timestamp DateTime, username Nullable(String), realm Nullable(String), f3gppimsi_int UInt64, f3gppsgsnaddress_int Nullable(UInt32), timestamp_date Date, tac Nullable(String), mnc Nullable(String), tadig LowCardinality(String), country LowCardinality(String), tadig_op_ip Nullable(String) DEFAULT CAST('TADIG NOT FOUND', 'Nullable(String)'), mcc Nullable(UInt16) MATERIALIZED toUInt16OrNull(substring(f3gppsgsnmccmnc, 1, 6))) ENGINE = MergeTree(timestamp_date, (timestamp, radacctid, acctuniqueid), 8192); insert into radacct values (1, 'a', 'b', 'c', 'd', 'e', 2, 'a', 'b', 'c', 'd', 'e', 'f', 3, 4, 5, 6, 7, 'a', 'Stop', 'c', 'd', 'e', 'f', 'g', 'h', '2018-10-10 15:54:21', '2018-10-10 15:54:21', 8, 'a', 9, 10, 'a', 'b', '2018-10-10 15:54:21', 'a', 'b', 11, 12, '2018-10-10', 'a', 'b', 'c', 'd', 'e'); diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql index a2d19c22500..e1392d299dc 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql @@ -49,6 +49,7 @@ FORMAT PrettyCompactNoEscapes; DROP TABLE IF EXISTS check_system_tables; -- Check MergeTree declaration in old format +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE check_system_tables ( Event Date, diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by.sql b/tests/queries/0_stateless/00754_alter_modify_order_by.sql index cb81f868e7b..234bd61902b 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by.sql @@ -2,6 +2,7 @@ SET send_logs_level = 'fatal'; SET optimize_on_insert = 0; DROP TABLE IF EXISTS old_style; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE old_style(d Date, x UInt32) ENGINE MergeTree(d, x, 8192); ALTER TABLE old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36} DROP TABLE old_style; diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.sql b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.sql index c859c7b9921..29d0ef79b91 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.sql @@ -6,6 +6,7 @@ SET optimize_on_insert = 0; SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS old_style; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00754/old_style', 'r1', d, x, 8192); ALTER TABLE old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36 } DROP TABLE old_style; diff --git a/tests/queries/0_stateless/00794_materialized_view_with_column_defaults.sql b/tests/queries/0_stateless/00794_materialized_view_with_column_defaults.sql index a756f5c3537..43dcb322f1c 100644 --- a/tests/queries/0_stateless/00794_materialized_view_with_column_defaults.sql +++ b/tests/queries/0_stateless/00794_materialized_view_with_column_defaults.sql @@ -1,6 +1,7 @@ DROP TABLE IF EXISTS table_view; DROP TABLE IF EXISTS source_table; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE source_table ( date Date, datetime DateTime, diff --git a/tests/queries/0_stateless/00800_low_cardinality_distributed_insert.sql b/tests/queries/0_stateless/00800_low_cardinality_distributed_insert.sql index 600793853ca..7ddf1caa85f 100644 --- a/tests/queries/0_stateless/00800_low_cardinality_distributed_insert.sql +++ b/tests/queries/0_stateless/00800_low_cardinality_distributed_insert.sql @@ -5,6 +5,7 @@ SET insert_distributed_sync = 1; DROP TABLE IF EXISTS low_cardinality; DROP TABLE IF EXISTS low_cardinality_all; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE = MergeTree(d, x, 8192); CREATE TABLE low_cardinality_all (d Date, x UInt32, s LowCardinality(String)) ENGINE = Distributed(test_shard_localhost, currentDatabase(), low_cardinality, sipHash64(s)); diff --git a/tests/queries/0_stateless/00806_alter_update.sql b/tests/queries/0_stateless/00806_alter_update.sql index c4fe3969df3..c9b1bfb2d9d 100644 --- a/tests/queries/0_stateless/00806_alter_update.sql +++ b/tests/queries/0_stateless/00806_alter_update.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS alter_update_00806; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alter_update_00806 (d Date, e Enum8('foo'=1, 'bar'=2)) Engine = MergeTree(d, (d), 8192); INSERT INTO alter_update_00806 (d, e) VALUES ('2018-01-01', 'foo'); INSERT INTO alter_update_00806 (d, e) VALUES ('2018-01-02', 'bar'); diff --git a/tests/queries/0_stateless/00829_bitmap_function.sql b/tests/queries/0_stateless/00829_bitmap_function.sql index fde0176de5b..6a21f5caf0f 100644 --- a/tests/queries/0_stateless/00829_bitmap_function.sql +++ b/tests/queries/0_stateless/00829_bitmap_function.sql @@ -71,6 +71,7 @@ SELECT bitmapToArray(bitmapAnd(groupBitmapState(uid), bitmapBuild(CAST([1, 2, 3] -- bitmap state test DROP TABLE IF EXISTS bitmap_state_test; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE bitmap_state_test ( pickup_date Date, diff --git a/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh b/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh index 6640e0003cd..41a35d908d1 100755 --- a/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh +++ b/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh @@ -26,5 +26,5 @@ $CLICKHOUSE_CLIENT -q "select * FROM numberss(10);" 2>&1 | grep "Maybe you meant $CLICKHOUSE_CLIENT -q "select * FROM anothernumbers(10);" 2>&1 | grep -v "Maybe you meant: \['numbers'\,'numbers_mt'\]." &>/dev/null $CLICKHOUSE_CLIENT -q "select * FROM mynumbers(10);" 2>&1 | grep "Maybe you meant: \['numbers'\]." &>/dev/null -$CLICKHOUSE_CLIENT -q "CREATE TABLE stored_aggregates (d Date, Uniq AggregateFunction(uniq, UInt64)) ENGINE = MergeTre(d, d, 8192);" 2>&1 | grep "Maybe you meant: \['MergeTree'\]." &>/dev/null -$CLICKHOUSE_CLIENT -q "CREATE TABLE stored_aggregates (d Date, Uniq AgregateFunction(uniq, UInt64)) ENGINE = MergeTree(d, d, 8192);" 2>&1 | grep "Maybe you meant: \['AggregateFunction'\]." &>/dev/null +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE stored_aggregates (d Date, Uniq AggregateFunction(uniq, UInt64)) ENGINE = MergeTre(d, d, 8192);" 2>&1 | grep "Maybe you meant: \['MergeTree'\]." &>/dev/null +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE stored_aggregates (d Date, Uniq AgregateFunction(uniq, UInt64)) ENGINE = MergeTree(d, d, 8192);" 2>&1 | grep "Maybe you meant: \['AggregateFunction'\]." &>/dev/null diff --git a/tests/queries/0_stateless/00910_crash_when_distributed_modify_order_by.sql b/tests/queries/0_stateless/00910_crash_when_distributed_modify_order_by.sql index 6dea97d1f27..00811d8ab89 100644 --- a/tests/queries/0_stateless/00910_crash_when_distributed_modify_order_by.sql +++ b/tests/queries/0_stateless/00910_crash_when_distributed_modify_order_by.sql @@ -2,6 +2,7 @@ DROP TABLE IF EXISTS union1; DROP TABLE IF EXISTS union2; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE union1 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = MergeTree(date, (a, date), 8192); CREATE TABLE union2 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost, currentDatabase(), 'union1'); ALTER TABLE union2 MODIFY ORDER BY a; -- { serverError 48 } diff --git a/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.sh b/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.sh index b0757829424..9e82cb4d30b 100755 --- a/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.sh +++ b/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.sh @@ -8,8 +8,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS replicated_optimize1;" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS replicated_optimize2;" -$CLICKHOUSE_CLIENT -q "CREATE TABLE replicated_optimize1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00925/optimize', 'r1', d, k, 8192);" -$CLICKHOUSE_CLIENT -q "CREATE TABLE replicated_optimize2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00925/optimize', 'r2', d, k, 8192);" +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE replicated_optimize1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00925/optimize', 'r1', d, k, 8192);" +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE replicated_optimize2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00925/optimize', 'r2', d, k, 8192);" num_tries=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT is_leader FROM system.replicas WHERE database=currentDatabase() AND table='replicated_optimize1'") -ne 1 ]]; do diff --git a/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql b/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql index 3a9a9154abe..05382b855b1 100644 --- a/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql +++ b/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql @@ -15,6 +15,7 @@ SET insert_allow_materialized_columns=0; SELECT 'insert_distributed_sync=0'; SET insert_distributed_sync=0; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE local_00952 (date Date, value Date MATERIALIZED toDate('2017-08-01')) ENGINE = MergeTree(date, date, 8192); CREATE TABLE distributed_00952 AS local_00952 ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), local_00952, rand()); diff --git a/tests/queries/0_stateless/00981_in_subquery_with_tuple.sh b/tests/queries/0_stateless/00981_in_subquery_with_tuple.sh index 99173062595..a899886e463 100755 --- a/tests/queries/0_stateless/00981_in_subquery_with_tuple.sh +++ b/tests/queries/0_stateless/00981_in_subquery_with_tuple.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bug"; -$CLICKHOUSE_CLIENT --query="CREATE TABLE bug (d Date, s String) ENGINE = MergeTree(d, s, 8192)"; +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE bug (d Date, s String) ENGINE = MergeTree(d, s, 8192)"; $CLICKHOUSE_CLIENT --query="INSERT INTO bug VALUES ('2019-08-09', 'hello'), ('2019-08-10', 'world'), ('2019-08-11', 'world'), ('2019-08-12', 'hello')"; #SET force_primary_key = 1; diff --git a/tests/queries/0_stateless/01008_materialized_view_henyihanwobushi.sql b/tests/queries/0_stateless/01008_materialized_view_henyihanwobushi.sql index 8def546cd2e..1e91f6daf65 100644 --- a/tests/queries/0_stateless/01008_materialized_view_henyihanwobushi.sql +++ b/tests/queries/0_stateless/01008_materialized_view_henyihanwobushi.sql @@ -2,6 +2,7 @@ DROP TABLE IF EXISTS foo; DROP TABLE IF EXISTS bar; DROP TABLE IF EXISTS view_foo_bar; +set allow_deprecated_syntax_for_merge_tree=1; create table foo (ddate Date, id Int64, n String) ENGINE = ReplacingMergeTree(ddate, (id), 8192); create table bar (ddate Date, id Int64, n String, foo_id Int64) ENGINE = ReplacingMergeTree(ddate, (id), 8192); insert into bar (id, n, foo_id) values (1, 'bar_n_1', 1); diff --git a/tests/queries/0_stateless/01053_drop_database_mat_view.sql b/tests/queries/0_stateless/01053_drop_database_mat_view.sql index e9936d7d3b2..7651ac4885c 100644 --- a/tests/queries/0_stateless/01053_drop_database_mat_view.sql +++ b/tests/queries/0_stateless/01053_drop_database_mat_view.sql @@ -4,6 +4,7 @@ DROP DATABASE IF EXISTS some_tests; set allow_deprecated_database_ordinary=1; CREATE DATABASE some_tests ENGINE=Ordinary; -- Different inner table name with Atomic +set allow_deprecated_syntax_for_merge_tree=1; create table some_tests.my_table ENGINE = MergeTree(day, (day), 8192) as select today() as day, 'mystring' as str; show tables from some_tests; create materialized view some_tests.my_materialized_view ENGINE = MergeTree(day, (day), 8192) as select * from some_tests.my_table; diff --git a/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.sql b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.sql index aba84f43033..3777ebb1af3 100644 --- a/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.sql +++ b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.sql @@ -61,6 +61,7 @@ DROP TABLE IF EXISTS test_alter_on_mutation; DROP TABLE IF EXISTS nested_alter; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE nested_alter (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT '0') ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01062/nested_alter', 'r2', d, k, 8192); INSERT INTO nested_alter VALUES ('2015-01-01', 6,38,'2014-07-15 13:26:50',[10,20,30],['asd','qwe','qwe'],['2000-01-01','2000-01-01','2000-01-03'],'100500'); diff --git a/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql b/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql index bbb607fe89f..992973c97e8 100644 --- a/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql +++ b/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql @@ -1,5 +1,6 @@ -- Tags: no-parallel +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE old_syntax_01071_test (date Date, id UInt8) ENGINE = MergeTree(date, id, 8192); ALTER TABLE old_syntax_01071_test ADD INDEX id_minmax id TYPE minmax GRANULARITY 1; -- { serverError 36 } CREATE TABLE new_syntax_01071_test (date Date, id UInt8) ENGINE = MergeTree() ORDER BY id; diff --git a/tests/queries/0_stateless/01076_predicate_optimizer_with_view.sql b/tests/queries/0_stateless/01076_predicate_optimizer_with_view.sql index d652cbeea5d..cfa25179d05 100644 --- a/tests/queries/0_stateless/01076_predicate_optimizer_with_view.sql +++ b/tests/queries/0_stateless/01076_predicate_optimizer_with_view.sql @@ -1,6 +1,7 @@ DROP TABLE IF EXISTS test; DROP TABLE IF EXISTS test_view; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test(date Date, id Int8, name String, value Int64) ENGINE = MergeTree(date, (id, date), 8192); CREATE VIEW test_view AS SELECT * FROM test; diff --git a/tests/queries/0_stateless/01089_alter_settings_old_format.sql b/tests/queries/0_stateless/01089_alter_settings_old_format.sql index 15d3ac508cb..7e7674f4d43 100644 --- a/tests/queries/0_stateless/01089_alter_settings_old_format.sql +++ b/tests/queries/0_stateless/01089_alter_settings_old_format.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS old_format_mt; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE old_format_mt ( event_date Date, key UInt64, diff --git a/tests/queries/0_stateless/01101_prewhere_after_alter.sql b/tests/queries/0_stateless/01101_prewhere_after_alter.sql index 11ebac7448d..976eb586a9a 100644 --- a/tests/queries/0_stateless/01101_prewhere_after_alter.sql +++ b/tests/queries/0_stateless/01101_prewhere_after_alter.sql @@ -1,6 +1,7 @@ DROP TABLE IF EXISTS test_a; DROP TABLE IF EXISTS test_b; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test_a ( OldColumn String DEFAULT '', diff --git a/tests/queries/0_stateless/01125_generate_random_qoega.sql b/tests/queries/0_stateless/01125_generate_random_qoega.sql index 7fb586ad2b5..9088e411a80 100644 --- a/tests/queries/0_stateless/01125_generate_random_qoega.sql +++ b/tests/queries/0_stateless/01125_generate_random_qoega.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS mass_table_117; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE mass_table_117 (`dt` Date, `site_id` Int32, `site_key` String) ENGINE = MergeTree(dt, (site_id, site_key, dt), 8192); INSERT INTO mass_table_117 SELECT * FROM generateRandom('`dt` Date,`site_id` Int32,`site_key` String', 1, 10, 2) LIMIT 100; SELECT count(), sum(cityHash64(*)) FROM mass_table_117; diff --git a/tests/queries/0_stateless/01126_month_partitioning_consistent_code.sql b/tests/queries/0_stateless/01126_month_partitioning_consistent_code.sql index c9bfbbe5111..f5f04178d5e 100644 --- a/tests/queries/0_stateless/01126_month_partitioning_consistent_code.sql +++ b/tests/queries/0_stateless/01126_month_partitioning_consistent_code.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS mt; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE mt (d Date, x UInt8) ENGINE = MergeTree(d, x, 8192); INSERT INTO mt VALUES (52392, 1), (62677, 2); DROP TABLE mt; diff --git a/tests/queries/0_stateless/01127_month_partitioning_consistency_select.sql b/tests/queries/0_stateless/01127_month_partitioning_consistency_select.sql index eb5ea091ca3..2a1d04e6074 100644 --- a/tests/queries/0_stateless/01127_month_partitioning_consistency_select.sql +++ b/tests/queries/0_stateless/01127_month_partitioning_consistency_select.sql @@ -1,6 +1,7 @@ -- Tags: no-parallel DROP TABLE IF EXISTS mt; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE mt (d Date, x String) ENGINE = MergeTree(d, x, 8192); INSERT INTO mt VALUES ('2106-02-07', 'Hello'), ('1970-01-01', 'World'); diff --git a/tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.sql b/tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.sql index 9659db7973d..97b0182f06f 100644 --- a/tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.sql +++ b/tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.sql @@ -1,6 +1,7 @@ DROP TABLE IF EXISTS test_table; DROP TABLE IF EXISTS test_table_sharded; +set allow_deprecated_syntax_for_merge_tree=1; create table test_table_sharded( date Date, diff --git a/tests/queries/0_stateless/01355_alter_column_with_order.sql b/tests/queries/0_stateless/01355_alter_column_with_order.sql index c0c85e389ea..0b1b4c42cce 100644 --- a/tests/queries/0_stateless/01355_alter_column_with_order.sql +++ b/tests/queries/0_stateless/01355_alter_column_with_order.sql @@ -2,6 +2,7 @@ DROP TABLE IF EXISTS alter_test; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alter_test (CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192); ALTER TABLE alter_test ADD COLUMN Added1 UInt32 FIRST; diff --git a/tests/queries/0_stateless/01387_clear_column_default_depends.sql b/tests/queries/0_stateless/01387_clear_column_default_depends.sql index 21a41f09df6..733daafa91c 100644 --- a/tests/queries/0_stateless/01387_clear_column_default_depends.sql +++ b/tests/queries/0_stateless/01387_clear_column_default_depends.sql @@ -31,6 +31,7 @@ DROP TABLE test; -- The original report from Mikhail Petrov DROP TABLE IF EXISTS Test; +set allow_deprecated_syntax_for_merge_tree=1; create table Test (impression_id String,impression_id_compressed FixedString(16) DEFAULT UUIDStringToNum(substring(impression_id, 1, 36)), impression_id_hashed UInt16 DEFAULT reinterpretAsUInt16(impression_id_compressed), event_date Date ) ENGINE = MergeTree(event_date, impression_id_hashed, (event_date, impression_id_hashed), 8192); alter table Test clear column impression_id in partition '202001'; DROP TABLE Test; diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose.sh b/tests/queries/0_stateless/01417_freeze_partition_verbose.sh index 1f67100a4b6..12f104b5337 100755 --- a/tests/queries/0_stateless/01417_freeze_partition_verbose.sh +++ b/tests/queries/0_stateless/01417_freeze_partition_verbose.sh @@ -17,7 +17,7 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze SELECT number, toStri # also for old syntax ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_old_syntax;" -${CLICKHOUSE_CLIENT} --query "CREATE TABLE table_for_freeze_old_syntax (dt Date, value String) ENGINE = MergeTree(dt, (value), 8192);" +${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query "CREATE TABLE table_for_freeze_old_syntax (dt Date, value String) ENGINE = MergeTree(dt, (value), 8192);" ${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze_old_syntax SELECT toDate('2021-03-01'), toString(number) from numbers(10);" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze FREEZE WITH NAME 'test_01417' FORMAT TSVWithNames SETTINGS alter_partition_verbose_result = 1;" \ diff --git a/tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.sql b/tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.sql index 223c27804df..752bc6b377f 100644 --- a/tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.sql +++ b/tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.sql @@ -37,6 +37,7 @@ ATTACH TABLE modify_sample_replicated; SELECT count(), min(y), max(y), sum(y), uniqExact(y) FROM modify_sample_replicated SAMPLE 0.1; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE modify_sample_old (d Date DEFAULT '2000-01-01', x UInt8, y UInt64) ENGINE = MergeTree(d, (x, y), 8192); ALTER TABLE modify_sample_old MODIFY SAMPLE BY x; -- { serverError 36 } diff --git a/tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.sql b/tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.sql index a271295a0cd..620daeb9f35 100644 --- a/tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.sql +++ b/tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.sql @@ -20,6 +20,7 @@ SELECT count() FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 100 DROP TABLE IF EXISTS test_local; DROP TABLE IF EXISTS test_distributed; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test_local (date Date, value UInt32) ENGINE = MergeTree(date, date, 8192); CREATE TABLE test_distributed AS test_local ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), test_local, rand()); diff --git a/tests/queries/0_stateless/01648_mutations_and_escaping.sql b/tests/queries/0_stateless/01648_mutations_and_escaping.sql index 689da842f16..18bd0b8ee3b 100644 --- a/tests/queries/0_stateless/01648_mutations_and_escaping.sql +++ b/tests/queries/0_stateless/01648_mutations_and_escaping.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS mutations_and_escaping_1648; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE mutations_and_escaping_1648 (d Date, e Enum8('foo'=1, 'bar'=2)) Engine = MergeTree(d, (d), 8192); INSERT INTO mutations_and_escaping_1648 (d, e) VALUES ('2018-01-01', 'foo'); INSERT INTO mutations_and_escaping_1648 (d, e) VALUES ('2018-01-02', 'bar'); diff --git a/tests/queries/0_stateless/01713_table_ttl_old_syntax_zookeeper.sql b/tests/queries/0_stateless/01713_table_ttl_old_syntax_zookeeper.sql index 1709a2d412a..11346a812f2 100644 --- a/tests/queries/0_stateless/01713_table_ttl_old_syntax_zookeeper.sql +++ b/tests/queries/0_stateless/01713_table_ttl_old_syntax_zookeeper.sql @@ -2,6 +2,7 @@ DROP TABLE IF EXISTS ttl_table; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE ttl_table ( date Date, diff --git a/tests/queries/0_stateless/01778_test_LowCardinality_FixedString_pk.sql b/tests/queries/0_stateless/01778_test_LowCardinality_FixedString_pk.sql index 1a0a1d35f76..78a9b35a40e 100644 --- a/tests/queries/0_stateless/01778_test_LowCardinality_FixedString_pk.sql +++ b/tests/queries/0_stateless/01778_test_LowCardinality_FixedString_pk.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS test_01778; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test_01778 ( `key` LowCardinality(FixedString(3)), diff --git a/tests/queries/0_stateless/01798_uniq_theta_sketch.sql b/tests/queries/0_stateless/01798_uniq_theta_sketch.sql index eace83d5cfa..59d5888c0a1 100644 --- a/tests/queries/0_stateless/01798_uniq_theta_sketch.sql +++ b/tests/queries/0_stateless/01798_uniq_theta_sketch.sql @@ -107,6 +107,7 @@ EXPLAIN SYNTAX select uniqTheta(-bitNot(-x)) from (select number % 2 as x from n DROP TABLE IF EXISTS stored_aggregates; -- simple +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE stored_aggregates ( d Date, diff --git a/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql b/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql index 7d151f063e1..9c6aa3146ee 100644 --- a/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql +++ b/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql @@ -19,6 +19,7 @@ DROP TABLE IF EXISTS broken_partition; DROP TABLE IF EXISTS old_partition_key; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE old_partition_key (sd Date, dh UInt64, ak UInt32, ed Date) ENGINE=MergeTree(sd, dh, (ak, ed, dh), 8192); ALTER TABLE old_partition_key DROP PARTITION ID '20210325_0_13241_6_12747'; --{serverError 248} diff --git a/tests/queries/0_stateless/02250_hints_for_columns.sh b/tests/queries/0_stateless/02250_hints_for_columns.sh index 45fd2f238b1..ea08d8c5535 100755 --- a/tests/queries/0_stateless/02250_hints_for_columns.sh +++ b/tests/queries/0_stateless/02250_hints_for_columns.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS t" -$CLICKHOUSE_CLIENT --query="CREATE TABLE t (CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192)" +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE t (CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192)" $CLICKHOUSE_CLIENT --query="ALTER TABLE t DROP COLUMN ToDro" 2>&1 | grep -q "Maybe you meant: \['ToDrop'\]" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/1_stateful/00040_aggregating_materialized_view.sql b/tests/queries/1_stateful/00040_aggregating_materialized_view.sql index 555a8b64d75..99fed62683f 100644 --- a/tests/queries/1_stateful/00040_aggregating_materialized_view.sql +++ b/tests/queries/1_stateful/00040_aggregating_materialized_view.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS test.basic_00040; +set allow_deprecated_syntax_for_merge_tree=1; CREATE MATERIALIZED VIEW test.basic_00040 ENGINE = AggregatingMergeTree(StartDate, (CounterID, StartDate), 8192) POPULATE AS diff --git a/tests/queries/1_stateful/00041_aggregating_materialized_view.sql b/tests/queries/1_stateful/00041_aggregating_materialized_view.sql index 0e59d2a88a9..ea99b78902c 100644 --- a/tests/queries/1_stateful/00041_aggregating_materialized_view.sql +++ b/tests/queries/1_stateful/00041_aggregating_materialized_view.sql @@ -9,6 +9,7 @@ CREATE TABLE test.visits_null UserID UInt64 ) ENGINE = Null; +set allow_deprecated_syntax_for_merge_tree=1; CREATE MATERIALIZED VIEW test.basic ENGINE = AggregatingMergeTree(StartDate, (CounterID, StartDate), 8192) AS SELECT diff --git a/tests/queries/1_stateful/00054_merge_tree_partitions.sql b/tests/queries/1_stateful/00054_merge_tree_partitions.sql index 73020952977..33f7e4e8666 100644 --- a/tests/queries/1_stateful/00054_merge_tree_partitions.sql +++ b/tests/queries/1_stateful/00054_merge_tree_partitions.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS test.partitions; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test.partitions (EventDate Date, CounterID UInt32) ENGINE = MergeTree(EventDate, CounterID, 8192); INSERT INTO test.partitions SELECT EventDate + UserID % 365 AS EventDate, CounterID FROM test.hits WHERE CounterID = 1704509; diff --git a/tests/queries/1_stateful/00071_merge_tree_optimize_aio.sql b/tests/queries/1_stateful/00071_merge_tree_optimize_aio.sql index 16c0097bf21..e0575122644 100644 --- a/tests/queries/1_stateful/00071_merge_tree_optimize_aio.sql +++ b/tests/queries/1_stateful/00071_merge_tree_optimize_aio.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS test.hits_snippet; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test.hits_snippet(EventTime DateTime('Asia/Dubai'), EventDate Date, CounterID UInt32, UserID UInt64, URL String, Referer String) ENGINE = MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192); SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; From cd8e5c7c49d4246014ce4cf423d087278745cfd3 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Jun 2022 17:43:54 +0800 Subject: [PATCH 13/72] update headers --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 3 +-- src/Storages/Hive/StorageHive.cpp | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index eefe4231b89..f922ff048b2 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -1,5 +1,4 @@ -#include -#include +#include "ArrowColumnToCHColumn.h" #if USE_ARROW || USE_ORC || USE_PARQUET diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 6d298c0033c..b717d373598 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -1,4 +1,3 @@ -#include #include #if USE_HIVE From c068593a0d7e34e4b4660a6e3d1f08cc0ee76e37 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Jun 2022 12:58:34 +0200 Subject: [PATCH 14/72] fix tests --- .../00148_summing_merge_tree_aggregate_function.sql | 1 + tests/queries/0_stateless/00214_primary_key_order.sql | 2 ++ tests/queries/0_stateless/00229_prewhere_column_missing.sql | 1 + .../0_stateless/00261_storage_aliases_and_array_join.sql | 1 + tests/queries/0_stateless/00262_alter_alias.sql | 1 + tests/queries/0_stateless/00276_sample.sql | 1 + tests/queries/0_stateless/00282_merging.sql | 2 ++ tests/queries/0_stateless/00294_shard_enums.sql | 1 + tests/queries/0_stateless/00311_array_primary_key.sql | 1 + .../0_stateless/00314_sample_factor_virtual_column.sql | 1 + tests/queries/0_stateless/00318_pk_tuple_order.sql | 1 + tests/queries/0_stateless/00319_index_for_like.sql | 1 + tests/queries/0_stateless/00321_pk_set.sql | 1 + .../queries/0_stateless/00327_summing_composite_nested.sql | 1 + tests/queries/0_stateless/00331_final_and_prewhere.sql | 1 + .../00361_shared_array_offsets_and_squash_blocks.sql | 1 + tests/queries/0_stateless/00363_defaults.sql | 1 + tests/queries/0_stateless/00386_enum_in_pk.sql | 1 + tests/queries/0_stateless/00392_enum_nested_alter.sql | 1 + .../0_stateless/00411_merge_tree_where_const_in_set.sql | 1 + .../0_stateless/00412_logical_expressions_optimizer.sql | 1 + .../00432_aggregate_function_scalars_and_constants.sql | 1 + .../0_stateless/00443_optimize_final_vertical_merge.sh | 2 +- .../queries/0_stateless/00443_preferred_block_size_bytes.sh | 6 +++--- tests/queries/0_stateless/00456_alter_nullable.sql | 2 ++ .../queries/0_stateless/00481_reading_from_last_granula.sql | 1 + .../0_stateless/00483_reading_from_array_structure.sql | 2 ++ tests/queries/0_stateless/00489_pk_subexpression.sql | 1 + tests/queries/0_stateless/00504_insert_miss_columns.sh | 4 ++-- .../00542_materialized_view_and_time_zone_tag.sql | 1 + .../0_stateless/00564_versioned_collapsing_merge_tree.sql | 1 + tests/queries/0_stateless/00565_enum_order.sh | 2 +- tests/queries/0_stateless/00579_virtual_column_and_lazy.sql | 1 + tests/queries/0_stateless/00614_array_nullable.sql | 1 + tests/queries/0_stateless/00616_final_single_part.sql | 1 + tests/queries/0_stateless/00623_truncate_table.sql | 2 ++ .../queries/0_stateless/00625_summing_merge_tree_merge.sql | 1 + .../0_stateless/00709_virtual_column_partition_id.sql | 1 + .../queries/0_stateless/00974_final_predicate_push_down.sql | 1 + tests/queries/0_stateless/01109_exchange_tables.sql | 1 + tests/queries/0_stateless/01128_generate_random_nested.sql | 2 ++ tests/queries/0_stateless/01190_full_attach_syntax.sql | 1 + tests/queries/0_stateless/01297_alter_distributed.sql | 1 + tests/queries/0_stateless/01299_alter_merge_tree.sql | 1 + tests/queries/0_stateless/01652_ttl_old_syntax.sql | 1 + tests/queries/0_stateless/01907_multiple_aliases.sql | 1 + .../0_stateless/02021_create_database_with_comment.sh | 2 +- tests/queries/0_stateless/02135_local_create_db.sh | 2 +- .../0_stateless/02265_rename_join_ordinary_to_atomic.sql | 1 + 49 files changed, 58 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00148_summing_merge_tree_aggregate_function.sql b/tests/queries/0_stateless/00148_summing_merge_tree_aggregate_function.sql index 04edf709bde..649c09dbbf1 100644 --- a/tests/queries/0_stateless/00148_summing_merge_tree_aggregate_function.sql +++ b/tests/queries/0_stateless/00148_summing_merge_tree_aggregate_function.sql @@ -2,6 +2,7 @@ drop table if exists summing_merge_tree_aggregate_function; drop table if exists summing_merge_tree_null; ---- partition merge +set allow_deprecated_syntax_for_merge_tree=1; create table summing_merge_tree_aggregate_function ( d Date, k UInt64, diff --git a/tests/queries/0_stateless/00214_primary_key_order.sql b/tests/queries/0_stateless/00214_primary_key_order.sql index 3c751e63e6d..e8a3be5f8dc 100644 --- a/tests/queries/0_stateless/00214_primary_key_order.sql +++ b/tests/queries/0_stateless/00214_primary_key_order.sql @@ -1,4 +1,6 @@ DROP TABLE IF EXISTS primary_key; + +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE primary_key (d Date DEFAULT today(), x Int8) ENGINE = MergeTree(d, -x, 1); INSERT INTO primary_key (x) VALUES (1), (2), (3); diff --git a/tests/queries/0_stateless/00229_prewhere_column_missing.sql b/tests/queries/0_stateless/00229_prewhere_column_missing.sql index 324e37bfce7..1fb74b04af0 100644 --- a/tests/queries/0_stateless/00229_prewhere_column_missing.sql +++ b/tests/queries/0_stateless/00229_prewhere_column_missing.sql @@ -1,5 +1,6 @@ drop table if exists prewhere_column_missing; +set allow_deprecated_syntax_for_merge_tree=1; create table prewhere_column_missing (d Date default '2015-01-01', x UInt64) engine=MergeTree(d, x, 1); insert into prewhere_column_missing (x) values (0); diff --git a/tests/queries/0_stateless/00261_storage_aliases_and_array_join.sql b/tests/queries/0_stateless/00261_storage_aliases_and_array_join.sql index 71562ffd6fa..bb3376a9e61 100644 --- a/tests/queries/0_stateless/00261_storage_aliases_and_array_join.sql +++ b/tests/queries/0_stateless/00261_storage_aliases_and_array_join.sql @@ -1,5 +1,6 @@ drop table if exists aliases_test; +set allow_deprecated_syntax_for_merge_tree=1; create table aliases_test ( date Date, id UInt64, array default ['zero','one','two'], diff --git a/tests/queries/0_stateless/00262_alter_alias.sql b/tests/queries/0_stateless/00262_alter_alias.sql index 56dbda65be7..1c19f8636d1 100644 --- a/tests/queries/0_stateless/00262_alter_alias.sql +++ b/tests/queries/0_stateless/00262_alter_alias.sql @@ -1,5 +1,6 @@ drop table if exists aliases_test; +set allow_deprecated_syntax_for_merge_tree=1; create table aliases_test (date default today(), id default rand(), array default [0, 1, 2]) engine=MergeTree(date, id, 1); insert into aliases_test (id) values (0); diff --git a/tests/queries/0_stateless/00276_sample.sql b/tests/queries/0_stateless/00276_sample.sql index cd28f18b2b8..b75ed188ec4 100644 --- a/tests/queries/0_stateless/00276_sample.sql +++ b/tests/queries/0_stateless/00276_sample.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS sample_00276; +set allow_deprecated_syntax_for_merge_tree=1; SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; SET max_block_size = 10; diff --git a/tests/queries/0_stateless/00282_merging.sql b/tests/queries/0_stateless/00282_merging.sql index a49cde87134..f4a3708eedf 100644 --- a/tests/queries/0_stateless/00282_merging.sql +++ b/tests/queries/0_stateless/00282_merging.sql @@ -1,4 +1,6 @@ DROP TABLE IF EXISTS merge; + +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE IF NOT EXISTS merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5); INSERT INTO merge (x) VALUES (1), (2), (3); diff --git a/tests/queries/0_stateless/00294_shard_enums.sql b/tests/queries/0_stateless/00294_shard_enums.sql index 414b9cfad03..dcd74ac3e3a 100644 --- a/tests/queries/0_stateless/00294_shard_enums.sql +++ b/tests/queries/0_stateless/00294_shard_enums.sql @@ -3,6 +3,7 @@ set max_threads = 1; drop table if exists enums; +set allow_deprecated_syntax_for_merge_tree=1; create table enums ( d Date default '2015-12-29', k default 0, e Enum8('world' = 2, 'hello' = 1), sign Enum8('minus' = -1, 'plus' = 1), diff --git a/tests/queries/0_stateless/00311_array_primary_key.sql b/tests/queries/0_stateless/00311_array_primary_key.sql index f6e21beab9b..348ef2d1c6f 100644 --- a/tests/queries/0_stateless/00311_array_primary_key.sql +++ b/tests/queries/0_stateless/00311_array_primary_key.sql @@ -1,5 +1,6 @@ -- Tags: no-parallel +set allow_deprecated_syntax_for_merge_tree=1; DROP TABLE IF EXISTS array_pk; CREATE TABLE array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); diff --git a/tests/queries/0_stateless/00314_sample_factor_virtual_column.sql b/tests/queries/0_stateless/00314_sample_factor_virtual_column.sql index b0ed4fdedcb..6e3dc019069 100644 --- a/tests/queries/0_stateless/00314_sample_factor_virtual_column.sql +++ b/tests/queries/0_stateless/00314_sample_factor_virtual_column.sql @@ -2,6 +2,7 @@ DROP TABLE IF EXISTS sample_00314_1; DROP TABLE IF EXISTS sample_00314_2; DROP TABLE IF EXISTS sample_merge_00314; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE sample_00314_1 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10); CREATE TABLE sample_00314_2 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10); diff --git a/tests/queries/0_stateless/00318_pk_tuple_order.sql b/tests/queries/0_stateless/00318_pk_tuple_order.sql index 8e2992167ee..585f35d2f3c 100644 --- a/tests/queries/0_stateless/00318_pk_tuple_order.sql +++ b/tests/queries/0_stateless/00318_pk_tuple_order.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS pk; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE pk (d Date DEFAULT '2000-01-01', x UInt64, y UInt64, z UInt64) ENGINE = MergeTree(d, (x, y, z), 1); INSERT INTO pk (x, y, z) VALUES (1, 11, 1235), (1, 11, 4395), (1, 22, 3545), (1, 22, 6984), (1, 33, 4596), (2, 11, 4563), (2, 11, 4578), (2, 11, 3572), (2, 22, 5786), (2, 22, 5786), (2, 22, 2791), (2, 22, 2791), (3, 33, 2791), (3, 33, 2791), (3, 33, 1235), (3, 44, 4935), (3, 44, 4578), (3, 55, 5786), (3, 55, 2791), (3, 55, 1235); diff --git a/tests/queries/0_stateless/00319_index_for_like.sql b/tests/queries/0_stateless/00319_index_for_like.sql index 57ebce439f3..e490e595142 100644 --- a/tests/queries/0_stateless/00319_index_for_like.sql +++ b/tests/queries/0_stateless/00319_index_for_like.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS index_for_like; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE index_for_like (s String, d Date DEFAULT today()) ENGINE = MergeTree(d, (s, d), 1); INSERT INTO index_for_like (s) VALUES ('Hello'), ('Hello, World'), ('Hello, World 1'), ('Hello 1'), ('Goodbye'), ('Goodbye, World'), ('Goodbye 1'), ('Goodbye, World 1'); diff --git a/tests/queries/0_stateless/00321_pk_set.sql b/tests/queries/0_stateless/00321_pk_set.sql index 073a87a6e13..bf61a684ac7 100644 --- a/tests/queries/0_stateless/00321_pk_set.sql +++ b/tests/queries/0_stateless/00321_pk_set.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS pk_set; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE pk_set (d Date, n UInt64, host String, code UInt64) ENGINE = MergeTree(d, (n, host, code), 1); INSERT INTO pk_set (n, host, code) VALUES (1, 'market', 100), (11, 'news', 100); diff --git a/tests/queries/0_stateless/00327_summing_composite_nested.sql b/tests/queries/0_stateless/00327_summing_composite_nested.sql index f9b251ebd8f..701735a7168 100644 --- a/tests/queries/0_stateless/00327_summing_composite_nested.sql +++ b/tests/queries/0_stateless/00327_summing_composite_nested.sql @@ -1,6 +1,7 @@ SET optimize_on_insert = 0; DROP TABLE IF EXISTS summing_composite_key; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE summing_composite_key (d Date, k UInt64, FirstMap Nested(k1 UInt32, k2ID Int8, s Float64), SecondMap Nested(k1ID UInt64, k2Key String, k3Type Int32, s Int64)) ENGINE = SummingMergeTree(d, k, 1); INSERT INTO summing_composite_key VALUES ('2000-01-01', 1, [1,2], ['3','4'], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]), ('2000-01-01', 1, [2,1], ['4','3'], [20,22], [2,2,1], [5,5,0], [-3,-3,-33], [10,100,1000]), ('2000-01-01', 2, [1,2], ['3','4'], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]), ('2000-01-01', 2, [2,1,1], ['4','3','3'], [20,22,33], [2,2], [5,5], [-3,-3], [10,100]), ('2000-01-01', 2, [1,2], ['3','4'], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]); diff --git a/tests/queries/0_stateless/00331_final_and_prewhere.sql b/tests/queries/0_stateless/00331_final_and_prewhere.sql index 02af6d9b450..5d0b80d6363 100644 --- a/tests/queries/0_stateless/00331_final_and_prewhere.sql +++ b/tests/queries/0_stateless/00331_final_and_prewhere.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS replace; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE replace ( EventDate Date, Id UInt64, Data String, Version UInt32) ENGINE = ReplacingMergeTree(EventDate, Id, 8192, Version); INSERT INTO replace VALUES ('2016-06-02', 1, 'version 1', 1); INSERT INTO replace VALUES ('2016-06-02', 2, 'version 1', 1); diff --git a/tests/queries/0_stateless/00361_shared_array_offsets_and_squash_blocks.sql b/tests/queries/0_stateless/00361_shared_array_offsets_and_squash_blocks.sql index 6160edd05b7..7815a35259a 100644 --- a/tests/queries/0_stateless/00361_shared_array_offsets_and_squash_blocks.sql +++ b/tests/queries/0_stateless/00361_shared_array_offsets_and_squash_blocks.sql @@ -1,6 +1,7 @@ DROP TABLE IF EXISTS nested1; DROP TABLE IF EXISTS nested2; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE nested1 (d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, b String)) ENGINE = MergeTree(d, x, 1); INSERT INTO nested1 (x, n.a, n.b) VALUES (1, ['Hello', 'World'], ['abc', 'def']), (2, [], []); diff --git a/tests/queries/0_stateless/00363_defaults.sql b/tests/queries/0_stateless/00363_defaults.sql index 4ebcc7b0f61..1ec3b13a130 100644 --- a/tests/queries/0_stateless/00363_defaults.sql +++ b/tests/queries/0_stateless/00363_defaults.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS prewhere_defaults; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE prewhere_defaults (d Date DEFAULT '2000-01-01', k UInt64 DEFAULT 0, x UInt16) ENGINE = MergeTree(d, k, 1); INSERT INTO prewhere_defaults (x) VALUES (1); diff --git a/tests/queries/0_stateless/00386_enum_in_pk.sql b/tests/queries/0_stateless/00386_enum_in_pk.sql index 75b8a166523..4fc79b5ef1b 100644 --- a/tests/queries/0_stateless/00386_enum_in_pk.sql +++ b/tests/queries/0_stateless/00386_enum_in_pk.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS enum_pk; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE enum_pk (date Date DEFAULT '0000-00-00', x Enum8('0' = 0, '1' = 1, '2' = 2), d Enum8('0' = 0, '1' = 1, '2' = 2)) ENGINE = MergeTree(date, x, 1); INSERT INTO enum_pk (x, d) VALUES ('0', '0')('1', '1')('0', '0')('1', '1')('1', '1')('0', '0')('0', '0')('2', '2')('0', '0')('1', '1')('1', '1')('1', '1')('1', '1')('0', '0'); diff --git a/tests/queries/0_stateless/00392_enum_nested_alter.sql b/tests/queries/0_stateless/00392_enum_nested_alter.sql index 205b9a7fec0..b5989885de4 100644 --- a/tests/queries/0_stateless/00392_enum_nested_alter.sql +++ b/tests/queries/0_stateless/00392_enum_nested_alter.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS enum_nested_alter; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE enum_nested_alter (d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, e Enum8('Hello' = 1), b UInt8)) ENGINE = MergeTree(d, x, 1); diff --git a/tests/queries/0_stateless/00411_merge_tree_where_const_in_set.sql b/tests/queries/0_stateless/00411_merge_tree_where_const_in_set.sql index 614f838c2eb..22779509a3d 100644 --- a/tests/queries/0_stateless/00411_merge_tree_where_const_in_set.sql +++ b/tests/queries/0_stateless/00411_merge_tree_where_const_in_set.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS const_in_const; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE const_in_const (id UInt64, date Date, uid UInt32, name String, Sign Int8) ENGINE = CollapsingMergeTree(date, intHash32(uid), (id, date, intHash32(uid)), 8192, Sign); INSERT INTO const_in_const VALUES(1, now(), 1, 'test1', 1); INSERT INTO const_in_const VALUES(2, now(), 1, 'test2', 1); diff --git a/tests/queries/0_stateless/00412_logical_expressions_optimizer.sql b/tests/queries/0_stateless/00412_logical_expressions_optimizer.sql index 5f8b15e980a..c4fad7d5064 100644 --- a/tests/queries/0_stateless/00412_logical_expressions_optimizer.sql +++ b/tests/queries/0_stateless/00412_logical_expressions_optimizer.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS merge_tree; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE merge_tree (x UInt64, date Date) ENGINE = MergeTree(date, x, 1); INSERT INTO merge_tree VALUES (1, '2000-01-01'); diff --git a/tests/queries/0_stateless/00432_aggregate_function_scalars_and_constants.sql b/tests/queries/0_stateless/00432_aggregate_function_scalars_and_constants.sql index c74b4f03371..a6f31b9357c 100644 --- a/tests/queries/0_stateless/00432_aggregate_function_scalars_and_constants.sql +++ b/tests/queries/0_stateless/00432_aggregate_function_scalars_and_constants.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS agg_func_col; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE agg_func_col (p Date, k UInt8, d AggregateFunction(sum, UInt64) DEFAULT arrayReduce('sumState', [toUInt64(200)])) ENGINE = AggregatingMergeTree(p, k, 1); INSERT INTO agg_func_col (k) VALUES (0); INSERT INTO agg_func_col (k, d) SELECT 1 AS k, arrayReduce('sumState', [toUInt64(100)]) AS d; diff --git a/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh b/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh index a32dde58164..3bdc5892ced 100755 --- a/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh +++ b/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh @@ -16,7 +16,7 @@ function get_num_parts { $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $name" -$CLICKHOUSE_CLIENT -q "CREATE TABLE $name ( +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE $name ( date Date, Sign Int8, ki UInt64, diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index 399a4677a44..5dcc2f3c181 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" -$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO preferred_block_size_bytes (s) SELECT '16_bytes_-_-_-_' AS s FROM system.numbers LIMIT 10, 90" $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE preferred_block_size_bytes" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=26 -q "SELECT DISTINCT blockSize(), ignore(p, s) FROM preferred_block_size_bytes" @@ -18,7 +18,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" # PREWHERE using empty column $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS pbs" -$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO pbs (p, i, sa) SELECT toDate(i % 30) AS p, number AS i, ['a'] AS sa FROM system.numbers LIMIT 1000" $CLICKHOUSE_CLIENT -q "ALTER TABLE pbs ADD COLUMN s UInt8 DEFAULT 0" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=100000 -q "SELECT count() FROM pbs PREWHERE s = 0" @@ -29,7 +29,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE pbs" # Nullable PREWHERE $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" -$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1-q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO nullable_prewhere SELECT toDate(0) AS p, if(number % 2 = 0, CAST(number AS Nullable(UInt64)), CAST(NULL AS Nullable(UInt64))) AS f, number as d FROM system.numbers LIMIT 1001" $CLICKHOUSE_CLIENT -q "SELECT sum(d), sum(f), max(d) FROM nullable_prewhere PREWHERE NOT isNull(f)" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" diff --git a/tests/queries/0_stateless/00456_alter_nullable.sql b/tests/queries/0_stateless/00456_alter_nullable.sql index 703d1a551a7..0fa3837767d 100644 --- a/tests/queries/0_stateless/00456_alter_nullable.sql +++ b/tests/queries/0_stateless/00456_alter_nullable.sql @@ -1,4 +1,6 @@ DROP TABLE IF EXISTS nullable_alter; + +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE nullable_alter (d Date DEFAULT '2000-01-01', x String) ENGINE = MergeTree(d, d, 1); INSERT INTO nullable_alter (x) VALUES ('Hello'), ('World'); diff --git a/tests/queries/0_stateless/00481_reading_from_last_granula.sql b/tests/queries/0_stateless/00481_reading_from_last_granula.sql index 29d42e41e14..c98068e466b 100644 --- a/tests/queries/0_stateless/00481_reading_from_last_granula.sql +++ b/tests/queries/0_stateless/00481_reading_from_last_granula.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS tab_00481; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE tab_00481 (date Date, value UInt64, s String, m FixedString(16)) ENGINE = MergeTree(date, (date, value), 8); INSERT INTO tab_00481 SELECT today() as date, number as value, '' as s, toFixedString('', 16) as m from system.numbers limit 42; SET preferred_max_column_in_block_size_bytes = 32; diff --git a/tests/queries/0_stateless/00483_reading_from_array_structure.sql b/tests/queries/0_stateless/00483_reading_from_array_structure.sql index 5ba152ef9b6..bab0dcd3707 100644 --- a/tests/queries/0_stateless/00483_reading_from_array_structure.sql +++ b/tests/queries/0_stateless/00483_reading_from_array_structure.sql @@ -1,4 +1,6 @@ drop table if exists `table_00483`; + +set allow_deprecated_syntax_for_merge_tree=1; create table `table_00483` (date Date, `Struct.Key1` Array(UInt64), `Struct.Key2` Array(UInt64), padding FixedString(16)) engine = MergeTree(date, (date), 16); insert into `table_00483` select today() as date, [number], [number + 1], toFixedString('', 16) from system.numbers limit 100; set preferred_max_column_in_block_size_bytes = 96; diff --git a/tests/queries/0_stateless/00489_pk_subexpression.sql b/tests/queries/0_stateless/00489_pk_subexpression.sql index 41499f0bd1b..6f76a13609c 100644 --- a/tests/queries/0_stateless/00489_pk_subexpression.sql +++ b/tests/queries/0_stateless/00489_pk_subexpression.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS pk; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE pk (d Date DEFAULT '2000-01-01', x DateTime, y UInt64, z UInt64) ENGINE = MergeTree(d, (toStartOfMinute(x), y, z), 1); INSERT INTO pk (x, y, z) VALUES (1, 11, 1235), (2, 11, 4395), (3, 22, 3545), (4, 22, 6984), (5, 33, 4596), (61, 11, 4563), (62, 11, 4578), (63, 11, 3572), (64, 22, 5786), (65, 22, 5786), (66, 22, 2791), (67, 22, 2791), (121, 33, 2791), (122, 33, 2791), (123, 33, 1235), (124, 44, 4935), (125, 44, 4578), (126, 55, 5786), (127, 55, 2791), (128, 55, 1235); diff --git a/tests/queries/0_stateless/00504_insert_miss_columns.sh b/tests/queries/0_stateless/00504_insert_miss_columns.sh index ea699ab58a5..fa2cc9d0b22 100755 --- a/tests/queries/0_stateless/00504_insert_miss_columns.sh +++ b/tests/queries/0_stateless/00504_insert_miss_columns.sh @@ -8,8 +8,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS advertiser"; $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS advertiser_test"; -$CLICKHOUSE_CLIENT -q "CREATE TABLE advertiser ( action_date Date, adblock UInt8, imps Int64 ) Engine = SummingMergeTree( action_date, ( adblock ), 8192, ( imps ) )"; -$CLICKHOUSE_CLIENT -q "CREATE TABLE advertiser_test ( action_date Date, adblock UInt8, imps Int64, Hash UInt64 ) Engine = SummingMergeTree( action_date, ( adblock, Hash ), 8192, ( imps ) )"; +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE advertiser ( action_date Date, adblock UInt8, imps Int64 ) Engine = SummingMergeTree( action_date, ( adblock ), 8192, ( imps ) )"; +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE advertiser_test ( action_date Date, adblock UInt8, imps Int64, Hash UInt64 ) Engine = SummingMergeTree( action_date, ( adblock, Hash ), 8192, ( imps ) )"; # This test will fail. It's ok. $CLICKHOUSE_CLIENT -q "INSERT INTO advertiser_test SELECT *, sipHash64( CAST(adblock AS String) ), CAST(1 AS Int8) FROM advertiser;" 2>/dev/null diff --git a/tests/queries/0_stateless/00542_materialized_view_and_time_zone_tag.sql b/tests/queries/0_stateless/00542_materialized_view_and_time_zone_tag.sql index 5e9277c90b6..88808ac20f9 100644 --- a/tests/queries/0_stateless/00542_materialized_view_and_time_zone_tag.sql +++ b/tests/queries/0_stateless/00542_materialized_view_and_time_zone_tag.sql @@ -2,6 +2,7 @@ DROP TABLE IF EXISTS m3; DROP TABLE IF EXISTS m1; DROP TABLE IF EXISTS x; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE x (d Date, t DateTime) ENGINE = MergeTree(d, (d, t), 1); CREATE MATERIALIZED VIEW m1 (d Date, t DateTime, c UInt64) ENGINE = SummingMergeTree(d, (d, t), 1) AS SELECT d, toStartOfMinute(x.t) as t, count() as c FROM x GROUP BY d, t; diff --git a/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql b/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql index 66bbb02183c..fdee9390642 100644 --- a/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql @@ -1,5 +1,6 @@ -- Tags: no-parallel +set allow_deprecated_syntax_for_merge_tree=1; set optimize_on_insert = 0; drop table if exists mult_tab; diff --git a/tests/queries/0_stateless/00565_enum_order.sh b/tests/queries/0_stateless/00565_enum_order.sh index 6958a403246..fedd81b91f8 100755 --- a/tests/queries/0_stateless/00565_enum_order.sh +++ b/tests/queries/0_stateless/00565_enum_order.sh @@ -10,7 +10,7 @@ $CLICKHOUSE_CLIENT <<"EOF" DROP TABLE IF EXISTS `test_log` EOF -$CLICKHOUSE_CLIENT <<"EOF" +$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 <<"EOF" CREATE TABLE `test_log` ( date Date, datetime DateTime, diff --git a/tests/queries/0_stateless/00579_virtual_column_and_lazy.sql b/tests/queries/0_stateless/00579_virtual_column_and_lazy.sql index b1f1ec4cfb2..ca58a5fc93b 100644 --- a/tests/queries/0_stateless/00579_virtual_column_and_lazy.sql +++ b/tests/queries/0_stateless/00579_virtual_column_and_lazy.sql @@ -2,6 +2,7 @@ DROP TABLE IF EXISTS sample_00579_1; DROP TABLE IF EXISTS sample_00579_2; DROP TABLE IF EXISTS sample_merge_00579; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE sample_00579_1 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10); CREATE TABLE sample_00579_2 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10); diff --git a/tests/queries/0_stateless/00614_array_nullable.sql b/tests/queries/0_stateless/00614_array_nullable.sql index d62ddb8242c..1cbfbf128cb 100644 --- a/tests/queries/0_stateless/00614_array_nullable.sql +++ b/tests/queries/0_stateless/00614_array_nullable.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS test; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test(date Date, keys Array(Nullable(UInt8))) ENGINE = MergeTree(date, date, 1); INSERT INTO test VALUES ('2017-09-10', [1, 2, 3, 4, 5, 6, 7, NULL]); SELECT * FROM test LIMIT 1; diff --git a/tests/queries/0_stateless/00616_final_single_part.sql b/tests/queries/0_stateless/00616_final_single_part.sql index 6618d0b1252..605f2c8b216 100644 --- a/tests/queries/0_stateless/00616_final_single_part.sql +++ b/tests/queries/0_stateless/00616_final_single_part.sql @@ -12,6 +12,7 @@ CREATE TABLE test_00616 ENGINE = MergeTree(date, x, 4096); INSERT INTO test_00616 VALUES ('2018-03-21', 1, 1), ('2018-03-21', 1, 2); +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE replacing_00616 ENGINE = ReplacingMergeTree(date, x, 4096, ver) AS SELECT * FROM test_00616; SELECT * FROM test_00616 ORDER BY ver; diff --git a/tests/queries/0_stateless/00623_truncate_table.sql b/tests/queries/0_stateless/00623_truncate_table.sql index 80369ff2bbc..16bc87ea213 100644 --- a/tests/queries/0_stateless/00623_truncate_table.sql +++ b/tests/queries/0_stateless/00623_truncate_table.sql @@ -1,5 +1,7 @@ -- Tags: no-parallel +set allow_deprecated_syntax_for_merge_tree=1; + DROP DATABASE IF EXISTS truncate_test; DROP TABLE IF EXISTS truncate_test_log; DROP TABLE IF EXISTS truncate_test_memory; diff --git a/tests/queries/0_stateless/00625_summing_merge_tree_merge.sql b/tests/queries/0_stateless/00625_summing_merge_tree_merge.sql index a214347a3a7..235048ad18b 100644 --- a/tests/queries/0_stateless/00625_summing_merge_tree_merge.sql +++ b/tests/queries/0_stateless/00625_summing_merge_tree_merge.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS tab_00625; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE tab_00625 ( date Date, diff --git a/tests/queries/0_stateless/00709_virtual_column_partition_id.sql b/tests/queries/0_stateless/00709_virtual_column_partition_id.sql index 084ab904d87..48a3a3fad6a 100644 --- a/tests/queries/0_stateless/00709_virtual_column_partition_id.sql +++ b/tests/queries/0_stateless/00709_virtual_column_partition_id.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS partition_id; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE IF NOT EXISTS partition_id (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5); INSERT INTO partition_id VALUES (100, 1), (200, 2), (300, 3); diff --git a/tests/queries/0_stateless/00974_final_predicate_push_down.sql b/tests/queries/0_stateless/00974_final_predicate_push_down.sql index 96bcbf9aae6..7a6378692f2 100644 --- a/tests/queries/0_stateless/00974_final_predicate_push_down.sql +++ b/tests/queries/0_stateless/00974_final_predicate_push_down.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS test_00974; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test_00974 ( date Date, diff --git a/tests/queries/0_stateless/01109_exchange_tables.sql b/tests/queries/0_stateless/01109_exchange_tables.sql index e6900745fed..c118945887b 100644 --- a/tests/queries/0_stateless/01109_exchange_tables.sql +++ b/tests/queries/0_stateless/01109_exchange_tables.sql @@ -30,6 +30,7 @@ SELECT * FROM t2; DROP DATABASE IF EXISTS test_01109_other_atomic; DROP DATABASE IF EXISTS test_01109_ordinary; CREATE DATABASE test_01109_other_atomic; +set allow_deprecated_database_ordinary=1; CREATE DATABASE test_01109_ordinary ENGINE=Ordinary; CREATE TABLE test_01109_other_atomic.t3 ENGINE=MergeTree() ORDER BY tuple() diff --git a/tests/queries/0_stateless/01128_generate_random_nested.sql b/tests/queries/0_stateless/01128_generate_random_nested.sql index 2af52e69893..8098db89491 100644 --- a/tests/queries/0_stateless/01128_generate_random_nested.sql +++ b/tests/queries/0_stateless/01128_generate_random_nested.sql @@ -1,4 +1,6 @@ DROP TABLE IF EXISTS mass_table_312; + +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE mass_table_312 (d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, b String)) ENGINE = MergeTree(d, x, 1); INSERT INTO mass_table_312 SELECT * FROM generateRandom('`d` Date,`x` UInt64,`n.a` Array(String),`n.b` Array(String)', 1, 10, 2) LIMIT 100; diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.sql b/tests/queries/0_stateless/01190_full_attach_syntax.sql index e0ffe7ede66..e66978e22e1 100644 --- a/tests/queries/0_stateless/01190_full_attach_syntax.sql +++ b/tests/queries/0_stateless/01190_full_attach_syntax.sql @@ -2,6 +2,7 @@ DROP DATABASE IF EXISTS test_01190; set allow_deprecated_database_ordinary=1; +set allow_deprecated_syntax_for_merge_tree=1; CREATE DATABASE test_01190 ENGINE=Ordinary; -- Full ATTACH requires UUID with Atomic USE test_01190; diff --git a/tests/queries/0_stateless/01297_alter_distributed.sql b/tests/queries/0_stateless/01297_alter_distributed.sql index cec64278d47..c79d98b7b3b 100644 --- a/tests/queries/0_stateless/01297_alter_distributed.sql +++ b/tests/queries/0_stateless/01297_alter_distributed.sql @@ -3,6 +3,7 @@ drop table if exists merge_distributed; drop table if exists merge_distributed1; +set allow_deprecated_syntax_for_merge_tree=1; create table merge_distributed1 ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); insert into merge_distributed1 values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); diff --git a/tests/queries/0_stateless/01299_alter_merge_tree.sql b/tests/queries/0_stateless/01299_alter_merge_tree.sql index 87608e6d15a..3c4467926f8 100644 --- a/tests/queries/0_stateless/01299_alter_merge_tree.sql +++ b/tests/queries/0_stateless/01299_alter_merge_tree.sql @@ -1,5 +1,6 @@ drop table if exists merge_tree; +set allow_deprecated_syntax_for_merge_tree=1; create table merge_tree ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); insert into merge_tree values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3) diff --git a/tests/queries/0_stateless/01652_ttl_old_syntax.sql b/tests/queries/0_stateless/01652_ttl_old_syntax.sql index 05c391b85e5..7b11247d968 100644 --- a/tests/queries/0_stateless/01652_ttl_old_syntax.sql +++ b/tests/queries/0_stateless/01652_ttl_old_syntax.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS ttl_old_syntax; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE ttl_old_syntax (d Date, i Int) ENGINE = MergeTree(d, i, 8291); ALTER TABLE ttl_old_syntax MODIFY TTL toDate('2020-01-01'); -- { serverError 36 } diff --git a/tests/queries/0_stateless/01907_multiple_aliases.sql b/tests/queries/0_stateless/01907_multiple_aliases.sql index 611960a5205..5e8efba7ab7 100644 --- a/tests/queries/0_stateless/01907_multiple_aliases.sql +++ b/tests/queries/0_stateless/01907_multiple_aliases.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS t; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE t (d Date, z UInt32) ENGINE = MergeTree(d, (z), 1); INSERT INTO t VALUES ('2017-01-01', 1); diff --git a/tests/queries/0_stateless/02021_create_database_with_comment.sh b/tests/queries/0_stateless/02021_create_database_with_comment.sh index ce7a7bef066..11e62e790b7 100755 --- a/tests/queries/0_stateless/02021_create_database_with_comment.sh +++ b/tests/queries/0_stateless/02021_create_database_with_comment.sh @@ -18,7 +18,7 @@ function test_db_comments() local ENGINE_NAME="$1" echo "engine : ${ENGINE_NAME}" - $CLICKHOUSE_CLIENT -nm < Date: Thu, 23 Jun 2022 14:11:16 +0200 Subject: [PATCH 15/72] fix tests --- .../queries/0_stateless/00443_preferred_block_size_bytes.sh | 6 +++--- tests/queries/0_stateless/00616_final_single_part.sql | 1 + tests/queries/0_stateless/01162_strange_mutations.sh | 2 +- tests/queries/0_stateless/01298_alter_merge.sql | 1 + .../0_stateless/01528_clickhouse_local_prepare_parts.sh | 2 +- 5 files changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index 5dcc2f3c181..399a4677a44 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" -$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO preferred_block_size_bytes (s) SELECT '16_bytes_-_-_-_' AS s FROM system.numbers LIMIT 10, 90" $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE preferred_block_size_bytes" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=26 -q "SELECT DISTINCT blockSize(), ignore(p, s) FROM preferred_block_size_bytes" @@ -18,7 +18,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" # PREWHERE using empty column $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS pbs" -$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO pbs (p, i, sa) SELECT toDate(i % 30) AS p, number AS i, ['a'] AS sa FROM system.numbers LIMIT 1000" $CLICKHOUSE_CLIENT -q "ALTER TABLE pbs ADD COLUMN s UInt8 DEFAULT 0" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=100000 -q "SELECT count() FROM pbs PREWHERE s = 0" @@ -29,7 +29,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE pbs" # Nullable PREWHERE $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" -$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1-q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO nullable_prewhere SELECT toDate(0) AS p, if(number % 2 = 0, CAST(number AS Nullable(UInt64)), CAST(NULL AS Nullable(UInt64))) AS f, number as d FROM system.numbers LIMIT 1001" $CLICKHOUSE_CLIENT -q "SELECT sum(d), sum(f), max(d) FROM nullable_prewhere PREWHERE NOT isNull(f)" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" diff --git a/tests/queries/0_stateless/00616_final_single_part.sql b/tests/queries/0_stateless/00616_final_single_part.sql index 605f2c8b216..8c7720f8960 100644 --- a/tests/queries/0_stateless/00616_final_single_part.sql +++ b/tests/queries/0_stateless/00616_final_single_part.sql @@ -3,6 +3,7 @@ SET optimize_on_insert = 0; DROP TABLE IF EXISTS test_00616; DROP TABLE IF EXISTS replacing_00616; +set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE test_00616 ( date Date, diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh index 504b8bac0e0..eea9ea5f7e5 100755 --- a/tests/queries/0_stateless/01162_strange_mutations.sh +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -28,7 +28,7 @@ do $CLICKHOUSE_CLIENT -q "CREATE TABLE test ENGINE=$engine AS SELECT number + 100 AS n, 0 AS test FROM numbers(50)" 2>&1| grep -Ev "Removing leftovers from table|removed by another replica" $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" if [[ $engine == *"ReplicatedMergeTree"* ]]; then - $CLICKHOUSE_CLIENT 0 -q "ALTER TABLE test + $CLICKHOUSE_CLIENT -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 'dummy')[n - 99] WHERE 1" 2>&1| grep -Fa "DB::Exception: " | grep -Fv "statement with subquery may be nondeterministic" $CLICKHOUSE_CLIENT --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1)[n - 99] WHERE 1" diff --git a/tests/queries/0_stateless/01298_alter_merge.sql b/tests/queries/0_stateless/01298_alter_merge.sql index 86c89c38c8c..24547086e0b 100644 --- a/tests/queries/0_stateless/01298_alter_merge.sql +++ b/tests/queries/0_stateless/01298_alter_merge.sql @@ -2,6 +2,7 @@ drop table if exists merge; drop table if exists merge1; drop table if exists merge2; +set allow_deprecated_syntax_for_merge_tree=1; create table merge1 ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); insert into merge1 values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); diff --git a/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh b/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh index 95ecbf09cf5..538d712ad9c 100755 --- a/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh +++ b/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh @@ -8,7 +8,7 @@ WORKING_FOLDER_01528="${CLICKHOUSE_TMP}/01528_clickhouse_local_prepare_parts" rm -rf "${WORKING_FOLDER_01528}" mkdir -p "${WORKING_FOLDER_01528}/metadata/local" - +echo "ATTACH DATABASE local ENGINE=Ordinary" > "${WORKING_FOLDER_01528}/metadata/local.sql" ## Checks scenario of preparing parts offline by clickhouse-local ## that is the metadata for the table we want to fill From e362e4fff1ade90eb36b762e3a51a0d197b64a15 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Jun 2022 20:43:35 +0200 Subject: [PATCH 16/72] Fix --- src/Storages/RabbitMQ/RabbitMQSource.cpp | 3 -- .../ReadBufferFromRabbitMQConsumer.cpp | 6 +++ .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 18 ++++---- .../integration/test_storage_rabbitmq/test.py | 41 +++++++++++++++++-- 4 files changed, 53 insertions(+), 15 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 71d80f0a632..bbcbaa2b32b 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -142,9 +142,6 @@ Chunk RabbitMQSource::generateImpl() while (true) { - if (buffer->eof()) - break; - auto new_rows = executor.execute(); if (new_rows) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index e2de5179990..3543085f5a0 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -47,6 +47,12 @@ ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() } +void ReadBufferFromRabbitMQConsumer::closeChannel() +{ + if (consumer_channel) + consumer_channel->close(); +} + void ReadBufferFromRabbitMQConsumer::subscribe() { for (const auto & queue_name : queues) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 8a527011a3c..bd55d169744 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -3,18 +3,24 @@ #include #include #include -#include -#include #include namespace Poco { - class Logger; +class Logger; +} + +namespace AMQP +{ +class TcpChannel; } namespace DB { +class RabbitMQHandler; +using ChannelPtr = std::unique_ptr; + class ReadBufferFromRabbitMQConsumer : public ReadBuffer { @@ -52,11 +58,7 @@ public: ChannelPtr & getChannel() { return consumer_channel; } void setupChannel(); bool needChannelUpdate(); - void closeChannel() - { - if (consumer_channel) - consumer_channel->close(); - } + void closeChannel(); void updateQueues(std::vector & queues_) { queues = queues_; } size_t queuesCount() { return queues.size(); } diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index c1bd136126f..59ac2f20886 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2745,7 +2745,40 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): break -if __name__ == "__main__": - cluster.start() - input("Cluster created, press any key to destroy...") - cluster.shutdown() +def test_rabbitmq_msgpack(rabbitmq_cluster): + + instance.query( + """ + drop table if exists rabbit_in; + drop table if exists rabbit_out; + create table + rabbit_in (val String) + engine=RabbitMQ + settings rabbitmq_host_port = 'localhost:5672', + rabbitmq_exchange_name = 'xhep', + rabbitmq_format = 'MsgPack', + rabbitmq_num_consumers = 1; + create table + rabbit_out (val String) + engine=RabbitMQ + settings rabbitmq_host_port = 'localhost:5672', + rabbitmq_exchange_name = 'xhep', + rabbitmq_format = 'MsgPack', + rabbitmq_num_consumers = 1; + set stream_like_engine_allow_direct_select=1; + insert into rabbit_out select 'kek'; + """ + ) + + result = "" + try_no = 0 + while True: + result = instance.query("select * from rabbit_in;") + if result.strip() == "kek": + break + else: + try_no = try_no + 1 + if try_no == 20: + break + sleep(1) + assert result.strip() == "kek" From 0d304f7b8cbfbbac48a8fe08a10b374c8987396c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Jun 2022 21:19:07 +0200 Subject: [PATCH 17/72] fix tests --- tests/integration/test_atomic_drop_table/test.py | 2 +- .../test_attach_partition_with_large_destination/test.py | 2 +- tests/integration/test_backup_restore/test.py | 2 +- tests/integration/test_backup_with_other_granularity/test.py | 2 +- tests/integration/test_dictionaries_dependency/test.py | 2 +- tests/integration/test_distributed_ddl/test.py | 2 +- .../test_distributed_storage_configuration/test.py | 2 +- tests/integration/test_filesystem_layout/test.py | 2 +- tests/integration/test_force_drop_table/test.py | 2 +- .../integration/test_keeper_multinode_blocade_leader/test.py | 4 ++-- tests/integration/test_merge_tree_s3_restore/test.py | 2 +- tests/integration/test_partition/test.py | 2 +- tests/integration/test_polymorphic_parts/test.py | 2 +- .../test_replicated_merge_tree_compatibility/test.py | 2 +- .../integration/test_replicated_merge_tree_s3_restore/test.py | 2 +- tests/integration/test_system_merges/test.py | 4 ++-- tests/performance/merge_table_streams.xml | 1 + tests/queries/0_stateless/00090_union_race_conditions_1.sh | 1 + .../0_stateless/00215_primary_key_order_zookeeper_long.sql | 1 + tests/queries/0_stateless/00505_distributed_secure.data | 1 + tests/queries/0_stateless/00632_get_sample_block_cache.sql | 1 + tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh | 1 + 22 files changed, 24 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_atomic_drop_table/test.py b/tests/integration/test_atomic_drop_table/test.py index 1fe88dde099..5ef2f57b684 100644 --- a/tests/integration/test_atomic_drop_table/test.py +++ b/tests/integration/test_atomic_drop_table/test.py @@ -20,7 +20,7 @@ def start_cluster(): try: cluster.start() node1.query( - "CREATE DATABASE zktest ENGINE=Ordinary;" + "CREATE DATABASE zktest ENGINE=Ordinary;", settings={"allow_deprecated_database_ordinary": 1} ) # Different behaviour with Atomic node1.query( """ diff --git a/tests/integration/test_attach_partition_with_large_destination/test.py b/tests/integration/test_attach_partition_with_large_destination/test.py index 0a4ab9fada1..1f4e92ebc55 100644 --- a/tests/integration/test_attach_partition_with_large_destination/test.py +++ b/tests/integration/test_attach_partition_with_large_destination/test.py @@ -34,7 +34,7 @@ def create_force_drop_flag(node): @pytest.mark.parametrize("engine", ["Ordinary", "Atomic"]) def test_attach_partition_with_large_destination(started_cluster, engine): # Initialize - node.query("CREATE DATABASE db ENGINE={}".format(engine)) + node.query("CREATE DATABASE db ENGINE={}".format(engine), settings={"allow_deprecated_database_ordinary": 1}) node.query( "CREATE TABLE db.destination (n UInt64) ENGINE=ReplicatedMergeTree('/test/destination', 'r1') ORDER BY n PARTITION BY n % 2" ) diff --git a/tests/integration/test_backup_restore/test.py b/tests/integration/test_backup_restore/test.py index 905abef05b0..a379c52579e 100644 --- a/tests/integration/test_backup_restore/test.py +++ b/tests/integration/test_backup_restore/test.py @@ -15,7 +15,7 @@ def started_cluster(): try: cluster.start() instance.query( - "CREATE DATABASE test ENGINE = Ordinary" + "CREATE DATABASE test ENGINE = Ordinary", settings={"allow_deprecated_database_ordinary": 1} ) # Different path in shadow/ with Atomic instance.query("DROP TABLE IF EXISTS test.tbl") instance.query( diff --git a/tests/integration/test_backup_with_other_granularity/test.py b/tests/integration/test_backup_with_other_granularity/test.py index 9cb998fb505..fac5331cb7d 100644 --- a/tests/integration/test_backup_with_other_granularity/test.py +++ b/tests/integration/test_backup_with_other_granularity/test.py @@ -202,7 +202,7 @@ def test_backup_from_old_version_config(started_cluster): def test_backup_and_alter(started_cluster): node4.query( - "CREATE DATABASE test ENGINE=Ordinary" + "CREATE DATABASE test ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1} ) # Different path in shadow/ with Atomic node4.query( diff --git a/tests/integration/test_dictionaries_dependency/test.py b/tests/integration/test_dictionaries_dependency/test.py index f57d4e42813..be5767111a9 100644 --- a/tests/integration/test_dictionaries_dependency/test.py +++ b/tests/integration/test_dictionaries_dependency/test.py @@ -16,7 +16,7 @@ def start_cluster(): for node in nodes: node.query("CREATE DATABASE IF NOT EXISTS test") # Different internal dictionary name with Atomic - node.query("CREATE DATABASE IF NOT EXISTS test_ordinary ENGINE=Ordinary") + node.query("CREATE DATABASE IF NOT EXISTS test_ordinary ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1}) node.query("CREATE DATABASE IF NOT EXISTS atest") node.query("CREATE DATABASE IF NOT EXISTS ztest") node.query("CREATE TABLE test.source(x UInt64, y UInt64) ENGINE=Log") diff --git a/tests/integration/test_distributed_ddl/test.py b/tests/integration/test_distributed_ddl/test.py index 2789541b519..99ca6684e56 100755 --- a/tests/integration/test_distributed_ddl/test.py +++ b/tests/integration/test_distributed_ddl/test.py @@ -552,7 +552,7 @@ def test_replicated_without_arguments(test_cluster): ) test_cluster.ddl_check_query( - instance, "CREATE DATABASE test_ordinary ON CLUSTER cluster ENGINE=Ordinary" + instance, "CREATE DATABASE test_ordinary ON CLUSTER cluster ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1} ) assert ( "are supported only for ON CLUSTER queries with Atomic database engine" diff --git a/tests/integration/test_distributed_storage_configuration/test.py b/tests/integration/test_distributed_storage_configuration/test.py index fa4e01bb7b3..1ce8846e815 100644 --- a/tests/integration/test_distributed_storage_configuration/test.py +++ b/tests/integration/test_distributed_storage_configuration/test.py @@ -20,7 +20,7 @@ def start_cluster(): try: cluster.start() node.query( - "CREATE DATABASE IF NOT EXISTS test ENGINE=Ordinary" + "CREATE DATABASE IF NOT EXISTS test ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1} ) # Different paths with Atomic yield cluster finally: diff --git a/tests/integration/test_filesystem_layout/test.py b/tests/integration/test_filesystem_layout/test.py index 34e377e0ae4..78c7b74adcb 100644 --- a/tests/integration/test_filesystem_layout/test.py +++ b/tests/integration/test_filesystem_layout/test.py @@ -16,7 +16,7 @@ def started_cluster(): def test_file_path_escaping(started_cluster): - node.query("CREATE DATABASE IF NOT EXISTS test ENGINE = Ordinary") + node.query("CREATE DATABASE IF NOT EXISTS test ENGINE = Ordinary", settings={"allow_deprecated_database_ordinary": 1}) node.query( """ CREATE TABLE test.`T.a_b,l-e!` (`~Id` UInt32) diff --git a/tests/integration/test_force_drop_table/test.py b/tests/integration/test_force_drop_table/test.py index c1eec1cd277..7731c5d7cf4 100644 --- a/tests/integration/test_force_drop_table/test.py +++ b/tests/integration/test_force_drop_table/test.py @@ -33,7 +33,7 @@ def create_force_drop_flag(node): @pytest.mark.parametrize("engine", ["Ordinary", "Atomic"]) def test_drop_materialized_view(started_cluster, engine): - node.query("CREATE DATABASE d ENGINE={}".format(engine)) + node.query("CREATE DATABASE d ENGINE={}".format(engine), settings={"allow_deprecated_database_ordinary": 1}) node.query( "CREATE TABLE d.rmt (n UInt64) ENGINE=ReplicatedMergeTree('/test/rmt', 'r1') ORDER BY n PARTITION BY n % 2" ) diff --git a/tests/integration/test_keeper_multinode_blocade_leader/test.py b/tests/integration/test_keeper_multinode_blocade_leader/test.py index c2d4039e122..4b1b70ffeab 100644 --- a/tests/integration/test_keeper_multinode_blocade_leader/test.py +++ b/tests/integration/test_keeper_multinode_blocade_leader/test.py @@ -95,7 +95,7 @@ def test_blocade_leader(started_cluster): wait_nodes() try: for i, node in enumerate([node1, node2, node3]): - node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary") + node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1}) node.query( "CREATE TABLE IF NOT EXISTS ordinary.t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format( i + 1 @@ -296,7 +296,7 @@ def test_blocade_leader_twice(started_cluster): wait_nodes() try: for i, node in enumerate([node1, node2, node3]): - node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary") + node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1}) node.query( "CREATE TABLE IF NOT EXISTS ordinary.t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format( i + 1 diff --git a/tests/integration/test_merge_tree_s3_restore/test.py b/tests/integration/test_merge_tree_s3_restore/test.py index e6ca4a78c25..9eae7ce72ad 100644 --- a/tests/integration/test_merge_tree_s3_restore/test.py +++ b/tests/integration/test_merge_tree_s3_restore/test.py @@ -93,7 +93,7 @@ def create_table( node.query( "CREATE DATABASE IF NOT EXISTS s3 ENGINE = {engine}".format( engine="Atomic" if db_atomic else "Ordinary" - ) + ), settings={"allow_deprecated_database_ordinary": 1} ) create_table_statement = """ diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index b396b58df10..43bbddfb2e1 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -14,7 +14,7 @@ def started_cluster(): try: cluster.start() q( - "CREATE DATABASE test ENGINE = Ordinary" + "CREATE DATABASE test ENGINE = Ordinary", settings={"allow_deprecated_database_ordinary": 1} ) # Different path in shadow/ with Atomic yield cluster diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index edd65ec002c..245b53d64bf 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -720,7 +720,7 @@ def test_in_memory_alters(start_cluster): def test_polymorphic_parts_index(start_cluster): node1.query( - "CREATE DATABASE test_index ENGINE=Ordinary" + "CREATE DATABASE test_index ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1} ) # Different paths with Atomic node1.query( """ diff --git a/tests/integration/test_replicated_merge_tree_compatibility/test.py b/tests/integration/test_replicated_merge_tree_compatibility/test.py index 00367daad33..476abd888ab 100644 --- a/tests/integration/test_replicated_merge_tree_compatibility/test.py +++ b/tests/integration/test_replicated_merge_tree_compatibility/test.py @@ -52,7 +52,7 @@ def test_replicated_merge_tree_defaults_compatibility(started_cluster): """ for node in (node1, node2): - node.query("CREATE DATABASE test ENGINE = Ordinary") + node.query("CREATE DATABASE test ENGINE = Ordinary", settings={"allow_deprecated_database_ordinary": 1}) node.query(create_query.format(replica=node.name)) node1.query("DETACH TABLE test.table") diff --git a/tests/integration/test_replicated_merge_tree_s3_restore/test.py b/tests/integration/test_replicated_merge_tree_s3_restore/test.py index 904bcfa4280..1454858f71e 100644 --- a/tests/integration/test_replicated_merge_tree_s3_restore/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_restore/test.py @@ -76,7 +76,7 @@ def create_table(node, table_name, schema, attach=False, db_atomic=False, uuid=" "CREATE DATABASE IF NOT EXISTS s3 {on_cluster} ENGINE = {engine}".format( engine="Atomic" if db_atomic else "Ordinary", on_cluster="ON CLUSTER '{cluster}'", - ) + ), settings={"allow_deprecated_database_ordinary": 1} ) create_table_statement = """ diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 775706f4df6..12f5cedbbb2 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -26,9 +26,9 @@ def started_cluster(): try: cluster.start() node1.query( - "CREATE DATABASE test ENGINE=Ordinary" + "CREATE DATABASE test ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1} ) # Different paths with Atomic - node2.query("CREATE DATABASE test ENGINE=Ordinary") + node2.query("CREATE DATABASE test ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1}) yield cluster finally: diff --git a/tests/performance/merge_table_streams.xml b/tests/performance/merge_table_streams.xml index efeb4547f37..1e053c98738 100644 --- a/tests/performance/merge_table_streams.xml +++ b/tests/performance/merge_table_streams.xml @@ -1,6 +1,7 @@ 5 + 1 From 4d0635371fcc3afe8c6fccbe0d5df048343f8e2f Mon Sep 17 00:00:00 2001 From: Jianmei Zhang Date: Thu, 10 Mar 2022 15:31:27 +0800 Subject: [PATCH 56/72] Support sql stanard create and drop index --- src/Interpreters/InterpreterAlterQuery.cpp | 2 + src/Parsers/ASTAlterQuery.cpp | 43 +++++++- src/Parsers/ASTAlterQuery.h | 8 ++ src/Parsers/ASTIndexDeclaration.cpp | 16 ++- src/Parsers/IAST.h | 1 + src/Parsers/ParserAlterQuery.cpp | 110 ++++++++++++++++++++- src/Parsers/ParserAlterQuery.h | 24 ++++- src/Parsers/formatAST.cpp | 3 +- src/Parsers/formatAST.h | 2 +- src/Storages/AlterCommands.cpp | 4 +- src/Storages/MutationCommands.cpp | 4 +- 11 files changed, 200 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 056a3d9f7b4..23b002350cf 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -295,11 +295,13 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS break; } case ASTAlterCommand::ADD_INDEX: + case ASTAlterCommand::STD_CREATE_INDEX: { required_access.emplace_back(AccessType::ALTER_ADD_INDEX, database, table); break; } case ASTAlterCommand::DROP_INDEX: + case ASTAlterCommand::STD_DROP_INDEX: { if (command.clear_index) required_access.emplace_back(AccessType::ALTER_CLEAR_INDEX, database, table); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index f53c39b192f..36193049a0a 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -1,9 +1,9 @@ +#include #include #include #include #include - namespace DB { namespace ErrorCodes @@ -100,7 +100,9 @@ const char * ASTAlterCommand::typeToString(ASTAlterCommand::Type type) case REMOVE_TTL: return "REMOVE_TTL"; case REMOVE_SAMPLE_BY: return "REMOVE_SAMPLE_BY"; case ADD_INDEX: return "ADD_INDEX"; + case STD_CREATE_INDEX: return "ADD_INDEX"; case DROP_INDEX: return "DROP_INDEX"; + case STD_DROP_INDEX: return "DROP_INDEX"; case MATERIALIZE_INDEX: return "MATERIALIZE_INDEX"; case ADD_CONSTRAINT: return "ADD_CONSTRAINT"; case DROP_CONSTRAINT: return "DROP_CONSTRAINT"; @@ -227,6 +229,15 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & index->formatImpl(settings, state, frame); } } + else if (type == ASTAlterCommand::STD_CREATE_INDEX) + { + auto name = index_decl->as()->name; + index_decl->as()->name = ""; + + index_decl->formatImpl(settings, state, frame); + + index_decl->as()->name = name; + } else if (type == ASTAlterCommand::DROP_INDEX) { settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_index ? "CLEAR " : "DROP ") << "INDEX " @@ -238,6 +249,14 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & partition->formatImpl(settings, state, frame); } } + else if (type == ASTAlterCommand::STD_DROP_INDEX) + { + if (settings.is_translate) + { + settings.ostr << "DROP INDEX " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); + index->formatImpl(settings, state, frame); + } + } else if (type == ASTAlterCommand::MATERIALIZE_INDEX) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE INDEX " << (settings.hilite ? hilite_none : ""); @@ -556,12 +575,26 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState frame.need_parens = false; std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str; switch (alter_object) { case AlterObjectType::TABLE: - settings.ostr << "ALTER TABLE "; + if (command_type == ASTAlterCommand::Type::STD_CREATE_INDEX) + { + settings.ostr << "CREATE INDEX " << (if_not_exists ? "IF NOT EXISTS " : ""); + index_name->formatImpl(settings, state, frame); + settings.ostr << " ON "; + } + else if (command_type == ASTAlterCommand::Type::STD_DROP_INDEX) + { + settings.ostr << "DROP INDEX " << (if_exists ? "IF EXISTS " : ""); + index_name->formatImpl(settings, state, frame); + settings.ostr << " ON "; + } + else + settings.ostr << "ALTER TABLE "; break; case AlterObjectType::DATABASE: settings.ostr << "ALTER DATABASE "; @@ -594,6 +627,12 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState FormatStateStacked frame_nested = frame; frame_nested.need_parens = false; frame_nested.expression_list_always_start_on_new_line = true; + + if ((command_type == ASTAlterCommand::Type::STD_CREATE_INDEX) || (command_type == ASTAlterCommand::Type::STD_DROP_INDEX)){ + frame_nested.expression_list_always_start_on_new_line = false; + settings.ostr << " "; + } + static_cast(command_list)->formatImplMultiline(settings, state, frame_nested); } diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 956f07811ae..0bcb7473e41 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -46,6 +46,8 @@ public: ADD_INDEX, DROP_INDEX, MATERIALIZE_INDEX, + STD_CREATE_INDEX, + STD_DROP_INDEX, ADD_CONSTRAINT, DROP_CONSTRAINT, @@ -226,6 +228,12 @@ public: }; AlterObjectType alter_object = AlterObjectType::UNKNOWN; + ASTAlterCommand::Type command_type = ASTAlterCommand::NO_TYPE; + + /// Used for SQL-standard CREATE INDEX and DROP INDEX + ASTPtr index_name; + bool if_not_exists = false; /// option for CREATE INDEX + bool if_exists = false; /// option for DROP INDEX ASTExpressionList * command_list = nullptr; diff --git a/src/Parsers/ASTIndexDeclaration.cpp b/src/Parsers/ASTIndexDeclaration.cpp index d8ebf825674..a9b30a0f433 100644 --- a/src/Parsers/ASTIndexDeclaration.cpp +++ b/src/Parsers/ASTIndexDeclaration.cpp @@ -25,9 +25,19 @@ ASTPtr ASTIndexDeclaration::clone() const void ASTIndexDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const { - s.ostr << backQuoteIfNeed(name); - s.ostr << " "; - expr->formatImpl(s, state, frame); + /// '' is from CREATE INDEX + if (name != "") + { + s.ostr << backQuoteIfNeed(name); + s.ostr << " "; + expr->formatImpl(s, state, frame); + } + else + { + s.ostr << "("; + expr->formatImpl(s, state, frame); + s.ostr << ")"; + } s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : ""); type->formatImpl(s, state, frame); s.ostr << (s.hilite ? hilite_keyword : "") << " GRANULARITY " << (s.hilite ? hilite_none : ""); diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index b73919f4f36..07b6bed4f5b 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -184,6 +184,7 @@ public: bool hilite = false; bool one_line; bool always_quote_identifiers = false; + bool is_translate = false; //convert current standard SQL to clickhouse dialect. IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; // Newline or whitespace. diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 9a3c1dd8bed..6a5ea0c2286 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -94,6 +95,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_delete("DELETE"); ParserKeyword s_update("UPDATE"); ParserKeyword s_where("WHERE"); + ParserKeyword s_create_index("CREATE INDEX"); ParserKeyword s_to("TO"); ParserKeyword s_remove("REMOVE"); @@ -112,6 +114,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserIdentifier parser_remove_property; ParserCompoundColumnDeclaration parser_col_decl; ParserIndexDeclaration parser_idx_decl; + ParserCreateIndexDeclaration parser_create_idx_decl; ParserConstraintDeclaration parser_constraint_decl; ParserProjectionDeclaration parser_projection_decl; ParserCompoundColumnDeclaration parser_modify_col_decl(false, false, true); @@ -156,7 +159,18 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected break; case ASTAlterQuery::AlterObjectType::TABLE: { - if (s_add_column.ignore(pos, expected)) + if (command_type == ASTAlterCommand::STD_CREATE_INDEX) + { + if (!parser_create_idx_decl.parse(pos, command->index_decl, expected)) + return false; + + command->type = ASTAlterCommand::STD_CREATE_INDEX; + } + else if (command_type == ASTAlterCommand::STD_DROP_INDEX) + { + command->type = ASTAlterCommand::STD_DROP_INDEX; + } + else if (s_add_column.ignore(pos, expected)) { if (s_if_not_exists.ignore(pos, expected)) command->if_not_exists = true; @@ -840,6 +854,44 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return true; } +bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_type("TYPE"); + ParserKeyword s_granularity("GRANULARITY"); + + ParserDataType data_type_p; + ParserExpression expression_p; + ParserUnsignedInteger granularity_p; + + ASTPtr expr; + ASTPtr type; + ASTPtr granularity; + + /// Skip name parser for SQL-standard CREATE INDEX + + if (!expression_p.parse(pos, expr, expected)) + return false; + + if (!s_type.ignore(pos, expected)) + return false; + + if (!data_type_p.parse(pos, type, expected)) + return false; + + if (!s_granularity.ignore(pos, expected)) + return false; + + if (!granularity_p.parse(pos, granularity, expected)) + return false; + + auto index = std::make_shared(); + index->granularity = granularity->as().value.safeGet(); + index->set(index->expr, expr); + index->set(index->type, type); + node = index; + + return true; +} bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { @@ -849,10 +901,45 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_alter_table("ALTER TABLE"); ParserKeyword s_alter_live_view("ALTER LIVE VIEW"); ParserKeyword s_alter_database("ALTER DATABASE"); + ParserKeyword s_create_index("CREATE INDEX"); + ParserKeyword s_drop_index("DROP INDEX"); + ParserKeyword s_on("ON"); + ParserKeyword s_if_not_exists("IF NOT EXISTS"); + ParserKeyword s_if_exists("IF EXISTS"); + + ParserCompoundIdentifier parser_name; ASTAlterQuery::AlterObjectType alter_object_type; - if (s_alter_table.ignore(pos, expected)) + if (s_create_index.ignore(pos, expected)) + { + alter_object_type = ASTAlterQuery::AlterObjectType::TABLE; + if (s_if_not_exists.ignore(pos, expected)) + query->if_not_exists = true; + + if (!parser_name.parse(pos, query->index_name, expected)) + return false; + + if (!s_on.ignore(pos, expected)) + return false; + + query->command_type = ASTAlterCommand::STD_CREATE_INDEX; + } + else if (s_drop_index.ignore(pos, expected)) + { + alter_object_type = ASTAlterQuery::AlterObjectType::TABLE; + if (s_if_exists.ignore(pos, expected)) + query->if_exists = true; + + if (!parser_name.parse(pos, query->index_name, expected)) + return false; + + if (!s_on.ignore(pos, expected)) + return false; + + query->command_type = ASTAlterCommand::STD_DROP_INDEX; + } + else if (s_alter_table.ignore(pos, expected)) { alter_object_type = ASTAlterQuery::AlterObjectType::TABLE; } @@ -891,6 +978,25 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!p_command_list.parse(pos, command_list, expected)) return false; + /// Set the index_name and exists flags for CREATE and DROP INDEX + if (query->command_type == ASTAlterCommand::STD_CREATE_INDEX) + { + ASTAlterCommand * command_ast = command_list->as()->children[0]->as(); + + command_ast->if_not_exists = query->if_not_exists; + + auto & ast_index_decl = command_ast->index_decl->as(); + ast_index_decl.name = query->index_name->as().name(); + + } + else if (query->command_type == ASTAlterCommand::STD_DROP_INDEX) + { + ASTAlterCommand * command_ast = command_list->as()->children[0]->as(); + + command_ast->if_exists = query->if_exists; + command_ast->as()->index = query->index_name; + } + query->set(query->command_list, command_list); query->alter_object = alter_object_type; diff --git a/src/Parsers/ParserAlterQuery.h b/src/Parsers/ParserAlterQuery.h index b0029ff88fd..25505b1c7f9 100644 --- a/src/Parsers/ParserAlterQuery.h +++ b/src/Parsers/ParserAlterQuery.h @@ -30,6 +30,9 @@ namespace DB * [MATERIALIZE INDEX [IF EXISTS] index_name [IN PARTITION partition]] * ALTER LIVE VIEW [db.name] * [REFRESH] + * + * CREATE INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value + * DROP INDEX [IF EXISTS] name on [db].name */ class ParserAlterQuery : public IParserBase @@ -48,9 +51,10 @@ protected: public: ASTAlterQuery::AlterObjectType alter_object; + ASTAlterCommand::Type command_type; - ParserAlterCommandList(ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE) - : alter_object(alter_object_) {} + ParserAlterCommandList(ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE, ASTAlterCommand::Type command_type_= ASTAlterCommand::NO_TYPE) + : alter_object(alter_object_), command_type(command_type_) {} }; @@ -62,10 +66,22 @@ protected: public: ASTAlterQuery::AlterObjectType alter_object; + ASTAlterCommand::Type command_type; - ParserAlterCommand(ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE) - : alter_object(alter_object_) {} + ParserAlterCommand(ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE, ASTAlterCommand::Type command_type_= ASTAlterCommand::NO_TYPE) + : alter_object(alter_object_), command_type(command_type_) {} }; +/** Part of CREATE INDEX expr TYPE typename(arg1, arg2, ...) GRANULARITY value */ +class ParserCreateIndexDeclaration : public IParserBase +{ +public: + ParserCreateIndexDeclaration() {} + +protected: + const char * getName() const override { return "index declaration in create index"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + } diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index fca8ea0aa35..6f5eb3a60b0 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -4,10 +4,11 @@ namespace DB { -void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line) +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool is_translate) { IAST::FormatSettings settings(buf, one_line); settings.hilite = hilite; + settings.is_translate = is_translate; ast.format(settings); } diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index 28af2400a4c..0bf845cd04d 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -11,7 +11,7 @@ class WriteBuffer; /** Takes a syntax tree and turns it back into text. * In case of INSERT query, the data will be missing. */ -void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false); +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false, bool is_translate = false); String serializeAST(const IAST & ast, bool one_line = true); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 845aae52582..1b04fe3ede4 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -211,7 +211,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.type = AlterCommand::REMOVE_SAMPLE_BY; return command; } - else if (command_ast->type == ASTAlterCommand::ADD_INDEX) + else if (command_ast->type == ASTAlterCommand::ADD_INDEX || command_ast->type == ASTAlterCommand::STD_CREATE_INDEX) { AlterCommand command; command.ast = command_ast->clone(); @@ -274,7 +274,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ return command; } - else if (command_ast->type == ASTAlterCommand::DROP_INDEX) + else if (command_ast->type == ASTAlterCommand::DROP_INDEX || command_ast->type == ASTAlterCommand::STD_DROP_INDEX) { AlterCommand command; command.ast = command_ast->clone(); diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 28dfe488869..0f9d17c5d1d 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -103,7 +103,7 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, return res; } - else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_INDEX) + else if (parse_alter_commands && (command->type == ASTAlterCommand::DROP_INDEX || command->type == ASTAlterCommand::STD_DROP_INDEX)) { MutationCommand res; res.ast = command->ptr(); @@ -160,7 +160,7 @@ std::shared_ptr MutationCommands::ast() const void MutationCommands::writeText(WriteBuffer & out) const { WriteBufferFromOwnString commands_buf; - formatAST(*ast(), commands_buf, /* hilite = */ false, /* one_line = */ true); + formatAST(*ast(), commands_buf, /* hilite = */ false, /* one_line = */ true, /* is_translate = */ true); writeEscapedString(commands_buf.str(), out); } From 13231d6a85485a7b80481313ce46080a028c60b2 Mon Sep 17 00:00:00 2001 From: Jianmei Zhang Date: Thu, 10 Mar 2022 16:08:50 +0800 Subject: [PATCH 57/72] Pass command type for create and drop index --- src/Parsers/ASTAlterQuery.cpp | 3 ++- src/Parsers/ParserAlterQuery.cpp | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 36193049a0a..465cd24c9d6 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -628,7 +628,8 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState frame_nested.need_parens = false; frame_nested.expression_list_always_start_on_new_line = true; - if ((command_type == ASTAlterCommand::Type::STD_CREATE_INDEX) || (command_type == ASTAlterCommand::Type::STD_DROP_INDEX)){ + if ((command_type == ASTAlterCommand::Type::STD_CREATE_INDEX) || (command_type == ASTAlterCommand::Type::STD_DROP_INDEX)) + { frame_nested.expression_list_always_start_on_new_line = false; settings.ostr << " "; } diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 6a5ea0c2286..99105d27bca 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -839,7 +839,7 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe node = command_list; ParserToken s_comma(TokenType::Comma); - ParserAlterCommand p_command(alter_object); + ParserAlterCommand p_command(alter_object, command_type); do { @@ -973,7 +973,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->cluster = cluster_str; } - ParserAlterCommandList p_command_list(alter_object_type); + ParserAlterCommandList p_command_list(alter_object_type, query->command_type); ASTPtr command_list; if (!p_command_list.parse(pos, command_list, expected)) return false; From d6b0bc2942992dac782c5a49a049b211b3814650 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Wed, 8 Jun 2022 16:27:00 +0800 Subject: [PATCH 58/72] Add test case --- ...319_sql_standard_create_drop_index.reference | 4 ++++ .../02319_sql_standard_create_drop_index.sql | 17 +++++++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference create mode 100644 tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql diff --git a/tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference new file mode 100644 index 00000000000..6565857f89d --- /dev/null +++ b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference @@ -0,0 +1,4 @@ +CREATE TABLE default.t_index\n(\n `a` Int32,\n `b` String,\n INDEX i_a a TYPE minmax GRANULARITY 4,\n INDEX i_b b TYPE bloom_filter GRANULARITY 2\n)\nENGINE = MergeTree\nORDER BY a\nSETTINGS index_granularity = 8192 +t_index i_a minmax a 4 +t_index i_b bloom_filter b 2 +t_index i_b bloom_filter b 2 diff --git a/tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql new file mode 100644 index 00000000000..a33505ced3a --- /dev/null +++ b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql @@ -0,0 +1,17 @@ +drop table if exists t_index; +create table t_index(a int, b String) engine=MergeTree() order by a; + +create index i_a on t_index(a) TYPE minmax GRANULARITY 4; +create index if not exists i_a on t_index(a) TYPE minmax GRANULARITY 2; + +create index i_b on t_index(b) TYPE bloom_filter GRANULARITY 2; + +show create table t_index; +select table, name, type, expr, granularity from system.data_skipping_indices where database = currentDatabase() and table = 't_index'; + +drop index i_a on t_index; +drop index if exists i_a on t_index; + +select table, name, type, expr, granularity from system.data_skipping_indices where database = currentDatabase() and table = 't_index'; + +drop table t_index; From 9154aec2010c3efa24a3f4bd9a9f648a14ec38a1 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Thu, 9 Jun 2022 11:27:46 +0800 Subject: [PATCH 59/72] Fix compile error --- src/Parsers/ASTIndexDeclaration.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTIndexDeclaration.cpp b/src/Parsers/ASTIndexDeclaration.cpp index a9b30a0f433..87500a565e4 100644 --- a/src/Parsers/ASTIndexDeclaration.cpp +++ b/src/Parsers/ASTIndexDeclaration.cpp @@ -26,7 +26,7 @@ ASTPtr ASTIndexDeclaration::clone() const void ASTIndexDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const { /// '' is from CREATE INDEX - if (name != "") + if (!name.empty()) { s.ostr << backQuoteIfNeed(name); s.ostr << " "; From 589cba8045ba06efc5e50c12124ed6c557733f57 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Thu, 16 Jun 2022 17:58:14 +0800 Subject: [PATCH 60/72] Support sql standard create index in seprate parser files --- .../InterpreterCreateIndexQuery.cpp | 82 ++++++++++++ .../InterpreterCreateIndexQuery.h | 25 ++++ src/Interpreters/InterpreterFactory.cpp | 6 + src/Parsers/ASTCreateIndexQuery.cpp | 65 ++++++++++ src/Parsers/ASTCreateIndexQuery.h | 40 ++++++ src/Parsers/ASTIndexDeclaration.cpp | 16 +-- src/Parsers/ASTIndexDeclaration.h | 1 + src/Parsers/ParserAlterQuery.cpp | 68 +--------- src/Parsers/ParserAlterQuery.h | 12 -- src/Parsers/ParserCreateIndexQuery.cpp | 120 ++++++++++++++++++ src/Parsers/ParserCreateIndexQuery.h | 31 +++++ src/Parsers/ParserQuery.cpp | 3 + 12 files changed, 383 insertions(+), 86 deletions(-) create mode 100644 src/Interpreters/InterpreterCreateIndexQuery.cpp create mode 100644 src/Interpreters/InterpreterCreateIndexQuery.h create mode 100644 src/Parsers/ASTCreateIndexQuery.cpp create mode 100644 src/Parsers/ASTCreateIndexQuery.h create mode 100644 src/Parsers/ParserCreateIndexQuery.cpp create mode 100644 src/Parsers/ParserCreateIndexQuery.h diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp new file mode 100644 index 00000000000..8a237162b54 --- /dev/null +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -0,0 +1,82 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_QUERY; + extern const int NOT_IMPLEMENTED; + extern const int TABLE_IS_READ_ONLY; +} + + +BlockIO InterpreterCreateIndexQuery::execute() +{ + FunctionNameNormalizer().visit(query_ptr.get()); + const auto & create_index = query_ptr->as(); + + AccessRightsElements required_access; + required_access.emplace_back(AccessType::ALTER_ADD_INDEX, create_index.getDatabase(), create_index.getTable()); + + if (!create_index.cluster.empty()) + { + DDLQueryOnClusterParams params; + params.access_to_check = std::move(required_access); + return executeDDLQueryOnCluster(query_ptr, getContext(), params); + } + + getContext()->checkAccess(required_access); + auto table_id = getContext()->resolveStorageID(create_index, Context::ResolveOrdinary); + query_ptr->as().setDatabase(table_id.database_name); + + DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); + if (typeid_cast(database.get()) + && !getContext()->getClientInfo().is_replicated_database_internal) + { + auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); + guard->releaseTableLock(); + return typeid_cast(database.get())->tryEnqueueReplicatedDDL(query_ptr, getContext()); + } + + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); + if (table->isStaticStorage()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only"); + + /// Convert ASTCreateIndexQuery to AlterCommand. + AlterCommands alter_commands; + + AlterCommand command; + command.index_decl = create_index.index_decl; + command.type = AlterCommand::ADD_INDEX; + command.index_name = create_index.index_name->as().name(); + command.if_not_exists = create_index.if_not_exists; + + /// Fill name in ASTIndexDeclaration + auto & ast_index_decl = command.index_decl->as(); + ast_index_decl.name = command.index_name; + + alter_commands.emplace_back(std::move(command)); + + auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout); + StorageInMemoryMetadata metadata = table->getInMemoryMetadata(); + alter_commands.validate(table, getContext()); + alter_commands.prepare(metadata); + table->checkAlterIsPossible(alter_commands, getContext()); + table->alter(alter_commands, getContext(), alter_lock); + + return {}; +} + +} diff --git a/src/Interpreters/InterpreterCreateIndexQuery.h b/src/Interpreters/InterpreterCreateIndexQuery.h new file mode 100644 index 00000000000..c6cb7285590 --- /dev/null +++ b/src/Interpreters/InterpreterCreateIndexQuery.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + +class InterpreterCreateIndexQuery : public IInterpreter, WithContext +{ +public: + InterpreterCreateIndexQuery(const ASTPtr & query_ptr_, ContextPtr context_) + : WithContext(context_) + , query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index c212eb50b97..a2edecbf1e4 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -42,6 +43,7 @@ #include #include #include +#include #include #include #include @@ -298,6 +300,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/src/Parsers/ASTCreateIndexQuery.cpp b/src/Parsers/ASTCreateIndexQuery.cpp new file mode 100644 index 00000000000..e83e95f1dc7 --- /dev/null +++ b/src/Parsers/ASTCreateIndexQuery.cpp @@ -0,0 +1,65 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int UNEXPECTED_AST_STRUCTURE; +} + +/** Get the text that identifies this element. */ +String ASTCreateIndexQuery::getID(char delim) const +{ + return "CreateIndexQuery" + (delim + getDatabase()) + delim + getTable(); +} + +ASTPtr ASTCreateIndexQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + res->index_name = index_name->clone(); + res->children.push_back(res->index_name); + + res->index_decl = index_decl->clone(); + res->children.push_back(res->index_decl); + return res; +} + +void ASTCreateIndexQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + frame.need_parens = false; + + std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); + + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str; + + settings.ostr << "CREATE INDEX " << (if_not_exists ? "IF NOT EXISTS " : ""); + index_name->formatImpl(settings, state, frame); + settings.ostr << " ON "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + + if (table) + { + if (database) + { + settings.ostr << indent_str << backQuoteIfNeed(getDatabase()); + settings.ostr << "."; + } + settings.ostr << indent_str << backQuoteIfNeed(getTable()); + } + + formatOnCluster(settings); + + if (!cluster.empty()) + settings.ostr << " "; + + index_decl->formatImpl(settings, state, frame); +} + +} diff --git a/src/Parsers/ASTCreateIndexQuery.h b/src/Parsers/ASTCreateIndexQuery.h new file mode 100644 index 00000000000..aa2bbfe8573 --- /dev/null +++ b/src/Parsers/ASTCreateIndexQuery.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/** + * CREATE INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value + */ + +class ASTCreateIndexQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster +{ +public: + bool if_not_exists{false}; + + ASTPtr index_name; + + /// Stores the IndexDeclaration here. + ASTPtr index_decl; + + String getID(char delim) const override; + + ASTPtr clone() const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams & params) const override + { + return removeOnCluster(clone(), params.default_database); + } + + virtual QueryKind getQueryKind() const override { return QueryKind::Create; } + +protected: + void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/ASTIndexDeclaration.cpp b/src/Parsers/ASTIndexDeclaration.cpp index 87500a565e4..cc988d1d307 100644 --- a/src/Parsers/ASTIndexDeclaration.cpp +++ b/src/Parsers/ASTIndexDeclaration.cpp @@ -25,19 +25,19 @@ ASTPtr ASTIndexDeclaration::clone() const void ASTIndexDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const { - /// '' is from CREATE INDEX - if (!name.empty()) - { - s.ostr << backQuoteIfNeed(name); - s.ostr << " "; - expr->formatImpl(s, state, frame); - } - else + if (from_create_index) { s.ostr << "("; expr->formatImpl(s, state, frame); s.ostr << ")"; } + else + { + s.ostr << backQuoteIfNeed(name); + s.ostr << " "; + expr->formatImpl(s, state, frame); + } + s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : ""); type->formatImpl(s, state, frame); s.ostr << (s.hilite ? hilite_keyword : "") << " GRANULARITY " << (s.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTIndexDeclaration.h b/src/Parsers/ASTIndexDeclaration.h index 8416ec6b0a6..31d5ef0e7f8 100644 --- a/src/Parsers/ASTIndexDeclaration.h +++ b/src/Parsers/ASTIndexDeclaration.h @@ -16,6 +16,7 @@ public: IAST * expr; ASTFunction * type; UInt64 granularity; + bool from_create_index = false; /** Get the text that identifies this element. */ String getID(char) const override { return "Index"; } diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 99105d27bca..f40a63ef584 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -95,7 +95,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_delete("DELETE"); ParserKeyword s_update("UPDATE"); ParserKeyword s_where("WHERE"); - ParserKeyword s_create_index("CREATE INDEX"); ParserKeyword s_to("TO"); ParserKeyword s_remove("REMOVE"); @@ -114,7 +113,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserIdentifier parser_remove_property; ParserCompoundColumnDeclaration parser_col_decl; ParserIndexDeclaration parser_idx_decl; - ParserCreateIndexDeclaration parser_create_idx_decl; ParserConstraintDeclaration parser_constraint_decl; ParserProjectionDeclaration parser_projection_decl; ParserCompoundColumnDeclaration parser_modify_col_decl(false, false, true); @@ -159,14 +157,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected break; case ASTAlterQuery::AlterObjectType::TABLE: { - if (command_type == ASTAlterCommand::STD_CREATE_INDEX) - { - if (!parser_create_idx_decl.parse(pos, command->index_decl, expected)) - return false; - - command->type = ASTAlterCommand::STD_CREATE_INDEX; - } - else if (command_type == ASTAlterCommand::STD_DROP_INDEX) + if (command_type == ASTAlterCommand::STD_DROP_INDEX) { command->type = ASTAlterCommand::STD_DROP_INDEX; } @@ -854,45 +845,6 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return true; } -bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserKeyword s_type("TYPE"); - ParserKeyword s_granularity("GRANULARITY"); - - ParserDataType data_type_p; - ParserExpression expression_p; - ParserUnsignedInteger granularity_p; - - ASTPtr expr; - ASTPtr type; - ASTPtr granularity; - - /// Skip name parser for SQL-standard CREATE INDEX - - if (!expression_p.parse(pos, expr, expected)) - return false; - - if (!s_type.ignore(pos, expected)) - return false; - - if (!data_type_p.parse(pos, type, expected)) - return false; - - if (!s_granularity.ignore(pos, expected)) - return false; - - if (!granularity_p.parse(pos, granularity, expected)) - return false; - - auto index = std::make_shared(); - index->granularity = granularity->as().value.safeGet(); - index->set(index->expr, expr); - index->set(index->type, type); - node = index; - - return true; -} - bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { auto query = std::make_shared(); @@ -901,31 +853,15 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_alter_table("ALTER TABLE"); ParserKeyword s_alter_live_view("ALTER LIVE VIEW"); ParserKeyword s_alter_database("ALTER DATABASE"); - ParserKeyword s_create_index("CREATE INDEX"); ParserKeyword s_drop_index("DROP INDEX"); ParserKeyword s_on("ON"); - ParserKeyword s_if_not_exists("IF NOT EXISTS"); ParserKeyword s_if_exists("IF EXISTS"); ParserCompoundIdentifier parser_name; ASTAlterQuery::AlterObjectType alter_object_type; - if (s_create_index.ignore(pos, expected)) - { - alter_object_type = ASTAlterQuery::AlterObjectType::TABLE; - if (s_if_not_exists.ignore(pos, expected)) - query->if_not_exists = true; - - if (!parser_name.parse(pos, query->index_name, expected)) - return false; - - if (!s_on.ignore(pos, expected)) - return false; - - query->command_type = ASTAlterCommand::STD_CREATE_INDEX; - } - else if (s_drop_index.ignore(pos, expected)) + if (s_drop_index.ignore(pos, expected)) { alter_object_type = ASTAlterQuery::AlterObjectType::TABLE; if (s_if_exists.ignore(pos, expected)) diff --git a/src/Parsers/ParserAlterQuery.h b/src/Parsers/ParserAlterQuery.h index 25505b1c7f9..22d3f2a13c2 100644 --- a/src/Parsers/ParserAlterQuery.h +++ b/src/Parsers/ParserAlterQuery.h @@ -31,7 +31,6 @@ namespace DB * ALTER LIVE VIEW [db.name] * [REFRESH] * - * CREATE INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value * DROP INDEX [IF EXISTS] name on [db].name */ @@ -73,15 +72,4 @@ public: }; -/** Part of CREATE INDEX expr TYPE typename(arg1, arg2, ...) GRANULARITY value */ -class ParserCreateIndexDeclaration : public IParserBase -{ -public: - ParserCreateIndexDeclaration() {} - -protected: - const char * getName() const override { return "index declaration in create index"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - } diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp new file mode 100644 index 00000000000..fe765281e21 --- /dev/null +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -0,0 +1,120 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_type("TYPE"); + ParserKeyword s_granularity("GRANULARITY"); + + ParserDataType data_type_p; + ParserExpression expression_p; + ParserUnsignedInteger granularity_p; + + ASTPtr expr; + ASTPtr type; + ASTPtr granularity; + + /// Skip name parser for SQL-standard CREATE INDEX + if (!expression_p.parse(pos, expr, expected)) + return false; + + if (!s_type.ignore(pos, expected)) + return false; + + if (!data_type_p.parse(pos, type, expected)) + return false; + + if (!s_granularity.ignore(pos, expected)) + return false; + + if (!granularity_p.parse(pos, granularity, expected)) + return false; + + auto index = std::make_shared(); + index->from_create_index = true; + index->granularity = granularity->as().value.safeGet(); + index->set(index->expr, expr); + index->set(index->type, type); + node = index; + + return true; +} + +bool ParserCreateIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + auto query = std::make_shared(); + node = query; + + ParserKeyword s_create("CREATE"); + ParserKeyword s_index("INDEX"); + ParserKeyword s_if_not_exists("IF NOT EXISTS"); + ParserKeyword s_on("ON"); + ParserIdentifier index_name_p; + ParserCreateIndexDeclaration parser_create_idx_decl; + + ASTPtr index_name; + ASTPtr index_decl; + + String cluster_str; + bool if_not_exists = false; + + if (!s_create.ignore(pos, expected)) + return false; + + if (!s_index.ignore(pos, expected)) + return false; + + if (s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + + if (!index_name_p.parse(pos, index_name, expected)) + return false; + + /// ON [db.] table_name + if (!s_on.ignore(pos, expected)) + return false; + + if (!parseDatabaseAndTableAsAST(pos, expected, query->database, query->table)) + return false; + + /// [ON cluster_name] + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + if (!parser_create_idx_decl.parse(pos, index_decl, expected)) + return false; + + query->index_name = index_name; + query->children.push_back(index_name); + + query->index_decl = index_decl; + query->children.push_back(index_decl); + + query->if_not_exists = if_not_exists; + query->cluster = cluster_str; + + if (query->database) + query->children.push_back(query->database); + + if (query->table) + query->children.push_back(query->table); + + return true; +} + +} diff --git a/src/Parsers/ParserCreateIndexQuery.h b/src/Parsers/ParserCreateIndexQuery.h new file mode 100644 index 00000000000..3dfdccc301f --- /dev/null +++ b/src/Parsers/ParserCreateIndexQuery.h @@ -0,0 +1,31 @@ +#pragma once + +#include + +namespace DB +{ + +/** Query like this: + * CREATE INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value + */ + +class ParserCreateIndexQuery : public IParserBase +{ +protected: + const char * getName() const override{ return "CREATE INDEX query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + + +/** Parser for index declaration in create index, where name is ignored. */ +class ParserCreateIndexDeclaration : public IParserBase +{ +public: + ParserCreateIndexDeclaration() {} + +protected: + const char * getName() const override { return "index declaration in create index"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index eaea5dd0f5f..3edaa384c36 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -43,6 +44,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateSettingsProfileQuery create_settings_profile_p; ParserCreateFunctionQuery create_function_p; ParserDropFunctionQuery drop_function_p; + ParserCreateIndexQuery create_index_p; ParserDropAccessEntityQuery drop_access_entity_p; ParserGrantQuery grant_p; ParserSetRoleQuery set_role_p; @@ -63,6 +65,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_settings_profile_p.parse(pos, node, expected) || create_function_p.parse(pos, node, expected) || drop_function_p.parse(pos, node, expected) + || create_index_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected) || external_ddl_p.parse(pos, node, expected) From 8d49aa4697e17a7cb588f7393a2f85832a43dd8c Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Fri, 17 Jun 2022 15:46:44 +0800 Subject: [PATCH 61/72] Support sql standard drop index in seprate parser files --- src/Interpreters/InterpreterAlterQuery.cpp | 2 - .../InterpreterCreateIndexQuery.cpp | 5 -- .../InterpreterCreateIndexQuery.h | 1 - .../InterpreterDropIndexQuery.cpp | 71 +++++++++++++++++++ src/Interpreters/InterpreterDropIndexQuery.h | 24 +++++++ src/Interpreters/InterpreterFactory.cpp | 6 ++ src/Parsers/ASTAlterQuery.cpp | 42 +---------- src/Parsers/ASTAlterQuery.h | 8 --- src/Parsers/ASTCreateIndexQuery.cpp | 4 -- src/Parsers/ASTDropIndexQuery.cpp | 63 ++++++++++++++++ src/Parsers/ASTDropIndexQuery.h | 44 ++++++++++++ src/Parsers/IAST.h | 1 - src/Parsers/ParserAlterQuery.cpp | 51 ++----------- src/Parsers/ParserAlterQuery.h | 12 ++-- src/Parsers/ParserCreateIndexQuery.cpp | 2 +- src/Parsers/ParserDropIndexQuery.cpp | 67 +++++++++++++++++ src/Parsers/ParserDropIndexQuery.h | 19 +++++ src/Parsers/ParserQuery.cpp | 3 + src/Parsers/formatAST.cpp | 3 +- src/Parsers/formatAST.h | 2 +- src/Storages/AlterCommands.cpp | 4 +- src/Storages/MutationCommands.cpp | 4 +- 22 files changed, 314 insertions(+), 124 deletions(-) create mode 100644 src/Interpreters/InterpreterDropIndexQuery.cpp create mode 100644 src/Interpreters/InterpreterDropIndexQuery.h create mode 100644 src/Parsers/ASTDropIndexQuery.cpp create mode 100644 src/Parsers/ASTDropIndexQuery.h create mode 100644 src/Parsers/ParserDropIndexQuery.cpp create mode 100644 src/Parsers/ParserDropIndexQuery.h diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 23b002350cf..056a3d9f7b4 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -295,13 +295,11 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS break; } case ASTAlterCommand::ADD_INDEX: - case ASTAlterCommand::STD_CREATE_INDEX: { required_access.emplace_back(AccessType::ALTER_ADD_INDEX, database, table); break; } case ASTAlterCommand::DROP_INDEX: - case ASTAlterCommand::STD_DROP_INDEX: { if (command.clear_index) required_access.emplace_back(AccessType::ALTER_CLEAR_INDEX, database, table); diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp index 8a237162b54..29c151d1e4d 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.cpp +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include @@ -15,16 +14,12 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; - extern const int INCORRECT_QUERY; - extern const int NOT_IMPLEMENTED; extern const int TABLE_IS_READ_ONLY; } BlockIO InterpreterCreateIndexQuery::execute() { - FunctionNameNormalizer().visit(query_ptr.get()); const auto & create_index = query_ptr->as(); AccessRightsElements required_access; diff --git a/src/Interpreters/InterpreterCreateIndexQuery.h b/src/Interpreters/InterpreterCreateIndexQuery.h index c6cb7285590..63eaaf5f1c2 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.h +++ b/src/Interpreters/InterpreterCreateIndexQuery.h @@ -18,7 +18,6 @@ public: BlockIO execute() override; private: - ASTPtr query_ptr; }; diff --git a/src/Interpreters/InterpreterDropIndexQuery.cpp b/src/Interpreters/InterpreterDropIndexQuery.cpp new file mode 100644 index 00000000000..6cc9334fad2 --- /dev/null +++ b/src/Interpreters/InterpreterDropIndexQuery.cpp @@ -0,0 +1,71 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TABLE_IS_READ_ONLY; +} + + +BlockIO InterpreterDropIndexQuery::execute() +{ + const auto & drop_index = query_ptr->as(); + + AccessRightsElements required_access; + required_access.emplace_back(AccessType::ALTER_DROP_INDEX, drop_index.getDatabase(), drop_index.getTable()); + + if (!drop_index.cluster.empty()) + { + DDLQueryOnClusterParams params; + params.access_to_check = std::move(required_access); + return executeDDLQueryOnCluster(query_ptr, getContext(), params); + } + + getContext()->checkAccess(required_access); + auto table_id = getContext()->resolveStorageID(drop_index, Context::ResolveOrdinary); + query_ptr->as().setDatabase(table_id.database_name); + + DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); + if (typeid_cast(database.get()) + && !getContext()->getClientInfo().is_replicated_database_internal) + { + auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); + guard->releaseTableLock(); + return typeid_cast(database.get())->tryEnqueueReplicatedDDL(query_ptr, getContext()); + } + + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); + if (table->isStaticStorage()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only"); + + /// Convert ASTDropIndexQuery to AlterCommand. + AlterCommands alter_commands; + + AlterCommand command; + command.ast = drop_index.convertToASTAlterCommand(); + command.type = AlterCommand::DROP_INDEX; + command.index_name = drop_index.index_name->as().name(); + command.if_exists = drop_index.if_exists; + + alter_commands.emplace_back(std::move(command)); + + auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout); + StorageInMemoryMetadata metadata = table->getInMemoryMetadata(); + alter_commands.validate(table, getContext()); + alter_commands.prepare(metadata); + table->checkAlterIsPossible(alter_commands, getContext()); + table->alter(alter_commands, getContext(), alter_lock); + + return {}; +} + +} diff --git a/src/Interpreters/InterpreterDropIndexQuery.h b/src/Interpreters/InterpreterDropIndexQuery.h new file mode 100644 index 00000000000..c6fb3add72f --- /dev/null +++ b/src/Interpreters/InterpreterDropIndexQuery.h @@ -0,0 +1,24 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + +class InterpreterDropIndexQuery : public IInterpreter, WithContext +{ +public: + InterpreterDropIndexQuery(const ASTPtr & query_ptr_, ContextPtr context_) + : WithContext(context_) + , query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index a2edecbf1e4..6b081467ae7 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -48,6 +49,7 @@ #include #include #include +#include #include #include #include @@ -304,6 +306,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 465cd24c9d6..cfcc5decdf5 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -4,6 +4,7 @@ #include #include + namespace DB { namespace ErrorCodes @@ -100,9 +101,7 @@ const char * ASTAlterCommand::typeToString(ASTAlterCommand::Type type) case REMOVE_TTL: return "REMOVE_TTL"; case REMOVE_SAMPLE_BY: return "REMOVE_SAMPLE_BY"; case ADD_INDEX: return "ADD_INDEX"; - case STD_CREATE_INDEX: return "ADD_INDEX"; case DROP_INDEX: return "DROP_INDEX"; - case STD_DROP_INDEX: return "DROP_INDEX"; case MATERIALIZE_INDEX: return "MATERIALIZE_INDEX"; case ADD_CONSTRAINT: return "ADD_CONSTRAINT"; case DROP_CONSTRAINT: return "DROP_CONSTRAINT"; @@ -229,15 +228,6 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & index->formatImpl(settings, state, frame); } } - else if (type == ASTAlterCommand::STD_CREATE_INDEX) - { - auto name = index_decl->as()->name; - index_decl->as()->name = ""; - - index_decl->formatImpl(settings, state, frame); - - index_decl->as()->name = name; - } else if (type == ASTAlterCommand::DROP_INDEX) { settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_index ? "CLEAR " : "DROP ") << "INDEX " @@ -249,14 +239,6 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & partition->formatImpl(settings, state, frame); } } - else if (type == ASTAlterCommand::STD_DROP_INDEX) - { - if (settings.is_translate) - { - settings.ostr << "DROP INDEX " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); - index->formatImpl(settings, state, frame); - } - } else if (type == ASTAlterCommand::MATERIALIZE_INDEX) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE INDEX " << (settings.hilite ? hilite_none : ""); @@ -581,20 +563,7 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState switch (alter_object) { case AlterObjectType::TABLE: - if (command_type == ASTAlterCommand::Type::STD_CREATE_INDEX) - { - settings.ostr << "CREATE INDEX " << (if_not_exists ? "IF NOT EXISTS " : ""); - index_name->formatImpl(settings, state, frame); - settings.ostr << " ON "; - } - else if (command_type == ASTAlterCommand::Type::STD_DROP_INDEX) - { - settings.ostr << "DROP INDEX " << (if_exists ? "IF EXISTS " : ""); - index_name->formatImpl(settings, state, frame); - settings.ostr << " ON "; - } - else - settings.ostr << "ALTER TABLE "; + settings.ostr << "ALTER TABLE "; break; case AlterObjectType::DATABASE: settings.ostr << "ALTER DATABASE "; @@ -627,13 +596,6 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState FormatStateStacked frame_nested = frame; frame_nested.need_parens = false; frame_nested.expression_list_always_start_on_new_line = true; - - if ((command_type == ASTAlterCommand::Type::STD_CREATE_INDEX) || (command_type == ASTAlterCommand::Type::STD_DROP_INDEX)) - { - frame_nested.expression_list_always_start_on_new_line = false; - settings.ostr << " "; - } - static_cast(command_list)->formatImplMultiline(settings, state, frame_nested); } diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 0bcb7473e41..956f07811ae 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -46,8 +46,6 @@ public: ADD_INDEX, DROP_INDEX, MATERIALIZE_INDEX, - STD_CREATE_INDEX, - STD_DROP_INDEX, ADD_CONSTRAINT, DROP_CONSTRAINT, @@ -228,12 +226,6 @@ public: }; AlterObjectType alter_object = AlterObjectType::UNKNOWN; - ASTAlterCommand::Type command_type = ASTAlterCommand::NO_TYPE; - - /// Used for SQL-standard CREATE INDEX and DROP INDEX - ASTPtr index_name; - bool if_not_exists = false; /// option for CREATE INDEX - bool if_exists = false; /// option for DROP INDEX ASTExpressionList * command_list = nullptr; diff --git a/src/Parsers/ASTCreateIndexQuery.cpp b/src/Parsers/ASTCreateIndexQuery.cpp index e83e95f1dc7..7a5c80551d6 100644 --- a/src/Parsers/ASTCreateIndexQuery.cpp +++ b/src/Parsers/ASTCreateIndexQuery.cpp @@ -6,10 +6,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int UNEXPECTED_AST_STRUCTURE; -} /** Get the text that identifies this element. */ String ASTCreateIndexQuery::getID(char delim) const diff --git a/src/Parsers/ASTDropIndexQuery.cpp b/src/Parsers/ASTDropIndexQuery.cpp new file mode 100644 index 00000000000..eca8c1676d5 --- /dev/null +++ b/src/Parsers/ASTDropIndexQuery.cpp @@ -0,0 +1,63 @@ +#include +#include +#include + + +namespace DB +{ + +/** Get the text that identifies this element. */ +String ASTDropIndexQuery::getID(char delim) const +{ + return "CreateIndexQuery" + (delim + getDatabase()) + delim + getTable(); +} + +ASTPtr ASTDropIndexQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + res->index_name = index_name->clone(); + res->children.push_back(res->index_name); + + return res; +} + +void ASTDropIndexQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + frame.need_parens = false; + + std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); + + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str; + + settings.ostr << "DROP INDEX " << (if_exists ? "IF EXISTS " : ""); + index_name->formatImpl(settings, state, frame); + settings.ostr << " ON "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + + if (table) + { + if (database) + { + settings.ostr << indent_str << backQuoteIfNeed(getDatabase()); + settings.ostr << "."; + } + settings.ostr << indent_str << backQuoteIfNeed(getTable()); + } + + formatOnCluster(settings); +} + +ASTPtr ASTDropIndexQuery::convertToASTAlterCommand() const +{ + auto command = std::make_shared(); + command->index = index_name->clone(); + command->if_exists = if_exists; + command->type = ASTAlterCommand::DROP_INDEX; + + return command; +} + +} diff --git a/src/Parsers/ASTDropIndexQuery.h b/src/Parsers/ASTDropIndexQuery.h new file mode 100644 index 00000000000..2a771c643ed --- /dev/null +++ b/src/Parsers/ASTDropIndexQuery.h @@ -0,0 +1,44 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + + +namespace DB +{ + +/** + * DROP INDEX [IF EXISTS] name on [db].name + */ + +class ASTDropIndexQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster +{ +public: + bool if_exists{false}; + + ASTPtr index_name; + + String getID(char delim) const override; + + ASTPtr clone() const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams & params) const override + { + return removeOnCluster(clone(), params.default_database); + } + + virtual QueryKind getQueryKind() const override { return QueryKind::Drop; } + + /// Convert ASTDropIndexQuery to ASTAlterCommand. + ASTPtr convertToASTAlterCommand() const; + +protected: + void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 07b6bed4f5b..b73919f4f36 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -184,7 +184,6 @@ public: bool hilite = false; bool one_line; bool always_quote_identifiers = false; - bool is_translate = false; //convert current standard SQL to clickhouse dialect. IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; // Newline or whitespace. diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index f40a63ef584..bc3af03a3c4 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -157,11 +156,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected break; case ASTAlterQuery::AlterObjectType::TABLE: { - if (command_type == ASTAlterCommand::STD_DROP_INDEX) - { - command->type = ASTAlterCommand::STD_DROP_INDEX; - } - else if (s_add_column.ignore(pos, expected)) + if (s_add_column.ignore(pos, expected)) { if (s_if_not_exists.ignore(pos, expected)) command->if_not_exists = true; @@ -830,7 +825,7 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe node = command_list; ParserToken s_comma(TokenType::Comma); - ParserAlterCommand p_command(alter_object, command_type); + ParserAlterCommand p_command(alter_object); do { @@ -853,29 +848,10 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_alter_table("ALTER TABLE"); ParserKeyword s_alter_live_view("ALTER LIVE VIEW"); ParserKeyword s_alter_database("ALTER DATABASE"); - ParserKeyword s_drop_index("DROP INDEX"); - ParserKeyword s_on("ON"); - ParserKeyword s_if_exists("IF EXISTS"); - - ParserCompoundIdentifier parser_name; ASTAlterQuery::AlterObjectType alter_object_type; - if (s_drop_index.ignore(pos, expected)) - { - alter_object_type = ASTAlterQuery::AlterObjectType::TABLE; - if (s_if_exists.ignore(pos, expected)) - query->if_exists = true; - - if (!parser_name.parse(pos, query->index_name, expected)) - return false; - - if (!s_on.ignore(pos, expected)) - return false; - - query->command_type = ASTAlterCommand::STD_DROP_INDEX; - } - else if (s_alter_table.ignore(pos, expected)) + if (s_alter_table.ignore(pos, expected)) { alter_object_type = ASTAlterQuery::AlterObjectType::TABLE; } @@ -909,30 +885,11 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->cluster = cluster_str; } - ParserAlterCommandList p_command_list(alter_object_type, query->command_type); + ParserAlterCommandList p_command_list(alter_object_type); ASTPtr command_list; if (!p_command_list.parse(pos, command_list, expected)) return false; - /// Set the index_name and exists flags for CREATE and DROP INDEX - if (query->command_type == ASTAlterCommand::STD_CREATE_INDEX) - { - ASTAlterCommand * command_ast = command_list->as()->children[0]->as(); - - command_ast->if_not_exists = query->if_not_exists; - - auto & ast_index_decl = command_ast->index_decl->as(); - ast_index_decl.name = query->index_name->as().name(); - - } - else if (query->command_type == ASTAlterCommand::STD_DROP_INDEX) - { - ASTAlterCommand * command_ast = command_list->as()->children[0]->as(); - - command_ast->if_exists = query->if_exists; - command_ast->as()->index = query->index_name; - } - query->set(query->command_list, command_list); query->alter_object = alter_object_type; diff --git a/src/Parsers/ParserAlterQuery.h b/src/Parsers/ParserAlterQuery.h index 22d3f2a13c2..b0029ff88fd 100644 --- a/src/Parsers/ParserAlterQuery.h +++ b/src/Parsers/ParserAlterQuery.h @@ -30,8 +30,6 @@ namespace DB * [MATERIALIZE INDEX [IF EXISTS] index_name [IN PARTITION partition]] * ALTER LIVE VIEW [db.name] * [REFRESH] - * - * DROP INDEX [IF EXISTS] name on [db].name */ class ParserAlterQuery : public IParserBase @@ -50,10 +48,9 @@ protected: public: ASTAlterQuery::AlterObjectType alter_object; - ASTAlterCommand::Type command_type; - ParserAlterCommandList(ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE, ASTAlterCommand::Type command_type_= ASTAlterCommand::NO_TYPE) - : alter_object(alter_object_), command_type(command_type_) {} + ParserAlterCommandList(ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE) + : alter_object(alter_object_) {} }; @@ -65,10 +62,9 @@ protected: public: ASTAlterQuery::AlterObjectType alter_object; - ASTAlterCommand::Type command_type; - ParserAlterCommand(ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE, ASTAlterCommand::Type command_type_= ASTAlterCommand::NO_TYPE) - : alter_object(alter_object_), command_type(command_type_) {} + ParserAlterCommand(ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE) + : alter_object(alter_object_) {} }; diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index fe765281e21..22411c71ee5 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -70,7 +70,7 @@ bool ParserCreateIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect String cluster_str; bool if_not_exists = false; - if (!s_create.ignore(pos, expected)) + if (!s_create.ignore(pos, expected)) return false; if (!s_index.ignore(pos, expected)) diff --git a/src/Parsers/ParserDropIndexQuery.cpp b/src/Parsers/ParserDropIndexQuery.cpp new file mode 100644 index 00000000000..19f31d6128d --- /dev/null +++ b/src/Parsers/ParserDropIndexQuery.cpp @@ -0,0 +1,67 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserDropIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + auto query = std::make_shared(); + node = query; + + ParserKeyword s_drop("DROP"); + ParserKeyword s_index("INDEX"); + ParserKeyword s_on("ON"); + ParserKeyword s_if_exists("IF EXISTS"); + ParserIdentifier index_name_p; + + String cluster_str; + bool if_exists = false; + + if (!s_drop.ignore(pos, expected)) + return false; + + if (!s_index.ignore(pos, expected)) + return false; + + if (s_if_exists.ignore(pos, expected)) + if_exists = true; + + if (!index_name_p.parse(pos, query->index_name, expected)) + return false; + + /// ON [db.] table_name + if (!s_on.ignore(pos, expected)) + return false; + + if (!parseDatabaseAndTableAsAST(pos, expected, query->database, query->table)) + return false; + + /// [ON cluster_name] + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + + query->cluster = std::move(cluster_str); + } + + if(query->index_name) + query->children.push_back(query->index_name); + + query->if_exists = if_exists; + + if (query->database) + query->children.push_back(query->database); + + if (query->table) + query->children.push_back(query->table); + + return true; +} + +} diff --git a/src/Parsers/ParserDropIndexQuery.h b/src/Parsers/ParserDropIndexQuery.h new file mode 100644 index 00000000000..1b6535c7efb --- /dev/null +++ b/src/Parsers/ParserDropIndexQuery.h @@ -0,0 +1,19 @@ +#pragma once + +#include + +namespace DB +{ + +/** Query like this: + * DROP INDEX [IF EXISTS] name ON [db].name + */ + +class ParserDropIndexQuery : public IParserBase +{ +protected: + const char * getName() const override{ return "DROP INDEX query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 3edaa384c36..a3cafee65d7 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -45,6 +46,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateFunctionQuery create_function_p; ParserDropFunctionQuery drop_function_p; ParserCreateIndexQuery create_index_p; + ParserDropIndexQuery drop_index_p; ParserDropAccessEntityQuery drop_access_entity_p; ParserGrantQuery grant_p; ParserSetRoleQuery set_role_p; @@ -66,6 +68,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_function_p.parse(pos, node, expected) || drop_function_p.parse(pos, node, expected) || create_index_p.parse(pos, node, expected) + || drop_index_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected) || external_ddl_p.parse(pos, node, expected) diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index 6f5eb3a60b0..fca8ea0aa35 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -4,11 +4,10 @@ namespace DB { -void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool is_translate) +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line) { IAST::FormatSettings settings(buf, one_line); settings.hilite = hilite; - settings.is_translate = is_translate; ast.format(settings); } diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index 0bf845cd04d..28af2400a4c 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -11,7 +11,7 @@ class WriteBuffer; /** Takes a syntax tree and turns it back into text. * In case of INSERT query, the data will be missing. */ -void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false, bool is_translate = false); +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false); String serializeAST(const IAST & ast, bool one_line = true); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 1b04fe3ede4..845aae52582 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -211,7 +211,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.type = AlterCommand::REMOVE_SAMPLE_BY; return command; } - else if (command_ast->type == ASTAlterCommand::ADD_INDEX || command_ast->type == ASTAlterCommand::STD_CREATE_INDEX) + else if (command_ast->type == ASTAlterCommand::ADD_INDEX) { AlterCommand command; command.ast = command_ast->clone(); @@ -274,7 +274,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ return command; } - else if (command_ast->type == ASTAlterCommand::DROP_INDEX || command_ast->type == ASTAlterCommand::STD_DROP_INDEX) + else if (command_ast->type == ASTAlterCommand::DROP_INDEX) { AlterCommand command; command.ast = command_ast->clone(); diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 0f9d17c5d1d..28dfe488869 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -103,7 +103,7 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, return res; } - else if (parse_alter_commands && (command->type == ASTAlterCommand::DROP_INDEX || command->type == ASTAlterCommand::STD_DROP_INDEX)) + else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_INDEX) { MutationCommand res; res.ast = command->ptr(); @@ -160,7 +160,7 @@ std::shared_ptr MutationCommands::ast() const void MutationCommands::writeText(WriteBuffer & out) const { WriteBufferFromOwnString commands_buf; - formatAST(*ast(), commands_buf, /* hilite = */ false, /* one_line = */ true, /* is_translate = */ true); + formatAST(*ast(), commands_buf, /* hilite = */ false, /* one_line = */ true); writeEscapedString(commands_buf.str(), out); } From d75af09f7435c8229ff3482bfa0fd80b6bd866f2 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Sat, 18 Jun 2022 18:18:31 +0800 Subject: [PATCH 62/72] Fix style error --- src/Parsers/ASTCreateIndexQuery.h | 3 +-- src/Parsers/ASTDropIndexQuery.cpp | 2 +- src/Parsers/ASTDropIndexQuery.h | 4 +--- src/Parsers/ParserDropIndexQuery.cpp | 2 +- 4 files changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Parsers/ASTCreateIndexQuery.h b/src/Parsers/ASTCreateIndexQuery.h index aa2bbfe8573..f3c6a7830a4 100644 --- a/src/Parsers/ASTCreateIndexQuery.h +++ b/src/Parsers/ASTCreateIndexQuery.h @@ -8,8 +8,7 @@ namespace DB { -/** - * CREATE INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value +/** CREATE INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value */ class ASTCreateIndexQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster diff --git a/src/Parsers/ASTDropIndexQuery.cpp b/src/Parsers/ASTDropIndexQuery.cpp index eca8c1676d5..78152d213b8 100644 --- a/src/Parsers/ASTDropIndexQuery.cpp +++ b/src/Parsers/ASTDropIndexQuery.cpp @@ -24,7 +24,7 @@ ASTPtr ASTDropIndexQuery::clone() const } void ASTDropIndexQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const -{ +{ frame.need_parens = false; std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); diff --git a/src/Parsers/ASTDropIndexQuery.h b/src/Parsers/ASTDropIndexQuery.h index 2a771c643ed..d7e39f797b5 100644 --- a/src/Parsers/ASTDropIndexQuery.h +++ b/src/Parsers/ASTDropIndexQuery.h @@ -8,12 +8,10 @@ #include - namespace DB { -/** - * DROP INDEX [IF EXISTS] name on [db].name +/** DROP INDEX [IF EXISTS] name on [db].name */ class ASTDropIndexQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster diff --git a/src/Parsers/ParserDropIndexQuery.cpp b/src/Parsers/ParserDropIndexQuery.cpp index 19f31d6128d..0844ea16ae0 100644 --- a/src/Parsers/ParserDropIndexQuery.cpp +++ b/src/Parsers/ParserDropIndexQuery.cpp @@ -50,7 +50,7 @@ bool ParserDropIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected query->cluster = std::move(cluster_str); } - if(query->index_name) + if (query->index_name) query->children.push_back(query->index_name); query->if_exists = if_exists; From f7ef571842f788993718b658dbf83b0b530f26cf Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 29 Jun 2022 03:10:10 +0300 Subject: [PATCH 63/72] Don't spoil return code of integration tests runner with redundant tee. --- tests/ci/integration_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index cf0dfe51e9b..9fda2d09ae6 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -225,7 +225,7 @@ if __name__ == "__main__": output_path_log = os.path.join(result_path, "main_script_log.txt") runner_path = os.path.join(repo_path, "tests/integration", "ci-runner.py") - run_command = f"sudo -E {runner_path} | tee {output_path_log}" + run_command = f"sudo -E {runner_path}" logging.info("Going to run command: `%s`", run_command) logging.info( "ENV parameters for runner:\n%s", From 0b400518830a83f4d8f6a264b9541cdf119b5c1c Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Wed, 29 Jun 2022 06:04:02 -0600 Subject: [PATCH 64/72] Update zh/tutorial.md Fix `include` from old mkdocs version --- docs/zh/getting-started/tutorial.md | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/docs/zh/getting-started/tutorial.md b/docs/zh/getting-started/tutorial.md index 3d66f79cb46..2a0095bd3a9 100644 --- a/docs/zh/getting-started/tutorial.md +++ b/docs/zh/getting-started/tutorial.md @@ -16,7 +16,17 @@ sidebar_label: 使用教程 例如,您选择`deb`安装包,执行: ``` bash -{% include 'install/deb.sh' %} +sudo apt-get install -y apt-transport-https ca-certificates dirmngr +sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754 + +echo "deb https://packages.clickhouse.com/deb stable main" | sudo tee \ + /etc/apt/sources.list.d/clickhouse.list +sudo apt-get update + +sudo apt-get install -y clickhouse-server clickhouse-client + +sudo service clickhouse-server start +clickhouse-client # or "clickhouse-client --password" if you've set up a password. ``` 在我们安装的软件中包含这些包: From c5e0869c46663dc49c87a5d8e45386f16132ff86 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 29 Jun 2022 14:20:26 +0200 Subject: [PATCH 65/72] fix flaky test --- tests/queries/0_stateless/01592_long_window_functions1.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01592_long_window_functions1.sql b/tests/queries/0_stateless/01592_long_window_functions1.sql index 4911b7aa792..022d8071ffa 100644 --- a/tests/queries/0_stateless/01592_long_window_functions1.sql +++ b/tests/queries/0_stateless/01592_long_window_functions1.sql @@ -4,6 +4,10 @@ drop table if exists stack; set max_insert_threads = 4; +-- Temporary disable aggregation in order, +-- because it may fail with UBSan. +set optimize_aggregation_in_order = 0; + create table stack(item_id Int64, brand_id Int64, rack_id Int64, dt DateTime, expiration_dt DateTime, quantity UInt64) Engine = MergeTree partition by toYYYYMM(dt) From baeb1811e194df4784339b737d135a9c30f58c50 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 29 Jun 2022 16:59:16 +0300 Subject: [PATCH 66/72] try to fix performance tests --- docker/test/performance-comparison/download.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/performance-comparison/download.sh b/docker/test/performance-comparison/download.sh index 463c08c5304..8d6c48935b7 100755 --- a/docker/test/performance-comparison/download.sh +++ b/docker/test/performance-comparison/download.sh @@ -73,6 +73,7 @@ function download ) & wait + cd db0 && echo "ATTACH DATABASE default ENGINE=Ordinary" > metadata/default.sql } download From 628ac3e1b8dd78f25cca79a128526dc93cd27243 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Wed, 29 Jun 2022 10:34:38 -0400 Subject: [PATCH 67/72] Apply suggestions from code review --- docs/zh/getting-started/tutorial.md | 3 --- 1 file changed, 3 deletions(-) diff --git a/docs/zh/getting-started/tutorial.md b/docs/zh/getting-started/tutorial.md index 2a0095bd3a9..5fa77bffa78 100644 --- a/docs/zh/getting-started/tutorial.md +++ b/docs/zh/getting-started/tutorial.md @@ -24,9 +24,6 @@ echo "deb https://packages.clickhouse.com/deb stable main" | sudo tee \ sudo apt-get update sudo apt-get install -y clickhouse-server clickhouse-client - -sudo service clickhouse-server start -clickhouse-client # or "clickhouse-client --password" if you've set up a password. ``` 在我们安装的软件中包含这些包: From 41460dcaca29b84d36fa52155b4dd4c9ef7675bd Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 29 Jun 2022 17:37:29 +0200 Subject: [PATCH 68/72] quick fix for 02112_with_fill_interval Seems like the problem is that now data more often come to FillingTransform in multiple chunks. Don't know why it affects the results, will continue investigation. --- tests/queries/0_stateless/02112_with_fill_interval.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02112_with_fill_interval.sql b/tests/queries/0_stateless/02112_with_fill_interval.sql index d2416f9a84b..16773780515 100644 --- a/tests/queries/0_stateless/02112_with_fill_interval.sql +++ b/tests/queries/0_stateless/02112_with_fill_interval.sql @@ -1,3 +1,5 @@ +SET max_threads = 1; + DROP TABLE IF EXISTS with_fill_date; CREATE TABLE with_fill_date (d Date, d32 Date32) ENGINE = Memory; From eb3f49426c3064df729f168e2b42b718d5bd4f1c Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Wed, 29 Jun 2022 09:58:55 -0700 Subject: [PATCH 69/72] Remove zlib in mariadb-connector-c --- contrib/mariadb-connector-c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/mariadb-connector-c b/contrib/mariadb-connector-c index 5f4034a3a63..e39608998f5 160000 --- a/contrib/mariadb-connector-c +++ b/contrib/mariadb-connector-c @@ -1 +1 @@ -Subproject commit 5f4034a3a6376416504f17186c55fe401c6d8e5e +Subproject commit e39608998f5f6944ece9ec61f48e9172ec1de660 From cd2cb10f605bcd706d9be731e0488d6c2bd3c369 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 29 Jun 2022 20:55:46 +0300 Subject: [PATCH 70/72] another try to fix performance tests --- docker/test/performance-comparison/download.sh | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docker/test/performance-comparison/download.sh b/docker/test/performance-comparison/download.sh index 8d6c48935b7..a125b1816e5 100755 --- a/docker/test/performance-comparison/download.sh +++ b/docker/test/performance-comparison/download.sh @@ -73,7 +73,16 @@ function download ) & wait + pwd + ls + ls db0 + ls db0/metadata cd db0 && echo "ATTACH DATABASE default ENGINE=Ordinary" > metadata/default.sql + cd db0 && echo "ATTACH DATABASE datasets ENGINE=Ordinary" > metadata/datasets.sql + pwd + ls + ls db0 + ls db0/metadata } download From 6bf9c7cd4d5de11b94b777c5d37158f9eeb9f084 Mon Sep 17 00:00:00 2001 From: HeenaBansal2009 Date: Wed, 29 Jun 2022 12:21:05 -0700 Subject: [PATCH 71/72] Updated index file for Geo functions --- docs/en/sql-reference/functions/geo/index.md | 96 ++++++++++---------- 1 file changed, 48 insertions(+), 48 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/index.md b/docs/en/sql-reference/functions/geo/index.md index ea43b3ef96c..c0162cb5b63 100644 --- a/docs/en/sql-reference/functions/geo/index.md +++ b/docs/en/sql-reference/functions/geo/index.md @@ -20,57 +20,57 @@ sidebar_position: 62 ## H3 Indexes Functions -- [h3IsValid](./h3#h3IsValid) -- [h3GetResolution](./h3#h3GetResolution) -- [h3EdgeAngle](./h3#h3EdgeAngle) -- [h3EdgeLengthM​](./h3#h3EdgeLengthM​) -- [h3EdgeLengthKm] (./h3#h3EdgeLengthKm) -- [geoToH3](./h3#geoToH3) -- [h3ToGeo](./h3#h3ToGeo) -- [h3ToGeoBoundary](./h3#h3ToGeoBoundary) -- [h3kRing](./h3#h3kRing) -- [h3GetBaseCell](./h3#h3GetBaseCell) -- [h3HexAreaM2](./h3#h3HexAreaM2) -- [h3HexAreaKm2](./h3#h3HexAreaKm2) -- [h3IndexesAreNeighbors](./h3#h3IndexesAreNeighbors) -- [h3ToChildren](./h3#h3ToChildren) -- [h3ToParent](./h3#h3ToParent) -- [h3ToString](./h3#h3ToString) -- [stringToH3](./h3#stringToH3) -- [h3GetResolution](./h3#h3GetResolution) -- [h3IsResClassIII](./h3#h3IsResClassIII) -- [h3IsPentagon](./h3#h3IsPentagon) -- [h3GetFaces](./h3#h3GetFaces) -- [h3CellAreaM2](./h3#h3CellAreaM2) -- [h3CellAreaRads2](./h3#h3CellAreaRads2) -- [h3ToCenterChild](./h3#h3ToCenterChild) -- [h3ExactEdgeLengthM](./h3#h3ExactEdgeLengthM) -- [h3ExactEdgeLengthKm](./h3#h3ExactEdgeLengthKm) -- [h3ExactEdgeLengthRads](./h3#h3ExactEdgeLengthRads) -- [h3NumHexagons](./h3#h3NumHexagons) -- [h3Line](./h3#h3Line) -- [h3Distance](./h3#h3Distance) -- [h3HexRing](./h3#h3HexRing) -- [h3GetUnidirectionalEdge](./h3#h3GetUnidirectionalEdge) -- [h3UnidirectionalEdgeIsValid](./h3#h3UnidirectionalEdgeIsValid) -- [h3GetOriginIndexFromUnidirectionalEdge](./h3#h3GetOriginIndexFromUnidirectionalEdge) -- [h3GetDestinationIndexFromUnidirectionalEdge](./h3#h3GetDestinationIndexFromUnidirectionalEdge) -- [h3GetIndexesFromUnidirectionalEdge](./h3#h3GetIndexesFromUnidirectionalEdge) -- [h3GetUnidirectionalEdgesFromHexagon](./h3#h3GetUnidirectionalEdgesFromHexagon) -- [h3GetUnidirectionalEdgeBoundary](./h3#h3GetUnidirectionalEdgeBoundary) +- [h3IsValid](./h3.md#h3IsValid) +- [h3GetResolution](./h3.md#h3GetResolution) +- [h3EdgeAngle](./h3.md#h3EdgeAngle) +- [h3EdgeLengthM​](./h3.md#h3EdgeLengthM​) +- [h3EdgeLengthKm](./h3.md#h3EdgeLengthKm) +- [geoToH3](./h3.md#geoToH3) +- [h3ToGeo](./h3.md#h3ToGeo) +- [h3ToGeoBoundary](./h3.md#h3ToGeoBoundary) +- [h3kRing](./h3.md#h3kRing) +- [h3GetBaseCell](./h3.md#h3GetBaseCell) +- [h3HexAreaM2](./h3.md#h3HexAreaM2) +- [h3HexAreaKm2](./h3.md#h3HexAreaKm2) +- [h3IndexesAreNeighbors](./h3.md#h3IndexesAreNeighbors) +- [h3ToChildren](./h3.md#h3ToChildren) +- [h3ToParent](./h3.md#h3ToParent) +- [h3ToString](./h3.md#h3ToString) +- [stringToH3](./h3.md#stringToH3) +- [h3GetResolution](./h3.md#h3GetResolution) +- [h3IsResClassIII](./h3.md#h3IsResClassIII) +- [h3IsPentagon](./h3.md#h3IsPentagon) +- [h3GetFaces](./h3.md#h3GetFaces) +- [h3CellAreaM2](./h3.md#h3CellAreaM2) +- [h3CellAreaRads2](./h3.md#h3CellAreaRads2) +- [h3ToCenterChild](./h3.md#h3ToCenterChild) +- [h3ExactEdgeLengthM](./h3.md#h3ExactEdgeLengthM) +- [h3ExactEdgeLengthKm](./h3.md#h3ExactEdgeLengthKm) +- [h3ExactEdgeLengthRads](./h3.md#h3ExactEdgeLengthRads) +- [h3NumHexagons](./h3.md#h3NumHexagons) +- [h3Line](./h3.md#h3Line) +- [h3Distance](./h3.md#h3Distance) +- [h3HexRing](./h3.md#h3HexRing) +- [h3GetUnidirectionalEdge](./h3.md#h3GetUnidirectionalEdge) +- [h3UnidirectionalEdgeIsValid](./h3.md#h3UnidirectionalEdgeIsValid) +- [h3GetOriginIndexFromUnidirectionalEdge](./h3.md#h3GetOriginIndexFromUnidirectionalEdge) +- [h3GetDestinationIndexFromUnidirectionalEdge](./h3.md#h3GetDestinationIndexFromUnidirectionalEdge) +- [h3GetIndexesFromUnidirectionalEdge](./h3.md#h3GetIndexesFromUnidirectionalEdge) +- [h3GetUnidirectionalEdgesFromHexagon](./h3.md#h3GetUnidirectionalEdgesFromHexagon) +- [h3GetUnidirectionalEdgeBoundary](./h3.md#h3GetUnidirectionalEdgeBoundary) ## S2 Index Functions -- [geoToS2](./s2#geoToS2) -- [s2ToGeo](./s2#s2ToGeo) -- [s2GetNeighbors](./s2#s2GetNeighbors) -- [s2CellsIntersect](./s2#s2CellsIntersect) -- [s2CapContains](./s2#s2CapContains) -- [s2CapUnion](./s2#s2CapUnion) -- [s2RectAdd](./s2#s2RectAdd) -- [s2RectContains](./s2#s2RectContains) -- [s2RectUinion](./s2#s2RectUinion) -- [s2RectIntersection](./s2#s2RectIntersection) +- [geoToS2](./s2.md#geoToS2) +- [s2ToGeo](./s2.md#s2ToGeo) +- [s2GetNeighbors](./s2.md#s2GetNeighbors) +- [s2CellsIntersect](./s2.md#s2CellsIntersect) +- [s2CapContains](./s2.md#s2CapContains) +- [s2CapUnion](./s2.md#s2CapUnion) +- [s2RectAdd](./s2.md#s2RectAdd) +- [s2RectContains](./s2.md#s2RectContains) +- [s2RectUinion](./s2.md#s2RectUinion) +- [s2RectIntersection](./s2.md#s2RectIntersection) [Original article](https://clickhouse.com/docs/en/sql-reference/functions/geo/) From 4bf1443daecea075ca31de92285b65d0a59b66f5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 30 Jun 2022 01:00:08 +0300 Subject: [PATCH 72/72] Update download.sh --- docker/test/performance-comparison/download.sh | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/docker/test/performance-comparison/download.sh b/docker/test/performance-comparison/download.sh index a125b1816e5..da866aabd9a 100755 --- a/docker/test/performance-comparison/download.sh +++ b/docker/test/performance-comparison/download.sh @@ -73,15 +73,8 @@ function download ) & wait - pwd - ls - ls db0 - ls db0/metadata - cd db0 && echo "ATTACH DATABASE default ENGINE=Ordinary" > metadata/default.sql - cd db0 && echo "ATTACH DATABASE datasets ENGINE=Ordinary" > metadata/datasets.sql - pwd - ls - ls db0 + echo "ATTACH DATABASE default ENGINE=Ordinary" > db0/metadata/default.sql + echo "ATTACH DATABASE datasets ENGINE=Ordinary" > db0/metadata/datasets.sql ls db0/metadata }