From ebf9681888afd51b5a1cc37fcc55d0d11b6798c3 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 18 Jun 2021 19:55:42 +0300 Subject: [PATCH 001/147] Create projection.md info about manipulating projections based on RFC #14730 --- .../statements/alter/projection.md | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 docs/en/sql-reference/statements/alter/projection.md diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md new file mode 100644 index 00000000000..f7f829c9589 --- /dev/null +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -0,0 +1,23 @@ +--- +toc_priority: 49 +toc_title: PROJECTION +--- + +# Manipulating Projections {#manipulations-with-projections} + +The following operations are available: + +- `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [WHERE] [GROUP BY] [ORDER BY]` - Adds projection description to tables metadata. + +- `ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. + +- `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). + +- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` + +The first two commands are lightweight in a sense that they only change metadata or remove files. + +Also, they are replicated, syncing indices metadata via ZooKeeper. + +!!! note "Note" + Projection manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). From da4dc26975c85441911fed22b9a625a91074bacb Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Sat, 19 Jun 2021 18:15:25 +0300 Subject: [PATCH 002/147] Update projection.md --- docs/en/sql-reference/statements/alter/projection.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index f7f829c9589..526202bed33 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -13,11 +13,11 @@ The following operations are available: - `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). -- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` +- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. -The first two commands are lightweight in a sense that they only change metadata or remove files. +The commands ADD, DROP and CLEAR are lightweight in a sense that they only change metadata or remove files. -Also, they are replicated, syncing indices metadata via ZooKeeper. +Also, they are replicated, syncing projections metadata via ZooKeeper. !!! note "Note" Projection manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). From abf4bf832bcf1717660bebd006e514d9f9436bbc Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Sat, 19 Jun 2021 18:32:18 +0300 Subject: [PATCH 003/147] Update projection.md Currently WHERE is not supported. --- docs/en/sql-reference/statements/alter/projection.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 526202bed33..9d762a07234 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -7,7 +7,7 @@ toc_title: PROJECTION The following operations are available: -- `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [WHERE] [GROUP BY] [ORDER BY]` - Adds projection description to tables metadata. +- `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [GROUP BY] [ORDER BY]` - Adds projection description to tables metadata. - `ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. From d9a77e3a1a9dd16ac5a6c08e0a86886732d7a493 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Jul 2021 16:21:38 +0300 Subject: [PATCH 004/147] improve CREATE OR REPLACE query --- src/Interpreters/InterpreterCreateQuery.cpp | 77 +++++++++---------- src/Interpreters/InterpreterRenameQuery.cpp | 16 +++- src/Interpreters/InterpreterRenameQuery.h | 3 + src/Interpreters/executeQuery.cpp | 70 ++++++++++++----- src/Interpreters/executeQuery.h | 4 + src/Parsers/ASTRenameQuery.h | 3 + src/Storages/StorageDistributed.cpp | 5 ++ src/Storages/StorageDistributed.h | 1 + .../0_stateless/01157_replace_table.reference | 20 +++++ .../0_stateless/01157_replace_table.sql | 48 ++++++++++++ tests/queries/skip_list.json | 1 + 11 files changed, 186 insertions(+), 62 deletions(-) create mode 100644 tests/queries/0_stateless/01157_replace_table.reference create mode 100644 tests/queries/0_stateless/01157_replace_table.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index de858bdbdc5..ccd92fb650c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -31,6 +32,7 @@ #include #include +#include #include #include #include @@ -796,36 +798,6 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data create.uuid = UUIDHelpers::Nil; create.to_inner_uuid = UUIDHelpers::Nil; } - - if (create.replace_table) - { - if (database->getUUID() == UUIDHelpers::Nil) - throw Exception(ErrorCodes::INCORRECT_QUERY, - "{} query is supported only for Atomic databases", - create.create_or_replace ? "CREATE OR REPLACE TABLE" : "REPLACE TABLE"); - - UUID uuid_of_table_to_replace; - if (create.create_or_replace) - { - uuid_of_table_to_replace = getContext()->tryResolveStorageID(StorageID(create.database, create.table)).uuid; - if (uuid_of_table_to_replace == UUIDHelpers::Nil) - { - /// Convert to usual CREATE - create.replace_table = false; - assert(!database->isTableExist(create.table, getContext())); - } - else - create.table = "_tmp_replace_" + toString(uuid_of_table_to_replace); - } - else - { - uuid_of_table_to_replace = getContext()->resolveStorageID(StorageID(create.database, create.table)).uuid; - if (uuid_of_table_to_replace == UUIDHelpers::Nil) - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", - backQuoteIfNeed(create.database), backQuoteIfNeed(create.table)); - create.table = "_tmp_replace_" + toString(uuid_of_table_to_replace); - } - } } @@ -1105,11 +1077,27 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, { auto ast_drop = std::make_shared(); String table_to_replace_name = create.table; - bool created = false; - bool replaced = false; + { + auto database = DatabaseCatalog::instance().getDatabase(create.database); + if (database->getUUID() == UUIDHelpers::Nil) + throw Exception(ErrorCodes::INCORRECT_QUERY, + "{} query is supported only for Atomic databases", + create.create_or_replace ? "CREATE OR REPLACE TABLE" : "REPLACE TABLE"); + + + UInt64 name_hash = sipHash64(create.database + create.table); + UInt16 random_suffix = thread_local_rng(); + create.table = fmt::format("_tmp_replace_{}_{}", + getHexUIntLowercase(name_hash), + getHexUIntLowercase(random_suffix)); + } + + bool created = false; + bool renamed = false; try { + /// Create temporary table (random name will be generated) [[maybe_unused]] bool done = doCreateTable(create, properties); assert(done); ast_drop->table = create.table; @@ -1117,9 +1105,12 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, ast_drop->database = create.database; ast_drop->kind = ASTDropQuery::Drop; created = true; - if (!create.replace_table) - return fillTableIfNeeded(create); + /// Try fill temporary table + BlockIO fill_io = fillTableIfNeeded(create); + executeTrivialBlockIO(fill_io, getContext()); + + /// Replace target table with created one auto ast_rename = std::make_shared(); ASTRenameQuery::Element elem { @@ -1130,19 +1121,27 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, ast_rename->elements.push_back(std::move(elem)); ast_rename->exchange = true; ast_rename->dictionary = create.is_dictionary; + /// Will execute ordinary RENAME instead of EXCHANGE if the target table does not exist + ast_rename->rename_if_cannot_exchange = create.create_or_replace; - InterpreterRenameQuery(ast_rename, getContext()).execute(); - replaced = true; + InterpreterRenameQuery interpreter_rename{ast_rename, getContext()}; + interpreter_rename.execute(); + renamed = true; - InterpreterDropQuery(ast_drop, getContext()).execute(); + if (!interpreter_rename.renamedInsteadOfExchange()) + { + /// Target table was replaced with new one, drop old table + InterpreterDropQuery(ast_drop, getContext()).execute(); + } create.table = table_to_replace_name; - return fillTableIfNeeded(create); + return {}; } catch (...) { - if (created && create.replace_table && !replaced) + /// Drop temporary table if it was successfully created, but was not renamed to target name + if (created && !renamed) InterpreterDropQuery(ast_drop, getContext()).execute(); throw; } diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 515559ad903..989cdad93d5 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -76,8 +76,20 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c for (const auto & elem : descriptions) { - if (!rename.exchange) + bool exchange_tables = rename.exchange; + if (exchange_tables) + { + bool allow_rename_instead_of_exchange = descriptions.size() == 1 && rename.rename_if_cannot_exchange; + if (allow_rename_instead_of_exchange && !database_catalog.isTableExist(StorageID(elem.to_database_name, elem.to_table_name), getContext())) + { + exchange_tables = false; + renamed_instead_of_exchange = true; + } + } + else + { database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name), getContext()); + } DatabasePtr database = database_catalog.getDatabase(elem.from_database_name); if (typeid_cast(database.get()) @@ -100,7 +112,7 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c elem.from_table_name, *database_catalog.getDatabase(elem.to_database_name), elem.to_table_name, - rename.exchange, + exchange_tables, rename.dictionary); } } diff --git a/src/Interpreters/InterpreterRenameQuery.h b/src/Interpreters/InterpreterRenameQuery.h index 49fdd50f52d..dfcd741754e 100644 --- a/src/Interpreters/InterpreterRenameQuery.h +++ b/src/Interpreters/InterpreterRenameQuery.h @@ -55,6 +55,8 @@ public: BlockIO execute() override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override; + bool renamedInsteadOfExchange() const { return renamed_instead_of_exchange; } + private: BlockIO executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions, TableGuards & ddl_guards); static BlockIO executeToDatabase(const ASTRenameQuery & rename, const RenameDescriptions & descriptions); @@ -62,6 +64,7 @@ private: AccessRightsElements getRequiredAccess() const; ASTPtr query_ptr; + bool renamed_instead_of_exchange{false}; }; } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index c69a5bcd3e1..8f9a8b8a3e9 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1060,26 +1060,30 @@ void executeQuery( } else if (pipeline.initialized()) { - const ASTQueryWithOutput * ast_query_with_output = dynamic_cast(ast.get()); - - WriteBuffer * out_buf = &ostr; - std::optional out_file_buf; - if (ast_query_with_output && ast_query_with_output->out_file) + if (pipeline.isCompleted()) { - if (!allow_into_outfile) - throw Exception("INTO OUTFILE is not allowed", ErrorCodes::INTO_OUTFILE_NOT_ALLOWED); - - const auto & out_file = typeid_cast(*ast_query_with_output->out_file).value.safeGet(); - out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT); - out_buf = &*out_file_buf; + pipeline.setProgressCallback(context->getProgressCallback()); } - - String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr) - ? getIdentifierName(ast_query_with_output->format) - : context->getDefaultFormat(); - - if (!pipeline.isCompleted()) + else { + const ASTQueryWithOutput * ast_query_with_output = dynamic_cast(ast.get()); + + WriteBuffer * out_buf = &ostr; + std::optional out_file_buf; + if (ast_query_with_output && ast_query_with_output->out_file) + { + if (!allow_into_outfile) + throw Exception("INTO OUTFILE is not allowed", ErrorCodes::INTO_OUTFILE_NOT_ALLOWED); + + const auto & out_file = typeid_cast(*ast_query_with_output->out_file).value.safeGet(); + out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT); + out_buf = &*out_file_buf; + } + + String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr) + ? getIdentifierName(ast_query_with_output->format) + : context->getDefaultFormat(); + pipeline.addSimpleTransform([](const Block & header) { return std::make_shared(header); @@ -1105,10 +1109,6 @@ void executeQuery( pipeline.setOutputFormat(std::move(out)); } - else - { - pipeline.setProgressCallback(context->getProgressCallback()); - } { auto executor = pipeline.execute(); @@ -1125,4 +1125,32 @@ void executeQuery( streams.onFinish(); } +void executeTrivialBlockIO(BlockIO & streams, ContextPtr context) +{ + try + { + if (streams.out) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query stream requires input, but no input buffer provided, it's a bug"); + if (streams.in) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query stream requires output, but no output buffer provided, it's a bug"); + + if (!streams.pipeline.initialized()) + return; + + if (!streams.pipeline.isCompleted()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query pipeline requires output, but no output buffer provided, it's a bug"); + + streams.pipeline.setProgressCallback(context->getProgressCallback()); + auto executor = streams.pipeline.execute(); + executor->execute(streams.pipeline.getNumThreads()); + } + catch (...) + { + streams.onException(); + throw; + } + + streams.onFinish(); +} + } diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index 6448b26a652..5eebb09c36b 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -53,4 +53,8 @@ BlockIO executeQuery( bool allow_processors /// If can use processors pipeline ); +/// Executes BlockIO returned from executeQuery(...) +/// if built pipeline does not require any input and does not produce any output. +void executeTrivialBlockIO(BlockIO & streams, ContextPtr context); + } diff --git a/src/Parsers/ASTRenameQuery.h b/src/Parsers/ASTRenameQuery.h index 611f81dc9e9..e0c58e3462e 100644 --- a/src/Parsers/ASTRenameQuery.h +++ b/src/Parsers/ASTRenameQuery.h @@ -34,6 +34,9 @@ public: bool database{false}; /// For RENAME DATABASE bool dictionary{false}; /// For RENAME DICTIONARY + /// Special flag for CREATE OR REPLACE. Do not throw if the second table does not exist. + bool rename_if_cannot_exchange{false}; + /** Get the text that identifies this element. */ String getID(char) const override { return "Rename"; } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f4d6ec5c6f7..f30f2fb8e09 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1137,6 +1137,11 @@ ActionLock StorageDistributed::getActionLock(StorageActionBlockType type) return {}; } +void StorageDistributed::flush() +{ + flushClusterNodesAllData(getContext()); +} + void StorageDistributed::flushClusterNodesAllData(ContextPtr local_context) { /// Sync SYSTEM FLUSH DISTRIBUTED with TRUNCATE diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index c63abbc6aa4..1a74b48a615 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -98,6 +98,7 @@ public: void startup() override; void shutdown() override; + void flush() override; void drop() override; bool storesDataOnDisk() const override { return true; } diff --git a/tests/queries/0_stateless/01157_replace_table.reference b/tests/queries/0_stateless/01157_replace_table.reference new file mode 100644 index 00000000000..9fddaf99847 --- /dev/null +++ b/tests/queries/0_stateless/01157_replace_table.reference @@ -0,0 +1,20 @@ +test flush on replace +1 s1 +2 s2 +3 s3 +exception on create and fill +0 +1 1 s1 +2 2 s2 +3 3 s3 +1 1 s1 +2 2 s2 +3 3 s3 +1 1 s1 +2 2 s2 +3 3 s3 +4 4 s4 +buf +dist +join +t diff --git a/tests/queries/0_stateless/01157_replace_table.sql b/tests/queries/0_stateless/01157_replace_table.sql new file mode 100644 index 00000000000..337290bb4d2 --- /dev/null +++ b/tests/queries/0_stateless/01157_replace_table.sql @@ -0,0 +1,48 @@ +drop table if exists t; +drop table if exists dist; +drop table if exists buf; +drop table if exists join; + +select 'test flush on replace'; +create table t (n UInt64, s String default 's' || toString(n)) engine=Memory; +create table dist (n int) engine=Distributed(test_shard_localhost, currentDatabase(), t); +create table buf (n int) engine=Buffer(currentDatabase(), dist, 1, 10, 100, 10, 100, 1000, 1000); + +insert into buf values (1); +replace table buf (n int) engine=Distributed(test_shard_localhost, currentDatabase(), dist); +replace table dist (n int) engine=Buffer(currentDatabase(), t, 1, 10, 100, 10, 100, 1000, 1000); + +insert into buf values (2); +replace table buf (n int) engine=Buffer(currentDatabase(), dist, 1, 10, 100, 10, 100, 1000, 1000); +replace table dist (n int) engine=Distributed(test_shard_localhost, currentDatabase(), t); + +insert into buf values (3); +replace table buf (n int) engine=Null; +replace table dist (n int) engine=Null; + +select * from t order by n; + +select 'exception on create and fill'; +-- table is not created if select fails +create or replace table join engine=Join(ANY, INNER, n) as select * from t where throwIf(n); -- { serverError 395 } +select count() from system.tables where database=currentDatabase() and name='join'; + +-- table is created and filled +create or replace table join engine=Join(ANY, INNER, n) as select * from t; +select * from numbers(10) as t any join join on t.number=join.n order by n; + +-- table is not replaced if select fails +insert into t(n) values (4); +replace table join engine=Join(ANY, INNER, n) as select * from t where throwIf(n); -- { serverError 395 } +select * from numbers(10) as t any join join on t.number=join.n order by n; + +-- table is replaced +replace table join engine=Join(ANY, INNER, n) as select * from t; +select * from numbers(10) as t any join join on t.number=join.n order by n; + +select name from system.tables where database=currentDatabase() order by name; + +drop table t; +drop table dist; +drop table buf; +drop table join; diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 7c1f998e91d..c9d517b0285 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -109,6 +109,7 @@ "00510_materizlized_view_and_deduplication_zookeeper", "00738_lock_for_inner_table", "01153_attach_mv_uuid", + "01157_replace_table", /// Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database. "rocksdb", "01914_exchange_dictionaries" /// Requires Atomic database From 1b2416007e383270eccd65b789903f28549e2287 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Jul 2021 19:43:59 +0300 Subject: [PATCH 005/147] fix --- src/Interpreters/executeQuery.cpp | 43 +++++++++++++++-------------- src/Storages/StorageDistributed.cpp | 9 +++++- 2 files changed, 30 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 8f9a8b8a3e9..ae52b5992b2 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -77,6 +77,7 @@ namespace ErrorCodes { extern const int INTO_OUTFILE_NOT_ALLOWED; extern const int QUERY_WAS_CANCELLED; + extern const int LOGICAL_ERROR; } @@ -1060,30 +1061,26 @@ void executeQuery( } else if (pipeline.initialized()) { - if (pipeline.isCompleted()) + const ASTQueryWithOutput * ast_query_with_output = dynamic_cast(ast.get()); + + WriteBuffer * out_buf = &ostr; + std::optional out_file_buf; + if (ast_query_with_output && ast_query_with_output->out_file) { - pipeline.setProgressCallback(context->getProgressCallback()); + if (!allow_into_outfile) + throw Exception("INTO OUTFILE is not allowed", ErrorCodes::INTO_OUTFILE_NOT_ALLOWED); + + const auto & out_file = typeid_cast(*ast_query_with_output->out_file).value.safeGet(); + out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT); + out_buf = &*out_file_buf; } - else + + String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr) + ? getIdentifierName(ast_query_with_output->format) + : context->getDefaultFormat(); + + if (!pipeline.isCompleted()) { - const ASTQueryWithOutput * ast_query_with_output = dynamic_cast(ast.get()); - - WriteBuffer * out_buf = &ostr; - std::optional out_file_buf; - if (ast_query_with_output && ast_query_with_output->out_file) - { - if (!allow_into_outfile) - throw Exception("INTO OUTFILE is not allowed", ErrorCodes::INTO_OUTFILE_NOT_ALLOWED); - - const auto & out_file = typeid_cast(*ast_query_with_output->out_file).value.safeGet(); - out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT); - out_buf = &*out_file_buf; - } - - String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr) - ? getIdentifierName(ast_query_with_output->format) - : context->getDefaultFormat(); - pipeline.addSimpleTransform([](const Block & header) { return std::make_shared(header); @@ -1109,6 +1106,10 @@ void executeQuery( pipeline.setOutputFormat(std::move(out)); } + else + { + pipeline.setProgressCallback(context->getProgressCallback()); + } { auto executor = pipeline.execute(); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f30f2fb8e09..740b2cfbfe9 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1139,7 +1139,14 @@ ActionLock StorageDistributed::getActionLock(StorageActionBlockType type) void StorageDistributed::flush() { - flushClusterNodesAllData(getContext()); + try + { + flushClusterNodesAllData(getContext()); + } + catch (...) + { + tryLogCurrentException(log, "Cannot flush"); + } } void StorageDistributed::flushClusterNodesAllData(ContextPtr local_context) From aa3f0b2032fb22d12fe7c909220a52ed0429f0f5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jul 2021 19:39:55 +0300 Subject: [PATCH 006/147] fix --- src/Interpreters/Context.cpp | 7 + src/Interpreters/Context.h | 2 + src/Interpreters/DDLTask.cpp | 4 +- src/Interpreters/DDLTask.h | 13 +- src/Interpreters/InterpreterCreateQuery.cpp | 46 +- src/Parsers/New/AST/CreateDictionaryQuery.cpp | 12 +- src/Parsers/New/AST/CreateDictionaryQuery.h | 7 +- src/Parsers/New/AST/CreateTableQuery.cpp | 17 +- src/Parsers/New/AST/CreateTableQuery.h | 8 +- src/Parsers/New/ClickHouseParser.cpp | 4619 +++++++++-------- src/Parsers/New/ClickHouseParser.g4 | 4 +- src/Parsers/New/ClickHouseParser.h | 4 + .../0_stateless/01157_replace_table.sql | 3 + .../01185_create_or_replace_table.reference | 6 +- .../01185_create_or_replace_table.sql | 37 +- .../01915_create_or_replace_dictionary.sql | 38 +- tests/queries/skip_list.json | 2 +- 17 files changed, 2496 insertions(+), 2333 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a0fce58b472..9b51fc5933d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2677,6 +2677,13 @@ ZooKeeperMetadataTransactionPtr Context::getZooKeeperMetadataTransaction() const return metadata_transaction; } +void Context::resetZooKeeperMetadataTransaction() +{ + assert(metadata_transaction); + assert(hasQueryContext()); + metadata_transaction = nullptr; +} + PartUUIDsPtr Context::getPartUUIDs() const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 7990bd7420b..3556783b695 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -782,6 +782,8 @@ public: void initZooKeeperMetadataTransaction(ZooKeeperMetadataTransactionPtr txn, bool attach_existing = false); /// Returns context of current distributed DDL query or nullptr. ZooKeeperMetadataTransactionPtr getZooKeeperMetadataTransaction() const; + /// Removes context of current distributed DDL. + void resetZooKeeperMetadataTransaction(); struct MySQLWireContext { diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 4fb44738d8d..bf8380f5af6 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int UNKNOWN_FORMAT_VERSION; extern const int UNKNOWN_TYPE_OF_QUERY; extern const int INCONSISTENT_CLUSTER_DEFINITION; + extern const int LOGICAL_ERROR; } HostID HostID::fromString(const String & host_port_str) @@ -401,7 +402,8 @@ UInt32 DDLTaskBase::getLogEntryNumber(const String & log_entry_name) void ZooKeeperMetadataTransaction::commit() { - assert(state == CREATED); + if (state != CREATED) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect state ({}), it's a bug", state); state = FAILED; current_zookeeper->multi(ops); state = COMMITTED; diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 703d691a358..00bb740f9c5 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -20,6 +20,11 @@ namespace fs = std::filesystem; namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + class ASTQueryWithOnCluster; using ZooKeeperPtr = std::shared_ptr; using ClusterPtr = std::shared_ptr; @@ -180,15 +185,19 @@ public: String getDatabaseZooKeeperPath() const { return zookeeper_path; } + ZooKeeperPtr getZooKeeper() const { return current_zookeeper; } + void addOp(Coordination::RequestPtr && op) { - assert(!isExecuted()); + if (isExecuted()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add ZooKeeper operation because query is executed. It's a bug."); ops.emplace_back(op); } void moveOpsTo(Coordination::Requests & other_ops) { - assert(!isExecuted()); + if (isExecuted()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add ZooKeeper operation because query is executed. It's a bug."); std::move(ops.begin(), ops.end(), std::back_inserter(other_ops)); ops.clear(); state = COMMITTED; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ccd92fb650c..c7ed4214e38 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -86,7 +87,6 @@ namespace ErrorCodes extern const int UNKNOWN_DATABASE; extern const int PATH_ACCESS_DENIED; extern const int NOT_IMPLEMENTED; - extern const int UNKNOWN_TABLE; } namespace fs = std::filesystem; @@ -1075,6 +1075,29 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties) { + /// Replicated database requires separate contexts for each DDL query + ContextPtr current_context = getContext(); + ContextMutablePtr create_context = Context::createCopy(current_context); + create_context->setQueryContext(std::const_pointer_cast(current_context)); + + auto make_drop_context = [&](bool on_error) -> ContextMutablePtr + { + ContextMutablePtr drop_context = Context::createCopy(current_context); + drop_context->makeQueryContext(); + if (on_error) + return drop_context; + + if (auto txn = current_context->getZooKeeperMetadataTransaction()) + { + /// Execute drop as separate query, because [CREATE OR] REPLACE query can be considered as + /// successfully executed after RENAME/EXCHANGE query. + drop_context->resetZooKeeperMetadataTransaction(); + auto drop_txn = std::make_shared(txn->getZooKeeper(), txn->getDatabaseZooKeeperPath(), txn->isInitialQuery()); + drop_context->initZooKeeperMetadataTransaction(drop_txn); + } + return drop_context; + }; + auto ast_drop = std::make_shared(); String table_to_replace_name = create.table; @@ -1091,6 +1114,11 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, create.table = fmt::format("_tmp_replace_{}_{}", getHexUIntLowercase(name_hash), getHexUIntLowercase(random_suffix)); + + ast_drop->table = create.table; + ast_drop->is_dictionary = create.is_dictionary; + ast_drop->database = create.database; + ast_drop->kind = ASTDropQuery::Drop; } bool created = false; @@ -1098,12 +1126,8 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, try { /// Create temporary table (random name will be generated) - [[maybe_unused]] bool done = doCreateTable(create, properties); + [[maybe_unused]] bool done = InterpreterCreateQuery(query_ptr, create_context).doCreateTable(create, properties); assert(done); - ast_drop->table = create.table; - ast_drop->is_dictionary = create.is_dictionary; - ast_drop->database = create.database; - ast_drop->kind = ASTDropQuery::Drop; created = true; /// Try fill temporary table @@ -1124,14 +1148,15 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, /// Will execute ordinary RENAME instead of EXCHANGE if the target table does not exist ast_rename->rename_if_cannot_exchange = create.create_or_replace; - InterpreterRenameQuery interpreter_rename{ast_rename, getContext()}; + InterpreterRenameQuery interpreter_rename{ast_rename, current_context}; interpreter_rename.execute(); renamed = true; if (!interpreter_rename.renamedInsteadOfExchange()) { /// Target table was replaced with new one, drop old table - InterpreterDropQuery(ast_drop, getContext()).execute(); + auto drop_context = make_drop_context(false); + InterpreterDropQuery(ast_drop, drop_context).execute(); } create.table = table_to_replace_name; @@ -1142,7 +1167,10 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, { /// Drop temporary table if it was successfully created, but was not renamed to target name if (created && !renamed) - InterpreterDropQuery(ast_drop, getContext()).execute(); + { + auto drop_context = make_drop_context(true); + InterpreterDropQuery(ast_drop, drop_context).execute(); + } throw; } } diff --git a/src/Parsers/New/AST/CreateDictionaryQuery.cpp b/src/Parsers/New/AST/CreateDictionaryQuery.cpp index 75413df495b..dfc996054ee 100644 --- a/src/Parsers/New/AST/CreateDictionaryQuery.cpp +++ b/src/Parsers/New/AST/CreateDictionaryQuery.cpp @@ -226,11 +226,17 @@ CreateDictionaryQuery::CreateDictionaryQuery( PtrTo cluster, bool attach_, bool if_not_exists_, + bool create_or_replace_, + bool replace_, PtrTo identifier, PtrTo uuid, PtrTo schema, PtrTo engine) - : DDLQuery(cluster, {identifier, uuid, schema, engine}), attach(attach_), if_not_exists(if_not_exists_) + : DDLQuery(cluster, {identifier, uuid, schema, engine}) + , attach(attach_) + , if_not_exists(if_not_exists_) + , create_or_replace(create_or_replace_) + , replace(replace_) { } @@ -251,6 +257,8 @@ ASTPtr CreateDictionaryQuery::convertToOld() const query->is_dictionary = true; query->attach = attach; query->if_not_exists = if_not_exists; + query->create_or_replace = create_or_replace; + query->replace_table = replace; query->set(query->dictionary_attributes_list, get(SCHEMA)->convertToOld()); query->set(query->dictionary, get(ENGINE)->convertToOld()); @@ -272,7 +280,7 @@ antlrcpp::Any ParseTreeVisitor::visitCreateDictionaryStmt(ClickHouseParser::Crea auto schema = ctx->dictionarySchemaClause() ? visit(ctx->dictionarySchemaClause()).as>() : nullptr; auto engine = ctx->dictionaryEngineClause() ? visit(ctx->dictionaryEngineClause()).as>() : nullptr; return std::make_shared( - cluster, !!ctx->ATTACH(), !!ctx->IF(), visit(ctx->tableIdentifier()), uuid, schema, engine); + cluster, !!ctx->ATTACH(), !!ctx->IF(), !!ctx->OR(), !!ctx->REPLACE(), visit(ctx->tableIdentifier()), uuid, schema, engine); } antlrcpp::Any ParseTreeVisitor::visitDictionaryArgExpr(ClickHouseParser::DictionaryArgExprContext *ctx) diff --git a/src/Parsers/New/AST/CreateDictionaryQuery.h b/src/Parsers/New/AST/CreateDictionaryQuery.h index 3c5be3f391c..b070c6edd98 100644 --- a/src/Parsers/New/AST/CreateDictionaryQuery.h +++ b/src/Parsers/New/AST/CreateDictionaryQuery.h @@ -161,6 +161,8 @@ class CreateDictionaryQuery : public DDLQuery PtrTo cluster, bool attach, bool if_not_exists, + bool create_or_replace, + bool replace, PtrTo identifier, PtrTo uuid, PtrTo schema, @@ -177,7 +179,10 @@ class CreateDictionaryQuery : public DDLQuery ENGINE, // DictionaryEngineClause }; - const bool attach, if_not_exists; + const bool attach; + const bool if_not_exists; + const bool create_or_replace; + const bool replace; }; } diff --git a/src/Parsers/New/AST/CreateTableQuery.cpp b/src/Parsers/New/AST/CreateTableQuery.cpp index 1767c08451a..b24c34de9d2 100644 --- a/src/Parsers/New/AST/CreateTableQuery.cpp +++ b/src/Parsers/New/AST/CreateTableQuery.cpp @@ -99,12 +99,19 @@ CreateTableQuery::CreateTableQuery( bool attach_, bool temporary_, bool if_not_exists_, + bool create_or_replace_, + bool replace_, PtrTo identifier, PtrTo uuid, PtrTo schema, PtrTo engine, PtrTo query) - : DDLQuery(cluster, {identifier, uuid, schema, engine, query}), attach(attach_), temporary(temporary_), if_not_exists(if_not_exists_) + : DDLQuery(cluster, {identifier, uuid, schema, engine, query}) + , attach(attach_) + , temporary(temporary_) + , if_not_exists(if_not_exists_) + , create_or_replace(create_or_replace_) + , replace(replace_) { } @@ -125,6 +132,8 @@ ASTPtr CreateTableQuery::convertToOld() const query->attach = attach; query->if_not_exists = if_not_exists; query->temporary = temporary; + query->create_or_replace = create_or_replace; + query->replace_table = replace; if (has(SCHEMA)) { @@ -164,6 +173,10 @@ String CreateTableQuery::dumpInfo() const else info += "temporary=false, "; if (if_not_exists) info += "if_not_exists=true"; else info += "if_not_exists=false"; + if (create_or_replace) info += "create_or_replace=true"; + else info += "create_or_replace=false"; + if (replace) info += "replace=true"; + else info += "replace=false"; return info; } @@ -191,7 +204,7 @@ antlrcpp::Any ParseTreeVisitor::visitCreateTableStmt(ClickHouseParser::CreateTab auto engine = ctx->engineClause() ? visit(ctx->engineClause()).as>() : nullptr; auto query = ctx->subqueryClause() ? visit(ctx->subqueryClause()).as>() : nullptr; return std::make_shared( - cluster, !!ctx->ATTACH(), !!ctx->TEMPORARY(), !!ctx->IF(), visit(ctx->tableIdentifier()), uuid, schema, engine, query); + cluster, !!ctx->ATTACH(), !!ctx->TEMPORARY(), !!ctx->IF(), !!ctx->OR(), !!ctx->REPLACE(), visit(ctx->tableIdentifier()), uuid, schema, engine, query); } antlrcpp::Any ParseTreeVisitor::visitSchemaDescriptionClause(ClickHouseParser::SchemaDescriptionClauseContext *ctx) diff --git a/src/Parsers/New/AST/CreateTableQuery.h b/src/Parsers/New/AST/CreateTableQuery.h index 4fe19832b1d..39c43cda902 100644 --- a/src/Parsers/New/AST/CreateTableQuery.h +++ b/src/Parsers/New/AST/CreateTableQuery.h @@ -50,6 +50,8 @@ class CreateTableQuery : public DDLQuery bool attach, bool temporary, bool if_not_exists, + bool create_or_replace, + bool replace, PtrTo identifier, PtrTo uuid, PtrTo schema, @@ -68,7 +70,11 @@ class CreateTableQuery : public DDLQuery SUBQUERY, // SelectUnionQuery }; - const bool attach, temporary, if_not_exists; + const bool attach; + const bool temporary; + const bool if_not_exists; + const bool create_or_replace; + const bool replace; String dumpInfo() const override; }; diff --git a/src/Parsers/New/ClickHouseParser.cpp b/src/Parsers/New/ClickHouseParser.cpp index 174f838f19d..a9405aa0da1 100644 --- a/src/Parsers/New/ClickHouseParser.cpp +++ b/src/Parsers/New/ClickHouseParser.cpp @@ -107,6 +107,7 @@ ClickHouseParser::QueryStmtContext* ClickHouseParser::queryStmt() { case ClickHouseParser::KILL: case ClickHouseParser::OPTIMIZE: case ClickHouseParser::RENAME: + case ClickHouseParser::REPLACE: case ClickHouseParser::SELECT: case ClickHouseParser::SET: case ClickHouseParser::SHOW: @@ -2656,6 +2657,10 @@ tree::TerminalNode* ClickHouseParser::CreateDictionaryStmtContext::CREATE() { return getToken(ClickHouseParser::CREATE, 0); } +tree::TerminalNode* ClickHouseParser::CreateDictionaryStmtContext::REPLACE() { + return getToken(ClickHouseParser::REPLACE, 0); +} + tree::TerminalNode* ClickHouseParser::CreateDictionaryStmtContext::IF() { return getToken(ClickHouseParser::IF, 0); } @@ -2676,6 +2681,10 @@ ClickHouseParser::ClusterClauseContext* ClickHouseParser::CreateDictionaryStmtCo return getRuleContext(0); } +tree::TerminalNode* ClickHouseParser::CreateDictionaryStmtContext::OR() { + return getToken(ClickHouseParser::OR, 0); +} + ClickHouseParser::CreateDictionaryStmtContext::CreateDictionaryStmtContext(CreateStmtContext *ctx) { copyFrom(ctx); } antlrcpp::Any ClickHouseParser::CreateDictionaryStmtContext::accept(tree::ParseTreeVisitor *visitor) { @@ -2892,6 +2901,10 @@ tree::TerminalNode* ClickHouseParser::CreateTableStmtContext::CREATE() { return getToken(ClickHouseParser::CREATE, 0); } +tree::TerminalNode* ClickHouseParser::CreateTableStmtContext::REPLACE() { + return getToken(ClickHouseParser::REPLACE, 0); +} + tree::TerminalNode* ClickHouseParser::CreateTableStmtContext::TEMPORARY() { return getToken(ClickHouseParser::TEMPORARY, 0); } @@ -2928,6 +2941,10 @@ ClickHouseParser::SubqueryClauseContext* ClickHouseParser::CreateTableStmtContex return getRuleContext(0); } +tree::TerminalNode* ClickHouseParser::CreateTableStmtContext::OR() { + return getToken(ClickHouseParser::OR, 0); +} + ClickHouseParser::CreateTableStmtContext::CreateTableStmtContext(CreateStmtContext *ctx) { copyFrom(ctx); } antlrcpp::Any ClickHouseParser::CreateTableStmtContext::accept(tree::ParseTreeVisitor *visitor) { @@ -2945,9 +2962,9 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { exitRule(); }); try { - setState(654); + setState(670); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 71, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 75, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); @@ -3003,55 +3020,78 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(531); - _la = _input->LA(1); - if (!(_la == ClickHouseParser::ATTACH + setState(538); + _errHandler->sync(this); + switch (_input->LA(1)) { + case ClickHouseParser::ATTACH: { + setState(531); + match(ClickHouseParser::ATTACH); + break; + } - || _la == ClickHouseParser::CREATE)) { - _errHandler->recoverInline(this); + case ClickHouseParser::CREATE: { + setState(532); + match(ClickHouseParser::CREATE); + setState(535); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::OR) { + setState(533); + match(ClickHouseParser::OR); + setState(534); + match(ClickHouseParser::REPLACE); + } + break; + } + + case ClickHouseParser::REPLACE: { + setState(537); + match(ClickHouseParser::REPLACE); + break; + } + + default: + throw NoViableAltException(this); } - else { - _errHandler->reportMatch(this); - consume(); - } - setState(532); + setState(540); match(ClickHouseParser::DICTIONARY); - setState(536); + setState(544); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 43, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 45, _ctx)) { case 1: { - setState(533); + setState(541); match(ClickHouseParser::IF); - setState(534); + setState(542); match(ClickHouseParser::NOT); - setState(535); + setState(543); match(ClickHouseParser::EXISTS); break; } } - setState(538); + setState(546); tableIdentifier(); - setState(540); + setState(548); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::UUID) { - setState(539); + setState(547); uuidClause(); } - setState(543); + setState(551); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(542); + setState(550); clusterClause(); } - setState(545); + setState(553); dictionarySchemaClause(); - setState(546); + setState(554); dictionaryEngineClause(); break; } @@ -3059,7 +3099,7 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(548); + setState(556); _la = _input->LA(1); if (!(_la == ClickHouseParser::ATTACH @@ -3070,81 +3110,81 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { _errHandler->reportMatch(this); consume(); } - setState(549); + setState(557); match(ClickHouseParser::LIVE); - setState(550); + setState(558); match(ClickHouseParser::VIEW); - setState(554); + setState(562); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 46, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 48, _ctx)) { case 1: { - setState(551); + setState(559); match(ClickHouseParser::IF); - setState(552); + setState(560); match(ClickHouseParser::NOT); - setState(553); + setState(561); match(ClickHouseParser::EXISTS); break; } } - setState(556); + setState(564); tableIdentifier(); - setState(558); + setState(566); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::UUID) { - setState(557); + setState(565); uuidClause(); } - setState(561); + setState(569); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(560); + setState(568); clusterClause(); } - setState(568); + setState(576); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::WITH) { - setState(563); + setState(571); match(ClickHouseParser::WITH); - setState(564); + setState(572); match(ClickHouseParser::TIMEOUT); - setState(566); + setState(574); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::DECIMAL_LITERAL) { - setState(565); + setState(573); match(ClickHouseParser::DECIMAL_LITERAL); } } - setState(571); + setState(579); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TO) { - setState(570); + setState(578); destinationClause(); } - setState(574); + setState(582); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 52, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 54, _ctx)) { case 1: { - setState(573); + setState(581); tableSchemaClause(); break; } } - setState(576); + setState(584); subqueryClause(); break; } @@ -3152,7 +3192,7 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { case 4: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 4); - setState(578); + setState(586); _la = _input->LA(1); if (!(_la == ClickHouseParser::ATTACH @@ -3163,69 +3203,69 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { _errHandler->reportMatch(this); consume(); } - setState(579); + setState(587); match(ClickHouseParser::MATERIALIZED); - setState(580); + setState(588); match(ClickHouseParser::VIEW); - setState(584); + setState(592); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 53, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 55, _ctx)) { case 1: { - setState(581); + setState(589); match(ClickHouseParser::IF); - setState(582); + setState(590); match(ClickHouseParser::NOT); - setState(583); + setState(591); match(ClickHouseParser::EXISTS); break; } } - setState(586); + setState(594); tableIdentifier(); - setState(588); + setState(596); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::UUID) { - setState(587); + setState(595); uuidClause(); } - setState(591); + setState(599); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(590); + setState(598); clusterClause(); } - setState(594); + setState(602); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::AS || _la == ClickHouseParser::LPAREN) { - setState(593); + setState(601); tableSchemaClause(); } - setState(601); + setState(609); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::TO: { - setState(596); + setState(604); destinationClause(); break; } case ClickHouseParser::ENGINE: { - setState(597); + setState(605); engineClause(); - setState(599); + setState(607); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::POPULATE) { - setState(598); + setState(606); match(ClickHouseParser::POPULATE); } break; @@ -3234,7 +3274,7 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { default: throw NoViableAltException(this); } - setState(603); + setState(611); subqueryClause(); break; } @@ -3242,85 +3282,108 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { case 5: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 5); - setState(605); - _la = _input->LA(1); - if (!(_la == ClickHouseParser::ATTACH + setState(620); + _errHandler->sync(this); + switch (_input->LA(1)) { + case ClickHouseParser::ATTACH: { + setState(613); + match(ClickHouseParser::ATTACH); + break; + } - || _la == ClickHouseParser::CREATE)) { - _errHandler->recoverInline(this); + case ClickHouseParser::CREATE: { + setState(614); + match(ClickHouseParser::CREATE); + setState(617); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::OR) { + setState(615); + match(ClickHouseParser::OR); + setState(616); + match(ClickHouseParser::REPLACE); + } + break; + } + + case ClickHouseParser::REPLACE: { + setState(619); + match(ClickHouseParser::REPLACE); + break; + } + + default: + throw NoViableAltException(this); } - else { - _errHandler->reportMatch(this); - consume(); - } - setState(607); + setState(623); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TEMPORARY) { - setState(606); + setState(622); match(ClickHouseParser::TEMPORARY); } - setState(609); + setState(625); match(ClickHouseParser::TABLE); - setState(613); + setState(629); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 60, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 64, _ctx)) { case 1: { - setState(610); + setState(626); match(ClickHouseParser::IF); - setState(611); + setState(627); match(ClickHouseParser::NOT); - setState(612); + setState(628); match(ClickHouseParser::EXISTS); break; } } - setState(615); + setState(631); tableIdentifier(); - setState(617); + setState(633); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::UUID) { - setState(616); + setState(632); uuidClause(); } - setState(620); + setState(636); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(619); + setState(635); clusterClause(); } - setState(623); + setState(639); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 63, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 67, _ctx)) { case 1: { - setState(622); + setState(638); tableSchemaClause(); break; } } - setState(626); + setState(642); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ENGINE) { - setState(625); + setState(641); engineClause(); } - setState(629); + setState(645); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::AS) { - setState(628); + setState(644); subqueryClause(); } break; @@ -3329,7 +3392,7 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { case 6: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 6); - setState(631); + setState(647); _la = _input->LA(1); if (!(_la == ClickHouseParser::ATTACH @@ -3340,63 +3403,63 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { _errHandler->reportMatch(this); consume(); } - setState(634); + setState(650); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OR) { - setState(632); + setState(648); match(ClickHouseParser::OR); - setState(633); + setState(649); match(ClickHouseParser::REPLACE); } - setState(636); + setState(652); match(ClickHouseParser::VIEW); - setState(640); + setState(656); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 67, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 71, _ctx)) { case 1: { - setState(637); + setState(653); match(ClickHouseParser::IF); - setState(638); + setState(654); match(ClickHouseParser::NOT); - setState(639); + setState(655); match(ClickHouseParser::EXISTS); break; } } - setState(642); + setState(658); tableIdentifier(); - setState(644); + setState(660); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::UUID) { - setState(643); + setState(659); uuidClause(); } - setState(647); + setState(663); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(646); + setState(662); clusterClause(); } - setState(650); + setState(666); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 70, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 74, _ctx)) { case 1: { - setState(649); + setState(665); tableSchemaClause(); break; } } - setState(652); + setState(668); subqueryClause(); break; } @@ -3465,23 +3528,23 @@ ClickHouseParser::DictionarySchemaClauseContext* ClickHouseParser::dictionarySch }); try { enterOuterAlt(_localctx, 1); - setState(656); + setState(672); match(ClickHouseParser::LPAREN); - setState(657); + setState(673); dictionaryAttrDfnt(); - setState(662); + setState(678); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(658); + setState(674); match(ClickHouseParser::COMMA); - setState(659); + setState(675); dictionaryAttrDfnt(); - setState(664); + setState(680); _errHandler->sync(this); _la = _input->LA(1); } - setState(665); + setState(681); match(ClickHouseParser::RPAREN); } @@ -3586,67 +3649,67 @@ ClickHouseParser::DictionaryAttrDfntContext* ClickHouseParser::dictionaryAttrDfn try { size_t alt; enterOuterAlt(_localctx, 1); - setState(667); + setState(683); identifier(); - setState(668); + setState(684); columnTypeExpr(); - setState(690); + setState(706); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 74, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 78, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(688); + setState(704); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 73, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 77, _ctx)) { case 1: { - setState(669); + setState(685); if (!(!_localctx->attrs.count("default"))) throw FailedPredicateException(this, "!$attrs.count(\"default\")"); - setState(670); + setState(686); match(ClickHouseParser::DEFAULT); - setState(671); + setState(687); literal(); _localctx->attrs.insert("default"); break; } case 2: { - setState(674); + setState(690); if (!(!_localctx->attrs.count("expression"))) throw FailedPredicateException(this, "!$attrs.count(\"expression\")"); - setState(675); + setState(691); match(ClickHouseParser::EXPRESSION); - setState(676); + setState(692); columnExpr(0); _localctx->attrs.insert("expression"); break; } case 3: { - setState(679); + setState(695); if (!(!_localctx->attrs.count("hierarchical"))) throw FailedPredicateException(this, "!$attrs.count(\"hierarchical\")"); - setState(680); + setState(696); match(ClickHouseParser::HIERARCHICAL); _localctx->attrs.insert("hierarchical"); break; } case 4: { - setState(682); + setState(698); if (!(!_localctx->attrs.count("injective"))) throw FailedPredicateException(this, "!$attrs.count(\"injective\")"); - setState(683); + setState(699); match(ClickHouseParser::INJECTIVE); _localctx->attrs.insert("injective"); break; } case 5: { - setState(685); + setState(701); if (!(!_localctx->attrs.count("is_object_id"))) throw FailedPredicateException(this, "!$attrs.count(\"is_object_id\")"); - setState(686); + setState(702); match(ClickHouseParser::IS_OBJECT_ID); _localctx->attrs.insert("is_object_id"); break; @@ -3654,9 +3717,9 @@ ClickHouseParser::DictionaryAttrDfntContext* ClickHouseParser::dictionaryAttrDfn } } - setState(692); + setState(708); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 74, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 78, _ctx); } } @@ -3741,70 +3804,70 @@ ClickHouseParser::DictionaryEngineClauseContext* ClickHouseParser::dictionaryEng try { size_t alt; enterOuterAlt(_localctx, 1); - setState(694); + setState(710); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 75, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 79, _ctx)) { case 1: { - setState(693); + setState(709); dictionaryPrimaryKeyClause(); break; } } - setState(718); + setState(734); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 77, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 81, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(716); + setState(732); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 76, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 80, _ctx)) { case 1: { - setState(696); + setState(712); if (!(!_localctx->clauses.count("source"))) throw FailedPredicateException(this, "!$clauses.count(\"source\")"); - setState(697); + setState(713); sourceClause(); _localctx->clauses.insert("source"); break; } case 2: { - setState(700); + setState(716); if (!(!_localctx->clauses.count("lifetime"))) throw FailedPredicateException(this, "!$clauses.count(\"lifetime\")"); - setState(701); + setState(717); lifetimeClause(); _localctx->clauses.insert("lifetime"); break; } case 3: { - setState(704); + setState(720); if (!(!_localctx->clauses.count("layout"))) throw FailedPredicateException(this, "!$clauses.count(\"layout\")"); - setState(705); + setState(721); layoutClause(); _localctx->clauses.insert("layout"); break; } case 4: { - setState(708); + setState(724); if (!(!_localctx->clauses.count("range"))) throw FailedPredicateException(this, "!$clauses.count(\"range\")"); - setState(709); + setState(725); rangeClause(); _localctx->clauses.insert("range"); break; } case 5: { - setState(712); + setState(728); if (!(!_localctx->clauses.count("settings"))) throw FailedPredicateException(this, "!$clauses.count(\"settings\")"); - setState(713); + setState(729); dictionarySettingsClause(); _localctx->clauses.insert("settings"); break; @@ -3812,9 +3875,9 @@ ClickHouseParser::DictionaryEngineClauseContext* ClickHouseParser::dictionaryEng } } - setState(720); + setState(736); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 77, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 81, _ctx); } } @@ -3866,11 +3929,11 @@ ClickHouseParser::DictionaryPrimaryKeyClauseContext* ClickHouseParser::dictionar }); try { enterOuterAlt(_localctx, 1); - setState(721); + setState(737); match(ClickHouseParser::PRIMARY); - setState(722); + setState(738); match(ClickHouseParser::KEY); - setState(723); + setState(739); columnExprList(); } @@ -3931,9 +3994,9 @@ ClickHouseParser::DictionaryArgExprContext* ClickHouseParser::dictionaryArgExpr( }); try { enterOuterAlt(_localctx, 1); - setState(725); + setState(741); identifier(); - setState(732); + setState(748); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::AFTER: @@ -4115,16 +4178,16 @@ ClickHouseParser::DictionaryArgExprContext* ClickHouseParser::dictionaryArgExpr( case ClickHouseParser::JSON_FALSE: case ClickHouseParser::JSON_TRUE: case ClickHouseParser::IDENTIFIER: { - setState(726); + setState(742); identifier(); - setState(729); + setState(745); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::LPAREN) { - setState(727); + setState(743); match(ClickHouseParser::LPAREN); - setState(728); + setState(744); match(ClickHouseParser::RPAREN); } break; @@ -4141,7 +4204,7 @@ ClickHouseParser::DictionaryArgExprContext* ClickHouseParser::dictionaryArgExpr( case ClickHouseParser::DASH: case ClickHouseParser::DOT: case ClickHouseParser::PLUS: { - setState(731); + setState(747); literal(); break; } @@ -4220,15 +4283,15 @@ ClickHouseParser::SourceClauseContext* ClickHouseParser::sourceClause() { }); try { enterOuterAlt(_localctx, 1); - setState(734); + setState(750); match(ClickHouseParser::SOURCE); - setState(735); + setState(751); match(ClickHouseParser::LPAREN); - setState(736); + setState(752); identifier(); - setState(737); + setState(753); match(ClickHouseParser::LPAREN); - setState(741); + setState(757); _errHandler->sync(this); _la = _input->LA(1); while ((((_la & ~ 0x3fULL) == 0) && @@ -4411,15 +4474,15 @@ ClickHouseParser::SourceClauseContext* ClickHouseParser::sourceClause() { | (1ULL << (ClickHouseParser::JSON_FALSE - 128)) | (1ULL << (ClickHouseParser::JSON_TRUE - 128)) | (1ULL << (ClickHouseParser::IDENTIFIER - 128)))) != 0)) { - setState(738); + setState(754); dictionaryArgExpr(); - setState(743); + setState(759); _errHandler->sync(this); _la = _input->LA(1); } - setState(744); + setState(760); match(ClickHouseParser::RPAREN); - setState(745); + setState(761); match(ClickHouseParser::RPAREN); } @@ -4487,39 +4550,39 @@ ClickHouseParser::LifetimeClauseContext* ClickHouseParser::lifetimeClause() { }); try { enterOuterAlt(_localctx, 1); - setState(747); + setState(763); match(ClickHouseParser::LIFETIME); - setState(748); + setState(764); match(ClickHouseParser::LPAREN); - setState(758); + setState(774); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::DECIMAL_LITERAL: { - setState(749); + setState(765); match(ClickHouseParser::DECIMAL_LITERAL); break; } case ClickHouseParser::MIN: { - setState(750); + setState(766); match(ClickHouseParser::MIN); - setState(751); + setState(767); match(ClickHouseParser::DECIMAL_LITERAL); - setState(752); + setState(768); match(ClickHouseParser::MAX); - setState(753); + setState(769); match(ClickHouseParser::DECIMAL_LITERAL); break; } case ClickHouseParser::MAX: { - setState(754); + setState(770); match(ClickHouseParser::MAX); - setState(755); + setState(771); match(ClickHouseParser::DECIMAL_LITERAL); - setState(756); + setState(772); match(ClickHouseParser::MIN); - setState(757); + setState(773); match(ClickHouseParser::DECIMAL_LITERAL); break; } @@ -4527,7 +4590,7 @@ ClickHouseParser::LifetimeClauseContext* ClickHouseParser::lifetimeClause() { default: throw NoViableAltException(this); } - setState(760); + setState(776); match(ClickHouseParser::RPAREN); } @@ -4600,15 +4663,15 @@ ClickHouseParser::LayoutClauseContext* ClickHouseParser::layoutClause() { }); try { enterOuterAlt(_localctx, 1); - setState(762); + setState(778); match(ClickHouseParser::LAYOUT); - setState(763); + setState(779); match(ClickHouseParser::LPAREN); - setState(764); + setState(780); identifier(); - setState(765); + setState(781); match(ClickHouseParser::LPAREN); - setState(769); + setState(785); _errHandler->sync(this); _la = _input->LA(1); while ((((_la & ~ 0x3fULL) == 0) && @@ -4791,15 +4854,15 @@ ClickHouseParser::LayoutClauseContext* ClickHouseParser::layoutClause() { | (1ULL << (ClickHouseParser::JSON_FALSE - 128)) | (1ULL << (ClickHouseParser::JSON_TRUE - 128)) | (1ULL << (ClickHouseParser::IDENTIFIER - 128)))) != 0)) { - setState(766); + setState(782); dictionaryArgExpr(); - setState(771); + setState(787); _errHandler->sync(this); _la = _input->LA(1); } - setState(772); + setState(788); match(ClickHouseParser::RPAREN); - setState(773); + setState(789); match(ClickHouseParser::RPAREN); } @@ -4867,33 +4930,33 @@ ClickHouseParser::RangeClauseContext* ClickHouseParser::rangeClause() { }); try { enterOuterAlt(_localctx, 1); - setState(775); + setState(791); match(ClickHouseParser::RANGE); - setState(776); + setState(792); match(ClickHouseParser::LPAREN); - setState(787); + setState(803); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::MIN: { - setState(777); + setState(793); match(ClickHouseParser::MIN); - setState(778); + setState(794); identifier(); - setState(779); + setState(795); match(ClickHouseParser::MAX); - setState(780); + setState(796); identifier(); break; } case ClickHouseParser::MAX: { - setState(782); + setState(798); match(ClickHouseParser::MAX); - setState(783); + setState(799); identifier(); - setState(784); + setState(800); match(ClickHouseParser::MIN); - setState(785); + setState(801); identifier(); break; } @@ -4901,7 +4964,7 @@ ClickHouseParser::RangeClauseContext* ClickHouseParser::rangeClause() { default: throw NoViableAltException(this); } - setState(789); + setState(805); match(ClickHouseParser::RPAREN); } @@ -4957,13 +5020,13 @@ ClickHouseParser::DictionarySettingsClauseContext* ClickHouseParser::dictionaryS }); try { enterOuterAlt(_localctx, 1); - setState(791); + setState(807); match(ClickHouseParser::SETTINGS); - setState(792); + setState(808); match(ClickHouseParser::LPAREN); - setState(793); + setState(809); settingExprList(); - setState(794); + setState(810); match(ClickHouseParser::RPAREN); } @@ -5019,11 +5082,11 @@ ClickHouseParser::ClusterClauseContext* ClickHouseParser::clusterClause() { }); try { enterOuterAlt(_localctx, 1); - setState(796); + setState(812); match(ClickHouseParser::ON); - setState(797); + setState(813); match(ClickHouseParser::CLUSTER); - setState(800); + setState(816); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::AFTER: @@ -5205,13 +5268,13 @@ ClickHouseParser::ClusterClauseContext* ClickHouseParser::clusterClause() { case ClickHouseParser::JSON_FALSE: case ClickHouseParser::JSON_TRUE: case ClickHouseParser::IDENTIFIER: { - setState(798); + setState(814); identifier(); break; } case ClickHouseParser::STRING_LITERAL: { - setState(799); + setState(815); match(ClickHouseParser::STRING_LITERAL); break; } @@ -5265,9 +5328,9 @@ ClickHouseParser::UuidClauseContext* ClickHouseParser::uuidClause() { }); try { enterOuterAlt(_localctx, 1); - setState(802); + setState(818); match(ClickHouseParser::UUID); - setState(803); + setState(819); match(ClickHouseParser::STRING_LITERAL); } @@ -5315,9 +5378,9 @@ ClickHouseParser::DestinationClauseContext* ClickHouseParser::destinationClause( }); try { enterOuterAlt(_localctx, 1); - setState(805); + setState(821); match(ClickHouseParser::TO); - setState(806); + setState(822); tableIdentifier(); } @@ -5365,9 +5428,9 @@ ClickHouseParser::SubqueryClauseContext* ClickHouseParser::subqueryClause() { }); try { enterOuterAlt(_localctx, 1); - setState(808); + setState(824); match(ClickHouseParser::AS); - setState(809); + setState(825); selectUnionStmt(); } @@ -5474,29 +5537,29 @@ ClickHouseParser::TableSchemaClauseContext* ClickHouseParser::tableSchemaClause( exitRule(); }); try { - setState(826); + setState(842); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 86, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 90, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(811); + setState(827); match(ClickHouseParser::LPAREN); - setState(812); + setState(828); tableElementExpr(); - setState(817); + setState(833); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(813); + setState(829); match(ClickHouseParser::COMMA); - setState(814); + setState(830); tableElementExpr(); - setState(819); + setState(835); _errHandler->sync(this); _la = _input->LA(1); } - setState(820); + setState(836); match(ClickHouseParser::RPAREN); break; } @@ -5504,9 +5567,9 @@ ClickHouseParser::TableSchemaClauseContext* ClickHouseParser::tableSchemaClause( case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(822); + setState(838); match(ClickHouseParser::AS); - setState(823); + setState(839); tableIdentifier(); break; } @@ -5514,9 +5577,9 @@ ClickHouseParser::TableSchemaClauseContext* ClickHouseParser::tableSchemaClause( case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(824); + setState(840); match(ClickHouseParser::AS); - setState(825); + setState(841); tableFunctionExpr(); break; } @@ -5613,71 +5676,71 @@ ClickHouseParser::EngineClauseContext* ClickHouseParser::engineClause() { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(828); + setState(844); engineExpr(); - setState(855); + setState(871); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 88, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 92, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(853); + setState(869); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 87, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 91, _ctx)) { case 1: { - setState(829); + setState(845); if (!(!_localctx->clauses.count("orderByClause"))) throw FailedPredicateException(this, "!$clauses.count(\"orderByClause\")"); - setState(830); + setState(846); orderByClause(); _localctx->clauses.insert("orderByClause"); break; } case 2: { - setState(833); + setState(849); if (!(!_localctx->clauses.count("partitionByClause"))) throw FailedPredicateException(this, "!$clauses.count(\"partitionByClause\")"); - setState(834); + setState(850); partitionByClause(); _localctx->clauses.insert("partitionByClause"); break; } case 3: { - setState(837); + setState(853); if (!(!_localctx->clauses.count("primaryKeyClause"))) throw FailedPredicateException(this, "!$clauses.count(\"primaryKeyClause\")"); - setState(838); + setState(854); primaryKeyClause(); _localctx->clauses.insert("primaryKeyClause"); break; } case 4: { - setState(841); + setState(857); if (!(!_localctx->clauses.count("sampleByClause"))) throw FailedPredicateException(this, "!$clauses.count(\"sampleByClause\")"); - setState(842); + setState(858); sampleByClause(); _localctx->clauses.insert("sampleByClause"); break; } case 5: { - setState(845); + setState(861); if (!(!_localctx->clauses.count("ttlClause"))) throw FailedPredicateException(this, "!$clauses.count(\"ttlClause\")"); - setState(846); + setState(862); ttlClause(); _localctx->clauses.insert("ttlClause"); break; } case 6: { - setState(849); + setState(865); if (!(!_localctx->clauses.count("settingsClause"))) throw FailedPredicateException(this, "!$clauses.count(\"settingsClause\")"); - setState(850); + setState(866); settingsClause(); _localctx->clauses.insert("settingsClause"); break; @@ -5685,9 +5748,9 @@ ClickHouseParser::EngineClauseContext* ClickHouseParser::engineClause() { } } - setState(857); + setState(873); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 88, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 92, _ctx); } } @@ -5739,11 +5802,11 @@ ClickHouseParser::PartitionByClauseContext* ClickHouseParser::partitionByClause( }); try { enterOuterAlt(_localctx, 1); - setState(858); + setState(874); match(ClickHouseParser::PARTITION); - setState(859); + setState(875); match(ClickHouseParser::BY); - setState(860); + setState(876); columnExpr(0); } @@ -5795,11 +5858,11 @@ ClickHouseParser::PrimaryKeyClauseContext* ClickHouseParser::primaryKeyClause() }); try { enterOuterAlt(_localctx, 1); - setState(862); + setState(878); match(ClickHouseParser::PRIMARY); - setState(863); + setState(879); match(ClickHouseParser::KEY); - setState(864); + setState(880); columnExpr(0); } @@ -5851,11 +5914,11 @@ ClickHouseParser::SampleByClauseContext* ClickHouseParser::sampleByClause() { }); try { enterOuterAlt(_localctx, 1); - setState(866); + setState(882); match(ClickHouseParser::SAMPLE); - setState(867); + setState(883); match(ClickHouseParser::BY); - setState(868); + setState(884); columnExpr(0); } @@ -5916,23 +5979,23 @@ ClickHouseParser::TtlClauseContext* ClickHouseParser::ttlClause() { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(870); + setState(886); match(ClickHouseParser::TTL); - setState(871); + setState(887); ttlExpr(); - setState(876); + setState(892); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 89, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 93, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(872); + setState(888); match(ClickHouseParser::COMMA); - setState(873); + setState(889); ttlExpr(); } - setState(878); + setState(894); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 89, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 93, _ctx); } } @@ -5997,26 +6060,26 @@ ClickHouseParser::EngineExprContext* ClickHouseParser::engineExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(879); + setState(895); match(ClickHouseParser::ENGINE); - setState(881); + setState(897); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::EQ_SINGLE) { - setState(880); + setState(896); match(ClickHouseParser::EQ_SINGLE); } - setState(883); + setState(899); identifierOrNull(); - setState(889); + setState(905); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 92, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 96, _ctx)) { case 1: { - setState(884); + setState(900); match(ClickHouseParser::LPAREN); - setState(886); + setState(902); _errHandler->sync(this); _la = _input->LA(1); @@ -6214,10 +6277,10 @@ ClickHouseParser::EngineExprContext* ClickHouseParser::engineExpr() { | (1ULL << (ClickHouseParser::LBRACKET - 197)) | (1ULL << (ClickHouseParser::LPAREN - 197)) | (1ULL << (ClickHouseParser::PLUS - 197)))) != 0)) { - setState(885); + setState(901); columnExprList(); } - setState(888); + setState(904); match(ClickHouseParser::RPAREN); break; } @@ -6333,13 +6396,13 @@ ClickHouseParser::TableElementExprContext* ClickHouseParser::tableElementExpr() exitRule(); }); try { - setState(901); + setState(917); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 93, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 97, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(891); + setState(907); tableColumnDfnt(); break; } @@ -6347,13 +6410,13 @@ ClickHouseParser::TableElementExprContext* ClickHouseParser::tableElementExpr() case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(892); + setState(908); match(ClickHouseParser::CONSTRAINT); - setState(893); + setState(909); identifier(); - setState(894); + setState(910); match(ClickHouseParser::CHECK); - setState(895); + setState(911); columnExpr(0); break; } @@ -6361,9 +6424,9 @@ ClickHouseParser::TableElementExprContext* ClickHouseParser::tableElementExpr() case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(897); + setState(913); match(ClickHouseParser::INDEX); - setState(898); + setState(914); tableIndexDfnt(); break; } @@ -6371,9 +6434,9 @@ ClickHouseParser::TableElementExprContext* ClickHouseParser::tableElementExpr() case 4: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 4); - setState(899); + setState(915); match(ClickHouseParser::PROJECTION); - setState(900); + setState(916); tableProjectionDfnt(); break; } @@ -6449,73 +6512,25 @@ ClickHouseParser::TableColumnDfntContext* ClickHouseParser::tableColumnDfnt() { exitRule(); }); try { - setState(935); + setState(951); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 102, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 106, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(903); + setState(919); nestedIdentifier(); - setState(904); + setState(920); columnTypeExpr(); - setState(906); + setState(922); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ALIAS || _la == ClickHouseParser::DEFAULT || _la == ClickHouseParser::MATERIALIZED) { - setState(905); + setState(921); tableColumnPropertyExpr(); } - setState(910); - _errHandler->sync(this); - - _la = _input->LA(1); - if (_la == ClickHouseParser::COMMENT) { - setState(908); - match(ClickHouseParser::COMMENT); - setState(909); - match(ClickHouseParser::STRING_LITERAL); - } - setState(913); - _errHandler->sync(this); - - _la = _input->LA(1); - if (_la == ClickHouseParser::CODEC) { - setState(912); - codecExpr(); - } - setState(917); - _errHandler->sync(this); - - _la = _input->LA(1); - if (_la == ClickHouseParser::TTL) { - setState(915); - match(ClickHouseParser::TTL); - setState(916); - columnExpr(0); - } - break; - } - - case 2: { - enterOuterAlt(_localctx, 2); - setState(919); - nestedIdentifier(); - setState(921); - _errHandler->sync(this); - - switch (getInterpreter()->adaptivePredict(_input, 98, _ctx)) { - case 1: { - setState(920); - columnTypeExpr(); - break; - } - - } - setState(923); - tableColumnPropertyExpr(); setState(926); _errHandler->sync(this); @@ -6547,6 +6562,54 @@ ClickHouseParser::TableColumnDfntContext* ClickHouseParser::tableColumnDfnt() { break; } + case 2: { + enterOuterAlt(_localctx, 2); + setState(935); + nestedIdentifier(); + setState(937); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 102, _ctx)) { + case 1: { + setState(936); + columnTypeExpr(); + break; + } + + } + setState(939); + tableColumnPropertyExpr(); + setState(942); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::COMMENT) { + setState(940); + match(ClickHouseParser::COMMENT); + setState(941); + match(ClickHouseParser::STRING_LITERAL); + } + setState(945); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::CODEC) { + setState(944); + codecExpr(); + } + setState(949); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::TTL) { + setState(947); + match(ClickHouseParser::TTL); + setState(948); + columnExpr(0); + } + break; + } + } } @@ -6603,7 +6666,7 @@ ClickHouseParser::TableColumnPropertyExprContext* ClickHouseParser::tableColumnP }); try { enterOuterAlt(_localctx, 1); - setState(937); + setState(953); _la = _input->LA(1); if (!(_la == ClickHouseParser::ALIAS @@ -6614,7 +6677,7 @@ ClickHouseParser::TableColumnPropertyExprContext* ClickHouseParser::tableColumnP _errHandler->reportMatch(this); consume(); } - setState(938); + setState(954); columnExpr(0); } @@ -6678,17 +6741,17 @@ ClickHouseParser::TableIndexDfntContext* ClickHouseParser::tableIndexDfnt() { }); try { enterOuterAlt(_localctx, 1); - setState(940); + setState(956); nestedIdentifier(); - setState(941); + setState(957); columnExpr(0); - setState(942); + setState(958); match(ClickHouseParser::TYPE); - setState(943); + setState(959); columnTypeExpr(); - setState(944); + setState(960); match(ClickHouseParser::GRANULARITY); - setState(945); + setState(961); match(ClickHouseParser::DECIMAL_LITERAL); } @@ -6736,9 +6799,9 @@ ClickHouseParser::TableProjectionDfntContext* ClickHouseParser::tableProjectionD }); try { enterOuterAlt(_localctx, 1); - setState(947); + setState(963); nestedIdentifier(); - setState(948); + setState(964); projectionSelectStmt(); } @@ -6807,25 +6870,25 @@ ClickHouseParser::CodecExprContext* ClickHouseParser::codecExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(950); + setState(966); match(ClickHouseParser::CODEC); - setState(951); + setState(967); match(ClickHouseParser::LPAREN); - setState(952); + setState(968); codecArgExpr(); - setState(957); + setState(973); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(953); + setState(969); match(ClickHouseParser::COMMA); - setState(954); + setState(970); codecArgExpr(); - setState(959); + setState(975); _errHandler->sync(this); _la = _input->LA(1); } - setState(960); + setState(976); match(ClickHouseParser::RPAREN); } @@ -6882,16 +6945,16 @@ ClickHouseParser::CodecArgExprContext* ClickHouseParser::codecArgExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(962); + setState(978); identifier(); - setState(968); + setState(984); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::LPAREN) { - setState(963); + setState(979); match(ClickHouseParser::LPAREN); - setState(965); + setState(981); _errHandler->sync(this); _la = _input->LA(1); @@ -7089,10 +7152,10 @@ ClickHouseParser::CodecArgExprContext* ClickHouseParser::codecArgExpr() { | (1ULL << (ClickHouseParser::LBRACKET - 197)) | (1ULL << (ClickHouseParser::LPAREN - 197)) | (1ULL << (ClickHouseParser::PLUS - 197)))) != 0)) { - setState(964); + setState(980); columnExprList(); } - setState(967); + setState(983); match(ClickHouseParser::RPAREN); } @@ -7157,34 +7220,34 @@ ClickHouseParser::TtlExprContext* ClickHouseParser::ttlExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(970); + setState(986); columnExpr(0); - setState(978); + setState(994); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 106, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 110, _ctx)) { case 1: { - setState(971); + setState(987); match(ClickHouseParser::DELETE); break; } case 2: { - setState(972); + setState(988); match(ClickHouseParser::TO); - setState(973); + setState(989); match(ClickHouseParser::DISK); - setState(974); + setState(990); match(ClickHouseParser::STRING_LITERAL); break; } case 3: { - setState(975); + setState(991); match(ClickHouseParser::TO); - setState(976); + setState(992); match(ClickHouseParser::VOLUME); - setState(977); + setState(993); match(ClickHouseParser::STRING_LITERAL); break; } @@ -7245,7 +7308,7 @@ ClickHouseParser::DescribeStmtContext* ClickHouseParser::describeStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(980); + setState(996); _la = _input->LA(1); if (!(_la == ClickHouseParser::DESC @@ -7256,18 +7319,18 @@ ClickHouseParser::DescribeStmtContext* ClickHouseParser::describeStmt() { _errHandler->reportMatch(this); consume(); } - setState(982); + setState(998); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 107, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 111, _ctx)) { case 1: { - setState(981); + setState(997); match(ClickHouseParser::TABLE); break; } } - setState(984); + setState(1000); tableExpr(0); } @@ -7400,55 +7463,13 @@ ClickHouseParser::DropStmtContext* ClickHouseParser::dropStmt() { exitRule(); }); try { - setState(1017); + setState(1033); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 115, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 119, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(986); - _la = _input->LA(1); - if (!(_la == ClickHouseParser::DETACH - - || _la == ClickHouseParser::DROP)) { - _errHandler->recoverInline(this); - } - else { - _errHandler->reportMatch(this); - consume(); - } - setState(987); - match(ClickHouseParser::DATABASE); - setState(990); - _errHandler->sync(this); - - switch (getInterpreter()->adaptivePredict(_input, 108, _ctx)) { - case 1: { - setState(988); - match(ClickHouseParser::IF); - setState(989); - match(ClickHouseParser::EXISTS); - break; - } - - } - setState(992); - databaseIdentifier(); - setState(994); - _errHandler->sync(this); - - _la = _input->LA(1); - if (_la == ClickHouseParser::ON) { - setState(993); - clusterClause(); - } - break; - } - - case 2: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 2); - setState(996); + setState(1002); _la = _input->LA(1); if (!(_la == ClickHouseParser::DETACH @@ -7460,31 +7481,73 @@ ClickHouseParser::DropStmtContext* ClickHouseParser::dropStmt() { consume(); } setState(1003); + match(ClickHouseParser::DATABASE); + setState(1006); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 112, _ctx)) { + case 1: { + setState(1004); + match(ClickHouseParser::IF); + setState(1005); + match(ClickHouseParser::EXISTS); + break; + } + + } + setState(1008); + databaseIdentifier(); + setState(1010); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::ON) { + setState(1009); + clusterClause(); + } + break; + } + + case 2: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 2); + setState(1012); + _la = _input->LA(1); + if (!(_la == ClickHouseParser::DETACH + + || _la == ClickHouseParser::DROP)) { + _errHandler->recoverInline(this); + } + else { + _errHandler->reportMatch(this); + consume(); + } + setState(1019); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::DICTIONARY: { - setState(997); + setState(1013); match(ClickHouseParser::DICTIONARY); break; } case ClickHouseParser::TABLE: case ClickHouseParser::TEMPORARY: { - setState(999); + setState(1015); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TEMPORARY) { - setState(998); + setState(1014); match(ClickHouseParser::TEMPORARY); } - setState(1001); + setState(1017); match(ClickHouseParser::TABLE); break; } case ClickHouseParser::VIEW: { - setState(1002); + setState(1018); match(ClickHouseParser::VIEW); break; } @@ -7492,37 +7555,37 @@ ClickHouseParser::DropStmtContext* ClickHouseParser::dropStmt() { default: throw NoViableAltException(this); } - setState(1007); + setState(1023); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 112, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 116, _ctx)) { case 1: { - setState(1005); + setState(1021); match(ClickHouseParser::IF); - setState(1006); + setState(1022); match(ClickHouseParser::EXISTS); break; } } - setState(1009); + setState(1025); tableIdentifier(); - setState(1011); + setState(1027); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(1010); + setState(1026); clusterClause(); } - setState(1015); + setState(1031); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::NO) { - setState(1013); + setState(1029); match(ClickHouseParser::NO); - setState(1014); + setState(1030); match(ClickHouseParser::DELAY); } break; @@ -7620,17 +7683,17 @@ ClickHouseParser::ExistsStmtContext* ClickHouseParser::existsStmt() { exitRule(); }); try { - setState(1032); + setState(1048); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 118, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 122, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1019); + setState(1035); match(ClickHouseParser::EXISTS); - setState(1020); + setState(1036); match(ClickHouseParser::DATABASE); - setState(1021); + setState(1037); databaseIdentifier(); break; } @@ -7638,40 +7701,40 @@ ClickHouseParser::ExistsStmtContext* ClickHouseParser::existsStmt() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(1022); + setState(1038); match(ClickHouseParser::EXISTS); - setState(1029); + setState(1045); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 117, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 121, _ctx)) { case 1: { - setState(1023); + setState(1039); match(ClickHouseParser::DICTIONARY); break; } case 2: { - setState(1025); + setState(1041); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TEMPORARY) { - setState(1024); + setState(1040); match(ClickHouseParser::TEMPORARY); } - setState(1027); + setState(1043); match(ClickHouseParser::TABLE); break; } case 3: { - setState(1028); + setState(1044); match(ClickHouseParser::VIEW); break; } } - setState(1031); + setState(1047); tableIdentifier(); break; } @@ -7755,17 +7818,17 @@ ClickHouseParser::ExplainStmtContext* ClickHouseParser::explainStmt() { exitRule(); }); try { - setState(1040); + setState(1056); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 119, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 123, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1034); + setState(1050); match(ClickHouseParser::EXPLAIN); - setState(1035); + setState(1051); match(ClickHouseParser::AST); - setState(1036); + setState(1052); query(); break; } @@ -7773,11 +7836,11 @@ ClickHouseParser::ExplainStmtContext* ClickHouseParser::explainStmt() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(1037); + setState(1053); match(ClickHouseParser::EXPLAIN); - setState(1038); + setState(1054); match(ClickHouseParser::SYNTAX); - setState(1039); + setState(1055); query(); break; } @@ -7853,51 +7916,51 @@ ClickHouseParser::InsertStmtContext* ClickHouseParser::insertStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1042); + setState(1058); match(ClickHouseParser::INSERT); - setState(1043); + setState(1059); match(ClickHouseParser::INTO); - setState(1045); + setState(1061); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 120, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 124, _ctx)) { case 1: { - setState(1044); + setState(1060); match(ClickHouseParser::TABLE); break; } } - setState(1050); + setState(1066); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 121, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 125, _ctx)) { case 1: { - setState(1047); + setState(1063); tableIdentifier(); break; } case 2: { - setState(1048); + setState(1064); match(ClickHouseParser::FUNCTION); - setState(1049); + setState(1065); tableFunctionExpr(); break; } } - setState(1053); + setState(1069); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 122, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 126, _ctx)) { case 1: { - setState(1052); + setState(1068); columnsClause(); break; } } - setState(1055); + setState(1071); dataClause(); } @@ -7962,23 +8025,23 @@ ClickHouseParser::ColumnsClauseContext* ClickHouseParser::columnsClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1057); + setState(1073); match(ClickHouseParser::LPAREN); - setState(1058); + setState(1074); nestedIdentifier(); - setState(1063); + setState(1079); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1059); + setState(1075); match(ClickHouseParser::COMMA); - setState(1060); + setState(1076); nestedIdentifier(); - setState(1065); + setState(1081); _errHandler->sync(this); _la = _input->LA(1); } - setState(1066); + setState(1082); match(ClickHouseParser::RPAREN); } @@ -8069,15 +8132,15 @@ ClickHouseParser::DataClauseContext* ClickHouseParser::dataClause() { exitRule(); }); try { - setState(1077); + setState(1093); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::FORMAT: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1068); + setState(1084); match(ClickHouseParser::FORMAT); - setState(1069); + setState(1085); identifier(); break; } @@ -8085,7 +8148,7 @@ ClickHouseParser::DataClauseContext* ClickHouseParser::dataClause() { case ClickHouseParser::VALUES: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(1070); + setState(1086); match(ClickHouseParser::VALUES); break; } @@ -8095,17 +8158,17 @@ ClickHouseParser::DataClauseContext* ClickHouseParser::dataClause() { case ClickHouseParser::LPAREN: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(1071); + setState(1087); selectUnionStmt(); - setState(1073); + setState(1089); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::SEMICOLON) { - setState(1072); + setState(1088); match(ClickHouseParser::SEMICOLON); } - setState(1075); + setState(1091); match(ClickHouseParser::EOF); break; } @@ -8188,28 +8251,28 @@ ClickHouseParser::KillStmtContext* ClickHouseParser::killStmt() { try { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1079); + setState(1095); match(ClickHouseParser::KILL); - setState(1080); + setState(1096); match(ClickHouseParser::MUTATION); - setState(1082); + setState(1098); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(1081); + setState(1097); clusterClause(); } - setState(1084); + setState(1100); whereClause(); - setState(1086); + setState(1102); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ASYNC || _la == ClickHouseParser::SYNC || _la == ClickHouseParser::TEST) { - setState(1085); + setState(1101); _la = _input->LA(1); if (!(_la == ClickHouseParser::ASYNC || _la == ClickHouseParser::SYNC @@ -8288,42 +8351,42 @@ ClickHouseParser::OptimizeStmtContext* ClickHouseParser::optimizeStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1088); + setState(1104); match(ClickHouseParser::OPTIMIZE); - setState(1089); + setState(1105); match(ClickHouseParser::TABLE); - setState(1090); + setState(1106); tableIdentifier(); - setState(1092); + setState(1108); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(1091); + setState(1107); clusterClause(); } - setState(1095); + setState(1111); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::PARTITION) { - setState(1094); + setState(1110); partitionClause(); } - setState(1098); + setState(1114); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::FINAL) { - setState(1097); + setState(1113); match(ClickHouseParser::FINAL); } - setState(1101); + setState(1117); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::DEDUPLICATE) { - setState(1100); + setState(1116); match(ClickHouseParser::DEDUPLICATE); } @@ -8401,38 +8464,38 @@ ClickHouseParser::RenameStmtContext* ClickHouseParser::renameStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1103); + setState(1119); match(ClickHouseParser::RENAME); - setState(1104); + setState(1120); match(ClickHouseParser::TABLE); - setState(1105); + setState(1121); tableIdentifier(); - setState(1106); + setState(1122); match(ClickHouseParser::TO); - setState(1107); + setState(1123); tableIdentifier(); - setState(1115); + setState(1131); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1108); + setState(1124); match(ClickHouseParser::COMMA); - setState(1109); + setState(1125); tableIdentifier(); - setState(1110); + setState(1126); match(ClickHouseParser::TO); - setState(1111); + setState(1127); tableIdentifier(); - setState(1117); + setState(1133); _errHandler->sync(this); _la = _input->LA(1); } - setState(1119); + setState(1135); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(1118); + setState(1134); clusterClause(); } @@ -8502,37 +8565,37 @@ ClickHouseParser::ProjectionSelectStmtContext* ClickHouseParser::projectionSelec }); try { enterOuterAlt(_localctx, 1); - setState(1121); + setState(1137); match(ClickHouseParser::LPAREN); - setState(1123); + setState(1139); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::WITH) { - setState(1122); + setState(1138); withClause(); } - setState(1125); + setState(1141); match(ClickHouseParser::SELECT); - setState(1126); + setState(1142); columnExprList(); - setState(1128); + setState(1144); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::GROUP) { - setState(1127); + setState(1143); groupByClause(); } - setState(1131); + setState(1147); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ORDER) { - setState(1130); + setState(1146); projectionOrderByClause(); } - setState(1133); + setState(1149); match(ClickHouseParser::RPAREN); } @@ -8597,19 +8660,19 @@ ClickHouseParser::SelectUnionStmtContext* ClickHouseParser::selectUnionStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1135); + setState(1151); selectStmtWithParens(); - setState(1141); + setState(1157); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::UNION) { - setState(1136); + setState(1152); match(ClickHouseParser::UNION); - setState(1137); + setState(1153); match(ClickHouseParser::ALL); - setState(1138); + setState(1154); selectStmtWithParens(); - setState(1143); + setState(1159); _errHandler->sync(this); _la = _input->LA(1); } @@ -8666,24 +8729,24 @@ ClickHouseParser::SelectStmtWithParensContext* ClickHouseParser::selectStmtWithP exitRule(); }); try { - setState(1149); + setState(1165); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::SELECT: case ClickHouseParser::WITH: { enterOuterAlt(_localctx, 1); - setState(1144); + setState(1160); selectStmt(); break; } case ClickHouseParser::LPAREN: { enterOuterAlt(_localctx, 2); - setState(1145); + setState(1161); match(ClickHouseParser::LPAREN); - setState(1146); + setState(1162); selectUnionStmt(); - setState(1147); + setState(1163); match(ClickHouseParser::RPAREN); break; } @@ -8810,90 +8873,90 @@ ClickHouseParser::SelectStmtContext* ClickHouseParser::selectStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1152); + setState(1168); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::WITH) { - setState(1151); + setState(1167); withClause(); } - setState(1154); + setState(1170); match(ClickHouseParser::SELECT); - setState(1156); + setState(1172); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 140, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 144, _ctx)) { case 1: { - setState(1155); + setState(1171); match(ClickHouseParser::DISTINCT); break; } } - setState(1159); + setState(1175); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 141, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 145, _ctx)) { case 1: { - setState(1158); + setState(1174); topClause(); break; } } - setState(1161); + setState(1177); columnExprList(); - setState(1163); + setState(1179); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::FROM) { - setState(1162); + setState(1178); fromClause(); } - setState(1166); + setState(1182); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ARRAY || _la == ClickHouseParser::INNER || _la == ClickHouseParser::LEFT) { - setState(1165); + setState(1181); arrayJoinClause(); } - setState(1169); + setState(1185); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::PREWHERE) { - setState(1168); + setState(1184); prewhereClause(); } - setState(1172); + setState(1188); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::WHERE) { - setState(1171); + setState(1187); whereClause(); } - setState(1175); + setState(1191); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::GROUP) { - setState(1174); + setState(1190); groupByClause(); } - setState(1179); + setState(1195); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 147, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 151, _ctx)) { case 1: { - setState(1177); + setState(1193); match(ClickHouseParser::WITH); - setState(1178); + setState(1194); _la = _input->LA(1); if (!(_la == ClickHouseParser::CUBE || _la == ClickHouseParser::ROLLUP)) { _errHandler->recoverInline(this); @@ -8906,57 +8969,57 @@ ClickHouseParser::SelectStmtContext* ClickHouseParser::selectStmt() { } } - setState(1183); + setState(1199); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::WITH) { - setState(1181); + setState(1197); match(ClickHouseParser::WITH); - setState(1182); + setState(1198); match(ClickHouseParser::TOTALS); } - setState(1186); + setState(1202); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::HAVING) { - setState(1185); + setState(1201); havingClause(); } - setState(1189); + setState(1205); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ORDER) { - setState(1188); + setState(1204); orderByClause(); } - setState(1192); + setState(1208); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 151, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 155, _ctx)) { case 1: { - setState(1191); + setState(1207); limitByClause(); break; } } - setState(1195); + setState(1211); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::LIMIT) { - setState(1194); + setState(1210); limitClause(); } - setState(1198); + setState(1214); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::SETTINGS) { - setState(1197); + setState(1213); settingsClause(); } @@ -9005,9 +9068,9 @@ ClickHouseParser::WithClauseContext* ClickHouseParser::withClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1200); + setState(1216); match(ClickHouseParser::WITH); - setState(1201); + setState(1217); columnExprList(); } @@ -9063,18 +9126,18 @@ ClickHouseParser::TopClauseContext* ClickHouseParser::topClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1203); + setState(1219); match(ClickHouseParser::TOP); - setState(1204); + setState(1220); match(ClickHouseParser::DECIMAL_LITERAL); - setState(1207); + setState(1223); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 154, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 158, _ctx)) { case 1: { - setState(1205); + setState(1221); match(ClickHouseParser::WITH); - setState(1206); + setState(1222); match(ClickHouseParser::TIES); break; } @@ -9126,9 +9189,9 @@ ClickHouseParser::FromClauseContext* ClickHouseParser::fromClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1209); + setState(1225); match(ClickHouseParser::FROM); - setState(1210); + setState(1226); joinExpr(0); } @@ -9189,14 +9252,14 @@ ClickHouseParser::ArrayJoinClauseContext* ClickHouseParser::arrayJoinClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1213); + setState(1229); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::INNER || _la == ClickHouseParser::LEFT) { - setState(1212); + setState(1228); _la = _input->LA(1); if (!(_la == ClickHouseParser::INNER @@ -9208,11 +9271,11 @@ ClickHouseParser::ArrayJoinClauseContext* ClickHouseParser::arrayJoinClause() { consume(); } } - setState(1215); + setState(1231); match(ClickHouseParser::ARRAY); - setState(1216); + setState(1232); match(ClickHouseParser::JOIN); - setState(1217); + setState(1233); columnExprList(); } @@ -9260,9 +9323,9 @@ ClickHouseParser::PrewhereClauseContext* ClickHouseParser::prewhereClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1219); + setState(1235); match(ClickHouseParser::PREWHERE); - setState(1220); + setState(1236); columnExpr(0); } @@ -9310,9 +9373,9 @@ ClickHouseParser::WhereClauseContext* ClickHouseParser::whereClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1222); + setState(1238); match(ClickHouseParser::WHERE); - setState(1223); + setState(1239); columnExpr(0); } @@ -9381,15 +9444,15 @@ ClickHouseParser::GroupByClauseContext* ClickHouseParser::groupByClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1225); + setState(1241); match(ClickHouseParser::GROUP); - setState(1226); + setState(1242); match(ClickHouseParser::BY); - setState(1233); + setState(1249); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 156, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 160, _ctx)) { case 1: { - setState(1227); + setState(1243); _la = _input->LA(1); if (!(_la == ClickHouseParser::CUBE || _la == ClickHouseParser::ROLLUP)) { _errHandler->recoverInline(this); @@ -9398,17 +9461,17 @@ ClickHouseParser::GroupByClauseContext* ClickHouseParser::groupByClause() { _errHandler->reportMatch(this); consume(); } - setState(1228); + setState(1244); match(ClickHouseParser::LPAREN); - setState(1229); + setState(1245); columnExprList(); - setState(1230); + setState(1246); match(ClickHouseParser::RPAREN); break; } case 2: { - setState(1232); + setState(1248); columnExprList(); break; } @@ -9460,9 +9523,9 @@ ClickHouseParser::HavingClauseContext* ClickHouseParser::havingClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1235); + setState(1251); match(ClickHouseParser::HAVING); - setState(1236); + setState(1252); columnExpr(0); } @@ -9514,11 +9577,11 @@ ClickHouseParser::OrderByClauseContext* ClickHouseParser::orderByClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1238); + setState(1254); match(ClickHouseParser::ORDER); - setState(1239); + setState(1255); match(ClickHouseParser::BY); - setState(1240); + setState(1256); orderExprList(); } @@ -9570,11 +9633,11 @@ ClickHouseParser::ProjectionOrderByClauseContext* ClickHouseParser::projectionOr }); try { enterOuterAlt(_localctx, 1); - setState(1242); + setState(1258); match(ClickHouseParser::ORDER); - setState(1243); + setState(1259); match(ClickHouseParser::BY); - setState(1244); + setState(1260); columnExprList(); } @@ -9630,13 +9693,13 @@ ClickHouseParser::LimitByClauseContext* ClickHouseParser::limitByClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1246); + setState(1262); match(ClickHouseParser::LIMIT); - setState(1247); + setState(1263); limitExpr(); - setState(1248); + setState(1264); match(ClickHouseParser::BY); - setState(1249); + setState(1265); columnExprList(); } @@ -9693,18 +9756,18 @@ ClickHouseParser::LimitClauseContext* ClickHouseParser::limitClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1251); + setState(1267); match(ClickHouseParser::LIMIT); - setState(1252); + setState(1268); limitExpr(); - setState(1255); + setState(1271); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::WITH) { - setState(1253); + setState(1269); match(ClickHouseParser::WITH); - setState(1254); + setState(1270); match(ClickHouseParser::TIES); } @@ -9753,9 +9816,9 @@ ClickHouseParser::SettingsClauseContext* ClickHouseParser::settingsClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1257); + setState(1273); match(ClickHouseParser::SETTINGS); - setState(1258); + setState(1274); settingExprList(); } @@ -9909,33 +9972,33 @@ ClickHouseParser::JoinExprContext* ClickHouseParser::joinExpr(int precedence) { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1272); + setState(1288); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 160, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 164, _ctx)) { case 1: { _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1261); + setState(1277); tableExpr(0); - setState(1263); + setState(1279); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 158, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 162, _ctx)) { case 1: { - setState(1262); + setState(1278); match(ClickHouseParser::FINAL); break; } } - setState(1266); + setState(1282); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 159, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 163, _ctx)) { case 1: { - setState(1265); + setState(1281); sampleClause(); break; } @@ -9948,38 +10011,38 @@ ClickHouseParser::JoinExprContext* ClickHouseParser::joinExpr(int precedence) { _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1268); + setState(1284); match(ClickHouseParser::LPAREN); - setState(1269); + setState(1285); joinExpr(0); - setState(1270); + setState(1286); match(ClickHouseParser::RPAREN); break; } } _ctx->stop = _input->LT(-1); - setState(1291); + setState(1307); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 164, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 168, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { if (!_parseListeners.empty()) triggerExitRuleEvent(); previousContext = _localctx; - setState(1289); + setState(1305); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 163, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 167, _ctx)) { case 1: { auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleJoinExpr); - setState(1274); + setState(1290); if (!(precpred(_ctx, 3))) throw FailedPredicateException(this, "precpred(_ctx, 3)"); - setState(1275); + setState(1291); joinOpCross(); - setState(1276); + setState(1292); joinExpr(4); break; } @@ -9988,17 +10051,17 @@ ClickHouseParser::JoinExprContext* ClickHouseParser::joinExpr(int precedence) { auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleJoinExpr); - setState(1278); + setState(1294); if (!(precpred(_ctx, 4))) throw FailedPredicateException(this, "precpred(_ctx, 4)"); - setState(1280); + setState(1296); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::GLOBAL || _la == ClickHouseParser::LOCAL) { - setState(1279); + setState(1295); _la = _input->LA(1); if (!(_la == ClickHouseParser::GLOBAL @@ -10010,7 +10073,7 @@ ClickHouseParser::JoinExprContext* ClickHouseParser::joinExpr(int precedence) { consume(); } } - setState(1283); + setState(1299); _errHandler->sync(this); _la = _input->LA(1); @@ -10024,23 +10087,23 @@ ClickHouseParser::JoinExprContext* ClickHouseParser::joinExpr(int precedence) { | (1ULL << (ClickHouseParser::LEFT - 81)) | (1ULL << (ClickHouseParser::RIGHT - 81)) | (1ULL << (ClickHouseParser::SEMI - 81)))) != 0)) { - setState(1282); + setState(1298); joinOp(); } - setState(1285); + setState(1301); match(ClickHouseParser::JOIN); - setState(1286); + setState(1302); joinExpr(0); - setState(1287); + setState(1303); joinConstraintClause(); break; } } } - setState(1293); + setState(1309); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 164, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 168, _ctx); } } catch (RecognitionException &e) { @@ -10169,17 +10232,17 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { exitRule(); }); try { - setState(1337); + setState(1353); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 178, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 182, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1303); + setState(1319); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 167, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 171, _ctx)) { case 1: { - setState(1295); + setState(1311); _errHandler->sync(this); _la = _input->LA(1); @@ -10187,7 +10250,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) | (1ULL << ClickHouseParser::ANY) | (1ULL << ClickHouseParser::ASOF))) != 0)) { - setState(1294); + setState(1310); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) @@ -10200,15 +10263,15 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { consume(); } } - setState(1297); + setState(1313); match(ClickHouseParser::INNER); break; } case 2: { - setState(1298); + setState(1314); match(ClickHouseParser::INNER); - setState(1300); + setState(1316); _errHandler->sync(this); _la = _input->LA(1); @@ -10216,7 +10279,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) | (1ULL << ClickHouseParser::ANY) | (1ULL << ClickHouseParser::ASOF))) != 0)) { - setState(1299); + setState(1315); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) @@ -10233,7 +10296,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { } case 3: { - setState(1302); + setState(1318); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) @@ -10255,11 +10318,11 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(1319); + setState(1335); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 172, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 176, _ctx)) { case 1: { - setState(1306); + setState(1322); _errHandler->sync(this); _la = _input->LA(1); @@ -10268,7 +10331,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { | (1ULL << ClickHouseParser::ANTI) | (1ULL << ClickHouseParser::ANY) | (1ULL << ClickHouseParser::ASOF))) != 0) || _la == ClickHouseParser::SEMI) { - setState(1305); + setState(1321); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) @@ -10282,7 +10345,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { consume(); } } - setState(1308); + setState(1324); _la = _input->LA(1); if (!(_la == ClickHouseParser::LEFT @@ -10293,19 +10356,19 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { _errHandler->reportMatch(this); consume(); } - setState(1310); + setState(1326); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OUTER) { - setState(1309); + setState(1325); match(ClickHouseParser::OUTER); } break; } case 2: { - setState(1312); + setState(1328); _la = _input->LA(1); if (!(_la == ClickHouseParser::LEFT @@ -10316,15 +10379,15 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { _errHandler->reportMatch(this); consume(); } - setState(1314); + setState(1330); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OUTER) { - setState(1313); + setState(1329); match(ClickHouseParser::OUTER); } - setState(1317); + setState(1333); _errHandler->sync(this); _la = _input->LA(1); @@ -10333,7 +10396,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { | (1ULL << ClickHouseParser::ANTI) | (1ULL << ClickHouseParser::ANY) | (1ULL << ClickHouseParser::ASOF))) != 0) || _la == ClickHouseParser::SEMI) { - setState(1316); + setState(1332); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) @@ -10357,18 +10420,18 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(1335); + setState(1351); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 177, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 181, _ctx)) { case 1: { - setState(1322); + setState(1338); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ALL || _la == ClickHouseParser::ANY) { - setState(1321); + setState(1337); _la = _input->LA(1); if (!(_la == ClickHouseParser::ALL @@ -10380,38 +10443,38 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { consume(); } } - setState(1324); + setState(1340); match(ClickHouseParser::FULL); - setState(1326); + setState(1342); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OUTER) { - setState(1325); + setState(1341); match(ClickHouseParser::OUTER); } break; } case 2: { - setState(1328); + setState(1344); match(ClickHouseParser::FULL); - setState(1330); + setState(1346); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OUTER) { - setState(1329); + setState(1345); match(ClickHouseParser::OUTER); } - setState(1333); + setState(1349); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ALL || _la == ClickHouseParser::ANY) { - setState(1332); + setState(1348); _la = _input->LA(1); if (!(_la == ClickHouseParser::ALL @@ -10489,21 +10552,21 @@ ClickHouseParser::JoinOpCrossContext* ClickHouseParser::joinOpCross() { exitRule(); }); try { - setState(1345); + setState(1361); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::CROSS: case ClickHouseParser::GLOBAL: case ClickHouseParser::LOCAL: { enterOuterAlt(_localctx, 1); - setState(1340); + setState(1356); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::GLOBAL || _la == ClickHouseParser::LOCAL) { - setState(1339); + setState(1355); _la = _input->LA(1); if (!(_la == ClickHouseParser::GLOBAL @@ -10515,16 +10578,16 @@ ClickHouseParser::JoinOpCrossContext* ClickHouseParser::joinOpCross() { consume(); } } - setState(1342); + setState(1358); match(ClickHouseParser::CROSS); - setState(1343); + setState(1359); match(ClickHouseParser::JOIN); break; } case ClickHouseParser::COMMA: { enterOuterAlt(_localctx, 2); - setState(1344); + setState(1360); match(ClickHouseParser::COMMA); break; } @@ -10589,36 +10652,36 @@ ClickHouseParser::JoinConstraintClauseContext* ClickHouseParser::joinConstraintC exitRule(); }); try { - setState(1356); + setState(1372); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 181, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 185, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(1347); + setState(1363); match(ClickHouseParser::ON); - setState(1348); + setState(1364); columnExprList(); break; } case 2: { enterOuterAlt(_localctx, 2); - setState(1349); + setState(1365); match(ClickHouseParser::USING); - setState(1350); + setState(1366); match(ClickHouseParser::LPAREN); - setState(1351); + setState(1367); columnExprList(); - setState(1352); + setState(1368); match(ClickHouseParser::RPAREN); break; } case 3: { enterOuterAlt(_localctx, 3); - setState(1354); + setState(1370); match(ClickHouseParser::USING); - setState(1355); + setState(1371); columnExprList(); break; } @@ -10678,18 +10741,18 @@ ClickHouseParser::SampleClauseContext* ClickHouseParser::sampleClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1358); + setState(1374); match(ClickHouseParser::SAMPLE); - setState(1359); + setState(1375); ratioExpr(); - setState(1362); + setState(1378); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 182, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 186, _ctx)) { case 1: { - setState(1360); + setState(1376); match(ClickHouseParser::OFFSET); - setState(1361); + setState(1377); ratioExpr(); break; } @@ -10750,14 +10813,14 @@ ClickHouseParser::LimitExprContext* ClickHouseParser::limitExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(1364); + setState(1380); columnExpr(0); - setState(1367); + setState(1383); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OFFSET || _la == ClickHouseParser::COMMA) { - setState(1365); + setState(1381); _la = _input->LA(1); if (!(_la == ClickHouseParser::OFFSET || _la == ClickHouseParser::COMMA)) { _errHandler->recoverInline(this); @@ -10766,7 +10829,7 @@ ClickHouseParser::LimitExprContext* ClickHouseParser::limitExpr() { _errHandler->reportMatch(this); consume(); } - setState(1366); + setState(1382); columnExpr(0); } @@ -10824,21 +10887,21 @@ ClickHouseParser::OrderExprListContext* ClickHouseParser::orderExprList() { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1369); + setState(1385); orderExpr(); - setState(1374); + setState(1390); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 184, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 188, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(1370); + setState(1386); match(ClickHouseParser::COMMA); - setState(1371); + setState(1387); orderExpr(); } - setState(1376); + setState(1392); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 184, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 188, _ctx); } } @@ -10915,14 +10978,14 @@ ClickHouseParser::OrderExprContext* ClickHouseParser::orderExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(1377); + setState(1393); columnExpr(0); - setState(1379); + setState(1395); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 185, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 189, _ctx)) { case 1: { - setState(1378); + setState(1394); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ASCENDING) @@ -10938,14 +11001,14 @@ ClickHouseParser::OrderExprContext* ClickHouseParser::orderExpr() { } } - setState(1383); + setState(1399); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 186, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 190, _ctx)) { case 1: { - setState(1381); + setState(1397); match(ClickHouseParser::NULLS); - setState(1382); + setState(1398); _la = _input->LA(1); if (!(_la == ClickHouseParser::FIRST @@ -10960,14 +11023,14 @@ ClickHouseParser::OrderExprContext* ClickHouseParser::orderExpr() { } } - setState(1387); + setState(1403); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 187, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 191, _ctx)) { case 1: { - setState(1385); + setState(1401); match(ClickHouseParser::COLLATE); - setState(1386); + setState(1402); match(ClickHouseParser::STRING_LITERAL); break; } @@ -11023,16 +11086,16 @@ ClickHouseParser::RatioExprContext* ClickHouseParser::ratioExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(1389); + setState(1405); numberLiteral(); - setState(1392); + setState(1408); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 188, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 192, _ctx)) { case 1: { - setState(1390); + setState(1406); match(ClickHouseParser::SLASH); - setState(1391); + setState(1407); numberLiteral(); break; } @@ -11093,21 +11156,21 @@ ClickHouseParser::SettingExprListContext* ClickHouseParser::settingExprList() { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1394); + setState(1410); settingExpr(); - setState(1399); + setState(1415); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 189, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 193, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(1395); + setState(1411); match(ClickHouseParser::COMMA); - setState(1396); + setState(1412); settingExpr(); } - setState(1401); + setState(1417); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 189, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 193, _ctx); } } @@ -11159,11 +11222,11 @@ ClickHouseParser::SettingExprContext* ClickHouseParser::settingExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(1402); + setState(1418); identifier(); - setState(1403); + setState(1419); match(ClickHouseParser::EQ_SINGLE); - setState(1404); + setState(1420); literal(); } @@ -11211,9 +11274,9 @@ ClickHouseParser::SetStmtContext* ClickHouseParser::setStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1406); + setState(1422); match(ClickHouseParser::SET); - setState(1407); + setState(1423); settingExprList(); } @@ -11426,19 +11489,19 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { exitRule(); }); try { - setState(1451); + setState(1467); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 197, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 201, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1409); + setState(1425); match(ClickHouseParser::SHOW); - setState(1410); + setState(1426); match(ClickHouseParser::CREATE); - setState(1411); + setState(1427); match(ClickHouseParser::DATABASE); - setState(1412); + setState(1428); databaseIdentifier(); break; } @@ -11446,13 +11509,13 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(1413); + setState(1429); match(ClickHouseParser::SHOW); - setState(1414); + setState(1430); match(ClickHouseParser::CREATE); - setState(1415); + setState(1431); match(ClickHouseParser::DICTIONARY); - setState(1416); + setState(1432); tableIdentifier(); break; } @@ -11460,33 +11523,33 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(1417); + setState(1433); match(ClickHouseParser::SHOW); - setState(1418); + setState(1434); match(ClickHouseParser::CREATE); - setState(1420); + setState(1436); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 190, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 194, _ctx)) { case 1: { - setState(1419); + setState(1435); match(ClickHouseParser::TEMPORARY); break; } } - setState(1423); + setState(1439); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 191, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 195, _ctx)) { case 1: { - setState(1422); + setState(1438); match(ClickHouseParser::TABLE); break; } } - setState(1425); + setState(1441); tableIdentifier(); break; } @@ -11494,9 +11557,9 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { case 4: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 4); - setState(1426); + setState(1442); match(ClickHouseParser::SHOW); - setState(1427); + setState(1443); match(ClickHouseParser::DATABASES); break; } @@ -11504,18 +11567,18 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { case 5: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 5); - setState(1428); + setState(1444); match(ClickHouseParser::SHOW); - setState(1429); + setState(1445); match(ClickHouseParser::DICTIONARIES); - setState(1432); + setState(1448); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::FROM) { - setState(1430); + setState(1446); match(ClickHouseParser::FROM); - setState(1431); + setState(1447); databaseIdentifier(); } break; @@ -11524,26 +11587,26 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { case 6: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 6); - setState(1434); + setState(1450); match(ClickHouseParser::SHOW); - setState(1436); + setState(1452); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TEMPORARY) { - setState(1435); + setState(1451); match(ClickHouseParser::TEMPORARY); } - setState(1438); + setState(1454); match(ClickHouseParser::TABLES); - setState(1441); + setState(1457); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::FROM || _la == ClickHouseParser::IN) { - setState(1439); + setState(1455); _la = _input->LA(1); if (!(_la == ClickHouseParser::FROM @@ -11554,22 +11617,22 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { _errHandler->reportMatch(this); consume(); } - setState(1440); + setState(1456); databaseIdentifier(); } - setState(1446); + setState(1462); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::LIKE: { - setState(1443); + setState(1459); match(ClickHouseParser::LIKE); - setState(1444); + setState(1460); match(ClickHouseParser::STRING_LITERAL); break; } case ClickHouseParser::WHERE: { - setState(1445); + setState(1461); whereClause(); break; } @@ -11585,12 +11648,12 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { default: break; } - setState(1449); + setState(1465); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::LIMIT) { - setState(1448); + setState(1464); limitClause(); } break; @@ -11703,62 +11766,62 @@ ClickHouseParser::SystemStmtContext* ClickHouseParser::systemStmt() { exitRule(); }); try { - setState(1487); + setState(1503); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 200, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 204, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(1453); + setState(1469); match(ClickHouseParser::SYSTEM); - setState(1454); + setState(1470); match(ClickHouseParser::FLUSH); - setState(1455); + setState(1471); match(ClickHouseParser::DISTRIBUTED); - setState(1456); + setState(1472); tableIdentifier(); break; } case 2: { enterOuterAlt(_localctx, 2); - setState(1457); + setState(1473); match(ClickHouseParser::SYSTEM); - setState(1458); + setState(1474); match(ClickHouseParser::FLUSH); - setState(1459); + setState(1475); match(ClickHouseParser::LOGS); break; } case 3: { enterOuterAlt(_localctx, 3); - setState(1460); + setState(1476); match(ClickHouseParser::SYSTEM); - setState(1461); + setState(1477); match(ClickHouseParser::RELOAD); - setState(1462); + setState(1478); match(ClickHouseParser::DICTIONARIES); break; } case 4: { enterOuterAlt(_localctx, 4); - setState(1463); + setState(1479); match(ClickHouseParser::SYSTEM); - setState(1464); + setState(1480); match(ClickHouseParser::RELOAD); - setState(1465); + setState(1481); match(ClickHouseParser::DICTIONARY); - setState(1466); + setState(1482); tableIdentifier(); break; } case 5: { enterOuterAlt(_localctx, 5); - setState(1467); + setState(1483); match(ClickHouseParser::SYSTEM); - setState(1468); + setState(1484); _la = _input->LA(1); if (!(_la == ClickHouseParser::START @@ -11769,34 +11832,34 @@ ClickHouseParser::SystemStmtContext* ClickHouseParser::systemStmt() { _errHandler->reportMatch(this); consume(); } - setState(1476); + setState(1492); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::DISTRIBUTED: { - setState(1469); + setState(1485); match(ClickHouseParser::DISTRIBUTED); - setState(1470); + setState(1486); match(ClickHouseParser::SENDS); break; } case ClickHouseParser::FETCHES: { - setState(1471); + setState(1487); match(ClickHouseParser::FETCHES); break; } case ClickHouseParser::MERGES: case ClickHouseParser::TTL: { - setState(1473); + setState(1489); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TTL) { - setState(1472); + setState(1488); match(ClickHouseParser::TTL); } - setState(1475); + setState(1491); match(ClickHouseParser::MERGES); break; } @@ -11804,16 +11867,16 @@ ClickHouseParser::SystemStmtContext* ClickHouseParser::systemStmt() { default: throw NoViableAltException(this); } - setState(1478); + setState(1494); tableIdentifier(); break; } case 6: { enterOuterAlt(_localctx, 6); - setState(1479); + setState(1495); match(ClickHouseParser::SYSTEM); - setState(1480); + setState(1496); _la = _input->LA(1); if (!(_la == ClickHouseParser::START @@ -11824,22 +11887,22 @@ ClickHouseParser::SystemStmtContext* ClickHouseParser::systemStmt() { _errHandler->reportMatch(this); consume(); } - setState(1481); + setState(1497); match(ClickHouseParser::REPLICATED); - setState(1482); + setState(1498); match(ClickHouseParser::SENDS); break; } case 7: { enterOuterAlt(_localctx, 7); - setState(1483); + setState(1499); match(ClickHouseParser::SYSTEM); - setState(1484); + setState(1500); match(ClickHouseParser::SYNC); - setState(1485); + setState(1501); match(ClickHouseParser::REPLICA); - setState(1486); + setState(1502); tableIdentifier(); break; } @@ -11912,51 +11975,51 @@ ClickHouseParser::TruncateStmtContext* ClickHouseParser::truncateStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1489); + setState(1505); match(ClickHouseParser::TRUNCATE); - setState(1491); + setState(1507); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 201, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 205, _ctx)) { case 1: { - setState(1490); + setState(1506); match(ClickHouseParser::TEMPORARY); break; } } - setState(1494); + setState(1510); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 202, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 206, _ctx)) { case 1: { - setState(1493); + setState(1509); match(ClickHouseParser::TABLE); break; } } - setState(1498); + setState(1514); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 203, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 207, _ctx)) { case 1: { - setState(1496); + setState(1512); match(ClickHouseParser::IF); - setState(1497); + setState(1513); match(ClickHouseParser::EXISTS); break; } } - setState(1500); + setState(1516); tableIdentifier(); - setState(1502); + setState(1518); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(1501); + setState(1517); clusterClause(); } @@ -12005,9 +12068,9 @@ ClickHouseParser::UseStmtContext* ClickHouseParser::useStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1504); + setState(1520); match(ClickHouseParser::USE); - setState(1505); + setState(1521); databaseIdentifier(); } @@ -12068,26 +12131,26 @@ ClickHouseParser::WatchStmtContext* ClickHouseParser::watchStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1507); + setState(1523); match(ClickHouseParser::WATCH); - setState(1508); + setState(1524); tableIdentifier(); - setState(1510); + setState(1526); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::EVENTS) { - setState(1509); + setState(1525); match(ClickHouseParser::EVENTS); } - setState(1514); + setState(1530); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::LIMIT) { - setState(1512); + setState(1528); match(ClickHouseParser::LIMIT); - setState(1513); + setState(1529); match(ClickHouseParser::DECIMAL_LITERAL); } @@ -12283,13 +12346,13 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { exitRule(); }); try { - setState(1563); + setState(1579); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 211, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 215, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1516); + setState(1532); identifier(); break; } @@ -12297,29 +12360,29 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(1517); + setState(1533); identifier(); - setState(1518); + setState(1534); match(ClickHouseParser::LPAREN); - setState(1519); + setState(1535); identifier(); - setState(1520); + setState(1536); columnTypeExpr(); - setState(1527); + setState(1543); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1521); + setState(1537); match(ClickHouseParser::COMMA); - setState(1522); + setState(1538); identifier(); - setState(1523); + setState(1539); columnTypeExpr(); - setState(1529); + setState(1545); _errHandler->sync(this); _la = _input->LA(1); } - setState(1530); + setState(1546); match(ClickHouseParser::RPAREN); break; } @@ -12327,25 +12390,25 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(1532); + setState(1548); identifier(); - setState(1533); + setState(1549); match(ClickHouseParser::LPAREN); - setState(1534); + setState(1550); enumValue(); - setState(1539); + setState(1555); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1535); + setState(1551); match(ClickHouseParser::COMMA); - setState(1536); + setState(1552); enumValue(); - setState(1541); + setState(1557); _errHandler->sync(this); _la = _input->LA(1); } - setState(1542); + setState(1558); match(ClickHouseParser::RPAREN); break; } @@ -12353,25 +12416,25 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { case 4: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 4); - setState(1544); + setState(1560); identifier(); - setState(1545); + setState(1561); match(ClickHouseParser::LPAREN); - setState(1546); + setState(1562); columnTypeExpr(); - setState(1551); + setState(1567); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1547); + setState(1563); match(ClickHouseParser::COMMA); - setState(1548); + setState(1564); columnTypeExpr(); - setState(1553); + setState(1569); _errHandler->sync(this); _la = _input->LA(1); } - setState(1554); + setState(1570); match(ClickHouseParser::RPAREN); break; } @@ -12379,11 +12442,11 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { case 5: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 5); - setState(1556); + setState(1572); identifier(); - setState(1557); + setState(1573); match(ClickHouseParser::LPAREN); - setState(1559); + setState(1575); _errHandler->sync(this); _la = _input->LA(1); @@ -12581,10 +12644,10 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { | (1ULL << (ClickHouseParser::LBRACKET - 197)) | (1ULL << (ClickHouseParser::LPAREN - 197)) | (1ULL << (ClickHouseParser::PLUS - 197)))) != 0)) { - setState(1558); + setState(1574); columnExprList(); } - setState(1561); + setState(1577); match(ClickHouseParser::RPAREN); break; } @@ -12645,21 +12708,21 @@ ClickHouseParser::ColumnExprListContext* ClickHouseParser::columnExprList() { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1565); + setState(1581); columnsExpr(); - setState(1570); + setState(1586); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 212, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 216, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(1566); + setState(1582); match(ClickHouseParser::COMMA); - setState(1567); + setState(1583); columnsExpr(); } - setState(1572); + setState(1588); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 212, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 216, _ctx); } } @@ -12754,13 +12817,13 @@ ClickHouseParser::ColumnsExprContext* ClickHouseParser::columnsExpr() { exitRule(); }); try { - setState(1584); + setState(1600); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 214, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 218, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1576); + setState(1592); _errHandler->sync(this); _la = _input->LA(1); @@ -12944,12 +13007,12 @@ ClickHouseParser::ColumnsExprContext* ClickHouseParser::columnsExpr() { | (1ULL << (ClickHouseParser::JSON_FALSE - 128)) | (1ULL << (ClickHouseParser::JSON_TRUE - 128)) | (1ULL << (ClickHouseParser::IDENTIFIER - 128)))) != 0)) { - setState(1573); + setState(1589); tableIdentifier(); - setState(1574); + setState(1590); match(ClickHouseParser::DOT); } - setState(1578); + setState(1594); match(ClickHouseParser::ASTERISK); break; } @@ -12957,11 +13020,11 @@ ClickHouseParser::ColumnsExprContext* ClickHouseParser::columnsExpr() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(1579); + setState(1595); match(ClickHouseParser::LPAREN); - setState(1580); + setState(1596); selectUnionStmt(); - setState(1581); + setState(1597); match(ClickHouseParser::RPAREN); break; } @@ -12969,7 +13032,7 @@ ClickHouseParser::ColumnsExprContext* ClickHouseParser::columnsExpr() { case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(1583); + setState(1599); columnExpr(0); break; } @@ -13821,54 +13884,54 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1693); + setState(1709); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 225, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 229, _ctx)) { case 1: { _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1587); + setState(1603); match(ClickHouseParser::CASE); - setState(1589); + setState(1605); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 215, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 219, _ctx)) { case 1: { - setState(1588); + setState(1604); columnExpr(0); break; } } - setState(1596); + setState(1612); _errHandler->sync(this); _la = _input->LA(1); do { - setState(1591); + setState(1607); match(ClickHouseParser::WHEN); - setState(1592); + setState(1608); columnExpr(0); - setState(1593); + setState(1609); match(ClickHouseParser::THEN); - setState(1594); + setState(1610); columnExpr(0); - setState(1598); + setState(1614); _errHandler->sync(this); _la = _input->LA(1); } while (_la == ClickHouseParser::WHEN); - setState(1602); + setState(1618); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ELSE) { - setState(1600); + setState(1616); match(ClickHouseParser::ELSE); - setState(1601); + setState(1617); columnExpr(0); } - setState(1604); + setState(1620); match(ClickHouseParser::END); break; } @@ -13877,17 +13940,17 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1606); + setState(1622); match(ClickHouseParser::CAST); - setState(1607); + setState(1623); match(ClickHouseParser::LPAREN); - setState(1608); + setState(1624); columnExpr(0); - setState(1609); + setState(1625); match(ClickHouseParser::AS); - setState(1610); + setState(1626); columnTypeExpr(); - setState(1611); + setState(1627); match(ClickHouseParser::RPAREN); break; } @@ -13896,9 +13959,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1613); + setState(1629); match(ClickHouseParser::DATE); - setState(1614); + setState(1630); match(ClickHouseParser::STRING_LITERAL); break; } @@ -13907,17 +13970,17 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1615); + setState(1631); match(ClickHouseParser::EXTRACT); - setState(1616); + setState(1632); match(ClickHouseParser::LPAREN); - setState(1617); + setState(1633); interval(); - setState(1618); + setState(1634); match(ClickHouseParser::FROM); - setState(1619); + setState(1635); columnExpr(0); - setState(1620); + setState(1636); match(ClickHouseParser::RPAREN); break; } @@ -13926,11 +13989,11 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1622); + setState(1638); match(ClickHouseParser::INTERVAL); - setState(1623); + setState(1639); columnExpr(0); - setState(1624); + setState(1640); interval(); break; } @@ -13939,27 +14002,27 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1626); + setState(1642); match(ClickHouseParser::SUBSTRING); - setState(1627); + setState(1643); match(ClickHouseParser::LPAREN); - setState(1628); + setState(1644); columnExpr(0); - setState(1629); + setState(1645); match(ClickHouseParser::FROM); - setState(1630); + setState(1646); columnExpr(0); - setState(1633); + setState(1649); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::FOR) { - setState(1631); + setState(1647); match(ClickHouseParser::FOR); - setState(1632); + setState(1648); columnExpr(0); } - setState(1635); + setState(1651); match(ClickHouseParser::RPAREN); break; } @@ -13968,9 +14031,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1637); + setState(1653); match(ClickHouseParser::TIMESTAMP); - setState(1638); + setState(1654); match(ClickHouseParser::STRING_LITERAL); break; } @@ -13979,11 +14042,11 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1639); + setState(1655); match(ClickHouseParser::TRIM); - setState(1640); + setState(1656); match(ClickHouseParser::LPAREN); - setState(1641); + setState(1657); _la = _input->LA(1); if (!(_la == ClickHouseParser::BOTH || _la == ClickHouseParser::LEADING || _la == ClickHouseParser::TRAILING)) { _errHandler->recoverInline(this); @@ -13992,13 +14055,13 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _errHandler->reportMatch(this); consume(); } - setState(1642); + setState(1658); match(ClickHouseParser::STRING_LITERAL); - setState(1643); + setState(1659); match(ClickHouseParser::FROM); - setState(1644); + setState(1660); columnExpr(0); - setState(1645); + setState(1661); match(ClickHouseParser::RPAREN); break; } @@ -14007,16 +14070,16 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1647); + setState(1663); identifier(); - setState(1653); + setState(1669); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 220, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 224, _ctx)) { case 1: { - setState(1648); + setState(1664); match(ClickHouseParser::LPAREN); - setState(1650); + setState(1666); _errHandler->sync(this); _la = _input->LA(1); @@ -14214,29 +14277,29 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::LBRACKET - 197)) | (1ULL << (ClickHouseParser::LPAREN - 197)) | (1ULL << (ClickHouseParser::PLUS - 197)))) != 0)) { - setState(1649); + setState(1665); columnExprList(); } - setState(1652); + setState(1668); match(ClickHouseParser::RPAREN); break; } } - setState(1655); + setState(1671); match(ClickHouseParser::LPAREN); - setState(1657); + setState(1673); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 221, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 225, _ctx)) { case 1: { - setState(1656); + setState(1672); match(ClickHouseParser::DISTINCT); break; } } - setState(1660); + setState(1676); _errHandler->sync(this); _la = _input->LA(1); @@ -14434,10 +14497,10 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::LBRACKET - 197)) | (1ULL << (ClickHouseParser::LPAREN - 197)) | (1ULL << (ClickHouseParser::PLUS - 197)))) != 0)) { - setState(1659); + setState(1675); columnArgList(); } - setState(1662); + setState(1678); match(ClickHouseParser::RPAREN); break; } @@ -14446,7 +14509,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1664); + setState(1680); literal(); break; } @@ -14455,9 +14518,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1665); + setState(1681); match(ClickHouseParser::DASH); - setState(1666); + setState(1682); columnExpr(17); break; } @@ -14466,9 +14529,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1667); + setState(1683); match(ClickHouseParser::NOT); - setState(1668); + setState(1684); columnExpr(12); break; } @@ -14477,7 +14540,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1672); + setState(1688); _errHandler->sync(this); _la = _input->LA(1); @@ -14661,12 +14724,12 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::JSON_FALSE - 128)) | (1ULL << (ClickHouseParser::JSON_TRUE - 128)) | (1ULL << (ClickHouseParser::IDENTIFIER - 128)))) != 0)) { - setState(1669); + setState(1685); tableIdentifier(); - setState(1670); + setState(1686); match(ClickHouseParser::DOT); } - setState(1674); + setState(1690); match(ClickHouseParser::ASTERISK); break; } @@ -14675,11 +14738,11 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1675); + setState(1691); match(ClickHouseParser::LPAREN); - setState(1676); + setState(1692); selectUnionStmt(); - setState(1677); + setState(1693); match(ClickHouseParser::RPAREN); break; } @@ -14688,11 +14751,11 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1679); + setState(1695); match(ClickHouseParser::LPAREN); - setState(1680); + setState(1696); columnExpr(0); - setState(1681); + setState(1697); match(ClickHouseParser::RPAREN); break; } @@ -14701,11 +14764,11 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1683); + setState(1699); match(ClickHouseParser::LPAREN); - setState(1684); + setState(1700); columnExprList(); - setState(1685); + setState(1701); match(ClickHouseParser::RPAREN); break; } @@ -14714,9 +14777,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1687); + setState(1703); match(ClickHouseParser::LBRACKET); - setState(1689); + setState(1705); _errHandler->sync(this); _la = _input->LA(1); @@ -14914,10 +14977,10 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::LBRACKET - 197)) | (1ULL << (ClickHouseParser::LPAREN - 197)) | (1ULL << (ClickHouseParser::PLUS - 197)))) != 0)) { - setState(1688); + setState(1704); columnExprList(); } - setState(1691); + setState(1707); match(ClickHouseParser::RBRACKET); break; } @@ -14926,32 +14989,32 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1692); + setState(1708); columnIdentifier(); break; } } _ctx->stop = _input->LT(-1); - setState(1766); + setState(1782); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 234, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 238, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { if (!_parseListeners.empty()) triggerExitRuleEvent(); previousContext = _localctx; - setState(1764); + setState(1780); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 233, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 237, _ctx)) { case 1: { auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1695); + setState(1711); if (!(precpred(_ctx, 16))) throw FailedPredicateException(this, "precpred(_ctx, 16)"); - setState(1696); + setState(1712); _la = _input->LA(1); if (!(((((_la - 191) & ~ 0x3fULL) == 0) && ((1ULL << (_la - 191)) & ((1ULL << (ClickHouseParser::ASTERISK - 191)) @@ -14963,7 +15026,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _errHandler->reportMatch(this); consume(); } - setState(1697); + setState(1713); columnExpr(17); break; } @@ -14972,10 +15035,10 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1698); + setState(1714); if (!(precpred(_ctx, 15))) throw FailedPredicateException(this, "precpred(_ctx, 15)"); - setState(1699); + setState(1715); _la = _input->LA(1); if (!(((((_la - 196) & ~ 0x3fULL) == 0) && ((1ULL << (_la - 196)) & ((1ULL << (ClickHouseParser::CONCAT - 196)) @@ -14987,7 +15050,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _errHandler->reportMatch(this); consume(); } - setState(1700); + setState(1716); columnExpr(16); break; } @@ -14996,86 +15059,86 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1701); + setState(1717); if (!(precpred(_ctx, 14))) throw FailedPredicateException(this, "precpred(_ctx, 14)"); - setState(1720); + setState(1736); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 229, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 233, _ctx)) { case 1: { - setState(1702); + setState(1718); match(ClickHouseParser::EQ_DOUBLE); break; } case 2: { - setState(1703); + setState(1719); match(ClickHouseParser::EQ_SINGLE); break; } case 3: { - setState(1704); + setState(1720); match(ClickHouseParser::NOT_EQ); break; } case 4: { - setState(1705); + setState(1721); match(ClickHouseParser::LE); break; } case 5: { - setState(1706); + setState(1722); match(ClickHouseParser::GE); break; } case 6: { - setState(1707); + setState(1723); match(ClickHouseParser::LT); break; } case 7: { - setState(1708); + setState(1724); match(ClickHouseParser::GT); break; } case 8: { - setState(1710); + setState(1726); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::GLOBAL) { - setState(1709); + setState(1725); match(ClickHouseParser::GLOBAL); } - setState(1713); + setState(1729); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::NOT) { - setState(1712); + setState(1728); match(ClickHouseParser::NOT); } - setState(1715); + setState(1731); match(ClickHouseParser::IN); break; } case 9: { - setState(1717); + setState(1733); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::NOT) { - setState(1716); + setState(1732); match(ClickHouseParser::NOT); } - setState(1719); + setState(1735); _la = _input->LA(1); if (!(_la == ClickHouseParser::ILIKE @@ -15090,7 +15153,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence } } - setState(1722); + setState(1738); columnExpr(15); break; } @@ -15099,12 +15162,12 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1723); + setState(1739); if (!(precpred(_ctx, 11))) throw FailedPredicateException(this, "precpred(_ctx, 11)"); - setState(1724); + setState(1740); match(ClickHouseParser::AND); - setState(1725); + setState(1741); columnExpr(12); break; } @@ -15113,12 +15176,12 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1726); + setState(1742); if (!(precpred(_ctx, 10))) throw FailedPredicateException(this, "precpred(_ctx, 10)"); - setState(1727); + setState(1743); match(ClickHouseParser::OR); - setState(1728); + setState(1744); columnExpr(11); break; } @@ -15127,24 +15190,24 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1729); + setState(1745); if (!(precpred(_ctx, 9))) throw FailedPredicateException(this, "precpred(_ctx, 9)"); - setState(1731); + setState(1747); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::NOT) { - setState(1730); + setState(1746); match(ClickHouseParser::NOT); } - setState(1733); + setState(1749); match(ClickHouseParser::BETWEEN); - setState(1734); + setState(1750); columnExpr(0); - setState(1735); + setState(1751); match(ClickHouseParser::AND); - setState(1736); + setState(1752); columnExpr(10); break; } @@ -15153,16 +15216,16 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1738); + setState(1754); if (!(precpred(_ctx, 8))) throw FailedPredicateException(this, "precpred(_ctx, 8)"); - setState(1739); + setState(1755); match(ClickHouseParser::QUERY); - setState(1740); + setState(1756); columnExpr(0); - setState(1741); + setState(1757); match(ClickHouseParser::COLON); - setState(1742); + setState(1758); columnExpr(8); break; } @@ -15171,14 +15234,14 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1744); + setState(1760); if (!(precpred(_ctx, 19))) throw FailedPredicateException(this, "precpred(_ctx, 19)"); - setState(1745); + setState(1761); match(ClickHouseParser::LBRACKET); - setState(1746); + setState(1762); columnExpr(0); - setState(1747); + setState(1763); match(ClickHouseParser::RBRACKET); break; } @@ -15187,12 +15250,12 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1749); + setState(1765); if (!(precpred(_ctx, 18))) throw FailedPredicateException(this, "precpred(_ctx, 18)"); - setState(1750); + setState(1766); match(ClickHouseParser::DOT); - setState(1751); + setState(1767); match(ClickHouseParser::DECIMAL_LITERAL); break; } @@ -15201,20 +15264,20 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1752); + setState(1768); if (!(precpred(_ctx, 13))) throw FailedPredicateException(this, "precpred(_ctx, 13)"); - setState(1753); + setState(1769); match(ClickHouseParser::IS); - setState(1755); + setState(1771); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::NOT) { - setState(1754); + setState(1770); match(ClickHouseParser::NOT); } - setState(1757); + setState(1773); match(ClickHouseParser::NULL_SQL); break; } @@ -15223,10 +15286,10 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1758); + setState(1774); if (!(precpred(_ctx, 7))) throw FailedPredicateException(this, "precpred(_ctx, 7)"); - setState(1762); + setState(1778); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::DATE: @@ -15234,15 +15297,15 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence case ClickHouseParser::ID: case ClickHouseParser::KEY: case ClickHouseParser::IDENTIFIER: { - setState(1759); + setState(1775); alias(); break; } case ClickHouseParser::AS: { - setState(1760); + setState(1776); match(ClickHouseParser::AS); - setState(1761); + setState(1777); identifier(); break; } @@ -15255,9 +15318,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence } } - setState(1768); + setState(1784); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 234, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 238, _ctx); } } catch (RecognitionException &e) { @@ -15312,17 +15375,17 @@ ClickHouseParser::ColumnArgListContext* ClickHouseParser::columnArgList() { }); try { enterOuterAlt(_localctx, 1); - setState(1769); + setState(1785); columnArgExpr(); - setState(1774); + setState(1790); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1770); + setState(1786); match(ClickHouseParser::COMMA); - setState(1771); + setState(1787); columnArgExpr(); - setState(1776); + setState(1792); _errHandler->sync(this); _la = _input->LA(1); } @@ -15371,19 +15434,19 @@ ClickHouseParser::ColumnArgExprContext* ClickHouseParser::columnArgExpr() { exitRule(); }); try { - setState(1779); + setState(1795); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 236, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 240, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(1777); + setState(1793); columnLambdaExpr(); break; } case 2: { enterOuterAlt(_localctx, 2); - setState(1778); + setState(1794); columnExpr(0); break; } @@ -15460,27 +15523,27 @@ ClickHouseParser::ColumnLambdaExprContext* ClickHouseParser::columnLambdaExpr() }); try { enterOuterAlt(_localctx, 1); - setState(1800); + setState(1816); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::LPAREN: { - setState(1781); + setState(1797); match(ClickHouseParser::LPAREN); - setState(1782); + setState(1798); identifier(); - setState(1787); + setState(1803); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1783); + setState(1799); match(ClickHouseParser::COMMA); - setState(1784); + setState(1800); identifier(); - setState(1789); + setState(1805); _errHandler->sync(this); _la = _input->LA(1); } - setState(1790); + setState(1806); match(ClickHouseParser::RPAREN); break; } @@ -15664,17 +15727,17 @@ ClickHouseParser::ColumnLambdaExprContext* ClickHouseParser::columnLambdaExpr() case ClickHouseParser::JSON_FALSE: case ClickHouseParser::JSON_TRUE: case ClickHouseParser::IDENTIFIER: { - setState(1792); + setState(1808); identifier(); - setState(1797); + setState(1813); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1793); + setState(1809); match(ClickHouseParser::COMMA); - setState(1794); + setState(1810); identifier(); - setState(1799); + setState(1815); _errHandler->sync(this); _la = _input->LA(1); } @@ -15684,9 +15747,9 @@ ClickHouseParser::ColumnLambdaExprContext* ClickHouseParser::columnLambdaExpr() default: throw NoViableAltException(this); } - setState(1802); + setState(1818); match(ClickHouseParser::ARROW); - setState(1803); + setState(1819); columnExpr(0); } @@ -15738,20 +15801,20 @@ ClickHouseParser::ColumnIdentifierContext* ClickHouseParser::columnIdentifier() }); try { enterOuterAlt(_localctx, 1); - setState(1808); + setState(1824); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 240, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 244, _ctx)) { case 1: { - setState(1805); + setState(1821); tableIdentifier(); - setState(1806); + setState(1822); match(ClickHouseParser::DOT); break; } } - setState(1810); + setState(1826); nestedIdentifier(); } @@ -15803,16 +15866,16 @@ ClickHouseParser::NestedIdentifierContext* ClickHouseParser::nestedIdentifier() }); try { enterOuterAlt(_localctx, 1); - setState(1812); + setState(1828); identifier(); - setState(1815); + setState(1831); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 241, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 245, _ctx)) { case 1: { - setState(1813); + setState(1829); match(ClickHouseParser::DOT); - setState(1814); + setState(1830); identifier(); break; } @@ -15942,15 +16005,15 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1824); + setState(1840); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 242, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 246, _ctx)) { case 1: { _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1818); + setState(1834); tableIdentifier(); break; } @@ -15959,7 +16022,7 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1819); + setState(1835); tableFunctionExpr(); break; } @@ -15968,20 +16031,20 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1820); + setState(1836); match(ClickHouseParser::LPAREN); - setState(1821); + setState(1837); selectUnionStmt(); - setState(1822); + setState(1838); match(ClickHouseParser::RPAREN); break; } } _ctx->stop = _input->LT(-1); - setState(1834); + setState(1850); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 244, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 248, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { if (!_parseListeners.empty()) @@ -15990,10 +16053,10 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleTableExpr); - setState(1826); + setState(1842); if (!(precpred(_ctx, 1))) throw FailedPredicateException(this, "precpred(_ctx, 1)"); - setState(1830); + setState(1846); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::DATE: @@ -16001,15 +16064,15 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) case ClickHouseParser::ID: case ClickHouseParser::KEY: case ClickHouseParser::IDENTIFIER: { - setState(1827); + setState(1843); alias(); break; } case ClickHouseParser::AS: { - setState(1828); + setState(1844); match(ClickHouseParser::AS); - setState(1829); + setState(1845); identifier(); break; } @@ -16018,9 +16081,9 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) throw NoViableAltException(this); } } - setState(1836); + setState(1852); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 244, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 248, _ctx); } } catch (RecognitionException &e) { @@ -16075,11 +16138,11 @@ ClickHouseParser::TableFunctionExprContext* ClickHouseParser::tableFunctionExpr( }); try { enterOuterAlt(_localctx, 1); - setState(1837); + setState(1853); identifier(); - setState(1838); + setState(1854); match(ClickHouseParser::LPAREN); - setState(1840); + setState(1856); _errHandler->sync(this); _la = _input->LA(1); @@ -16274,10 +16337,10 @@ ClickHouseParser::TableFunctionExprContext* ClickHouseParser::tableFunctionExpr( ((1ULL << (_la - 197)) & ((1ULL << (ClickHouseParser::DASH - 197)) | (1ULL << (ClickHouseParser::DOT - 197)) | (1ULL << (ClickHouseParser::PLUS - 197)))) != 0)) { - setState(1839); + setState(1855); tableArgList(); } - setState(1842); + setState(1858); match(ClickHouseParser::RPAREN); } @@ -16329,20 +16392,20 @@ ClickHouseParser::TableIdentifierContext* ClickHouseParser::tableIdentifier() { }); try { enterOuterAlt(_localctx, 1); - setState(1847); + setState(1863); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 246, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 250, _ctx)) { case 1: { - setState(1844); + setState(1860); databaseIdentifier(); - setState(1845); + setState(1861); match(ClickHouseParser::DOT); break; } } - setState(1849); + setState(1865); identifier(); } @@ -16399,17 +16462,17 @@ ClickHouseParser::TableArgListContext* ClickHouseParser::tableArgList() { }); try { enterOuterAlt(_localctx, 1); - setState(1851); + setState(1867); tableArgExpr(); - setState(1856); + setState(1872); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1852); + setState(1868); match(ClickHouseParser::COMMA); - setState(1853); + setState(1869); tableArgExpr(); - setState(1858); + setState(1874); _errHandler->sync(this); _la = _input->LA(1); } @@ -16462,26 +16525,26 @@ ClickHouseParser::TableArgExprContext* ClickHouseParser::tableArgExpr() { exitRule(); }); try { - setState(1862); + setState(1878); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 248, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 252, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(1859); + setState(1875); nestedIdentifier(); break; } case 2: { enterOuterAlt(_localctx, 2); - setState(1860); + setState(1876); tableFunctionExpr(); break; } case 3: { enterOuterAlt(_localctx, 3); - setState(1861); + setState(1877); literal(); break; } @@ -16529,7 +16592,7 @@ ClickHouseParser::DatabaseIdentifierContext* ClickHouseParser::databaseIdentifie }); try { enterOuterAlt(_localctx, 1); - setState(1864); + setState(1880); identifier(); } @@ -16589,21 +16652,21 @@ ClickHouseParser::FloatingLiteralContext* ClickHouseParser::floatingLiteral() { exitRule(); }); try { - setState(1874); + setState(1890); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::FLOATING_LITERAL: { enterOuterAlt(_localctx, 1); - setState(1866); + setState(1882); match(ClickHouseParser::FLOATING_LITERAL); break; } case ClickHouseParser::DOT: { enterOuterAlt(_localctx, 2); - setState(1867); + setState(1883); match(ClickHouseParser::DOT); - setState(1868); + setState(1884); _la = _input->LA(1); if (!(_la == ClickHouseParser::OCTAL_LITERAL @@ -16619,16 +16682,16 @@ ClickHouseParser::FloatingLiteralContext* ClickHouseParser::floatingLiteral() { case ClickHouseParser::DECIMAL_LITERAL: { enterOuterAlt(_localctx, 3); - setState(1869); + setState(1885); match(ClickHouseParser::DECIMAL_LITERAL); - setState(1870); + setState(1886); match(ClickHouseParser::DOT); - setState(1872); + setState(1888); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 249, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 253, _ctx)) { case 1: { - setState(1871); + setState(1887); _la = _input->LA(1); if (!(_la == ClickHouseParser::OCTAL_LITERAL @@ -16720,14 +16783,14 @@ ClickHouseParser::NumberLiteralContext* ClickHouseParser::numberLiteral() { }); try { enterOuterAlt(_localctx, 1); - setState(1877); + setState(1893); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::DASH || _la == ClickHouseParser::PLUS) { - setState(1876); + setState(1892); _la = _input->LA(1); if (!(_la == ClickHouseParser::DASH @@ -16739,41 +16802,41 @@ ClickHouseParser::NumberLiteralContext* ClickHouseParser::numberLiteral() { consume(); } } - setState(1885); + setState(1901); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 252, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 256, _ctx)) { case 1: { - setState(1879); + setState(1895); floatingLiteral(); break; } case 2: { - setState(1880); + setState(1896); match(ClickHouseParser::OCTAL_LITERAL); break; } case 3: { - setState(1881); + setState(1897); match(ClickHouseParser::DECIMAL_LITERAL); break; } case 4: { - setState(1882); + setState(1898); match(ClickHouseParser::HEXADECIMAL_LITERAL); break; } case 5: { - setState(1883); + setState(1899); match(ClickHouseParser::INF); break; } case 6: { - setState(1884); + setState(1900); match(ClickHouseParser::NAN_SQL); break; } @@ -16828,7 +16891,7 @@ ClickHouseParser::LiteralContext* ClickHouseParser::literal() { exitRule(); }); try { - setState(1890); + setState(1906); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::INF: @@ -16841,21 +16904,21 @@ ClickHouseParser::LiteralContext* ClickHouseParser::literal() { case ClickHouseParser::DOT: case ClickHouseParser::PLUS: { enterOuterAlt(_localctx, 1); - setState(1887); + setState(1903); numberLiteral(); break; } case ClickHouseParser::STRING_LITERAL: { enterOuterAlt(_localctx, 2); - setState(1888); + setState(1904); match(ClickHouseParser::STRING_LITERAL); break; } case ClickHouseParser::NULL_SQL: { enterOuterAlt(_localctx, 3); - setState(1889); + setState(1905); match(ClickHouseParser::NULL_SQL); break; } @@ -16934,7 +16997,7 @@ ClickHouseParser::IntervalContext* ClickHouseParser::interval() { }); try { enterOuterAlt(_localctx, 1); - setState(1892); + setState(1908); _la = _input->LA(1); if (!(_la == ClickHouseParser::DAY || ((((_la - 73) & ~ 0x3fULL) == 0) && ((1ULL << (_la - 73)) & ((1ULL << (ClickHouseParser::HOUR - 73)) @@ -17669,7 +17732,7 @@ ClickHouseParser::KeywordContext* ClickHouseParser::keyword() { }); try { enterOuterAlt(_localctx, 1); - setState(1894); + setState(1910); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::AFTER) @@ -17903,7 +17966,7 @@ ClickHouseParser::KeywordForAliasContext* ClickHouseParser::keywordForAlias() { }); try { enterOuterAlt(_localctx, 1); - setState(1896); + setState(1912); _la = _input->LA(1); if (!(((((_la - 34) & ~ 0x3fULL) == 0) && ((1ULL << (_la - 34)) & ((1ULL << (ClickHouseParser::DATE - 34)) @@ -17961,12 +18024,12 @@ ClickHouseParser::AliasContext* ClickHouseParser::alias() { exitRule(); }); try { - setState(1900); + setState(1916); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::IDENTIFIER: { enterOuterAlt(_localctx, 1); - setState(1898); + setState(1914); match(ClickHouseParser::IDENTIFIER); break; } @@ -17976,7 +18039,7 @@ ClickHouseParser::AliasContext* ClickHouseParser::alias() { case ClickHouseParser::ID: case ClickHouseParser::KEY: { enterOuterAlt(_localctx, 2); - setState(1899); + setState(1915); keywordForAlias(); break; } @@ -18033,12 +18096,12 @@ ClickHouseParser::IdentifierContext* ClickHouseParser::identifier() { exitRule(); }); try { - setState(1905); + setState(1921); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::IDENTIFIER: { enterOuterAlt(_localctx, 1); - setState(1902); + setState(1918); match(ClickHouseParser::IDENTIFIER); break; } @@ -18052,7 +18115,7 @@ ClickHouseParser::IdentifierContext* ClickHouseParser::identifier() { case ClickHouseParser::WEEK: case ClickHouseParser::YEAR: { enterOuterAlt(_localctx, 2); - setState(1903); + setState(1919); interval(); break; } @@ -18228,7 +18291,7 @@ ClickHouseParser::IdentifierContext* ClickHouseParser::identifier() { case ClickHouseParser::JSON_FALSE: case ClickHouseParser::JSON_TRUE: { enterOuterAlt(_localctx, 3); - setState(1904); + setState(1920); keyword(); break; } @@ -18281,7 +18344,7 @@ ClickHouseParser::IdentifierOrNullContext* ClickHouseParser::identifierOrNull() exitRule(); }); try { - setState(1909); + setState(1925); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::AFTER: @@ -18464,14 +18527,14 @@ ClickHouseParser::IdentifierOrNullContext* ClickHouseParser::identifierOrNull() case ClickHouseParser::JSON_TRUE: case ClickHouseParser::IDENTIFIER: { enterOuterAlt(_localctx, 1); - setState(1907); + setState(1923); identifier(); break; } case ClickHouseParser::NULL_SQL: { enterOuterAlt(_localctx, 2); - setState(1908); + setState(1924); match(ClickHouseParser::NULL_SQL); break; } @@ -18529,11 +18592,11 @@ ClickHouseParser::EnumValueContext* ClickHouseParser::enumValue() { }); try { enterOuterAlt(_localctx, 1); - setState(1911); + setState(1927); match(ClickHouseParser::STRING_LITERAL); - setState(1912); + setState(1928); match(ClickHouseParser::EQ_SINGLE); - setState(1913); + setState(1929); numberLiteral(); } @@ -18751,7 +18814,7 @@ ClickHouseParser::Initializer::Initializer() { _serializedATN = { 0x3, 0x608b, 0xa72a, 0x8133, 0xb9ed, 0x417c, 0x3be7, 0x7786, 0x5964, - 0x3, 0xe0, 0x77e, 0x4, 0x2, 0x9, 0x2, 0x4, 0x3, 0x9, 0x3, 0x4, 0x4, + 0x3, 0xe0, 0x78e, 0x4, 0x2, 0x9, 0x2, 0x4, 0x3, 0x9, 0x3, 0x4, 0x4, 0x9, 0x4, 0x4, 0x5, 0x9, 0x5, 0x4, 0x6, 0x9, 0x6, 0x4, 0x7, 0x9, 0x7, 0x4, 0x8, 0x9, 0x8, 0x4, 0x9, 0x9, 0x9, 0x4, 0xa, 0x9, 0xa, 0x4, 0xb, 0x9, 0xb, 0x4, 0xc, 0x9, 0xc, 0x4, 0xd, 0x9, 0xd, 0x4, 0xe, 0x9, 0xe, @@ -18833,239 +18896,242 @@ ClickHouseParser::Initializer::Initializer() { 0xb, 0x3, 0xb, 0x5, 0xb, 0x206, 0xa, 0xb, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x20d, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x211, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x214, 0xa, 0xc, 0x3, 0xc, - 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x21b, 0xa, 0xc, 0x3, - 0xc, 0x3, 0xc, 0x5, 0xc, 0x21f, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x222, - 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, - 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x22d, 0xa, 0xc, 0x3, 0xc, 0x3, - 0xc, 0x5, 0xc, 0x231, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x234, 0xa, 0xc, - 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x239, 0xa, 0xc, 0x5, 0xc, 0x23b, - 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x23e, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x241, - 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, - 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x24b, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, - 0xc, 0x24f, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x252, 0xa, 0xc, 0x3, 0xc, - 0x5, 0xc, 0x255, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x25a, - 0xa, 0xc, 0x5, 0xc, 0x25c, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, - 0xc, 0x5, 0xc, 0x262, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, - 0x5, 0xc, 0x268, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x26c, 0xa, + 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x21a, 0xa, 0xc, 0x3, 0xc, 0x5, + 0xc, 0x21d, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, + 0x223, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x227, 0xa, 0xc, 0x3, + 0xc, 0x5, 0xc, 0x22a, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, + 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x235, 0xa, + 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x239, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, + 0x23c, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x241, 0xa, + 0xc, 0x5, 0xc, 0x243, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x246, 0xa, 0xc, + 0x3, 0xc, 0x5, 0xc, 0x249, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, + 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x253, 0xa, 0xc, + 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x257, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x25a, + 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x25d, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, + 0xc, 0x5, 0xc, 0x262, 0xa, 0xc, 0x5, 0xc, 0x264, 0xa, 0xc, 0x3, 0xc, + 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x26c, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x26f, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x272, - 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x275, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x278, - 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x27d, 0xa, 0xc, 0x3, - 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x283, 0xa, 0xc, 0x3, 0xc, - 0x3, 0xc, 0x5, 0xc, 0x287, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x28a, 0xa, - 0xc, 0x3, 0xc, 0x5, 0xc, 0x28d, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, - 0x291, 0xa, 0xc, 0x3, 0xd, 0x3, 0xd, 0x3, 0xd, 0x3, 0xd, 0x7, 0xd, 0x297, - 0xa, 0xd, 0xc, 0xd, 0xe, 0xd, 0x29a, 0xb, 0xd, 0x3, 0xd, 0x3, 0xd, 0x3, - 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, - 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, - 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x7, - 0xe, 0x2b3, 0xa, 0xe, 0xc, 0xe, 0xe, 0xe, 0x2b6, 0xb, 0xe, 0x3, 0xf, - 0x5, 0xf, 0x2b9, 0xa, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, - 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, - 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, - 0xf, 0x3, 0xf, 0x7, 0xf, 0x2cf, 0xa, 0xf, 0xc, 0xf, 0xe, 0xf, 0x2d2, - 0xb, 0xf, 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, 0x3, 0x11, 0x3, - 0x11, 0x3, 0x11, 0x3, 0x11, 0x5, 0x11, 0x2dc, 0xa, 0x11, 0x3, 0x11, - 0x5, 0x11, 0x2df, 0xa, 0x11, 0x3, 0x12, 0x3, 0x12, 0x3, 0x12, 0x3, 0x12, - 0x3, 0x12, 0x7, 0x12, 0x2e6, 0xa, 0x12, 0xc, 0x12, 0xe, 0x12, 0x2e9, - 0xb, 0x12, 0x3, 0x12, 0x3, 0x12, 0x3, 0x12, 0x3, 0x13, 0x3, 0x13, 0x3, - 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, - 0x3, 0x13, 0x3, 0x13, 0x5, 0x13, 0x2f9, 0xa, 0x13, 0x3, 0x13, 0x3, 0x13, - 0x3, 0x14, 0x3, 0x14, 0x3, 0x14, 0x3, 0x14, 0x3, 0x14, 0x7, 0x14, 0x302, - 0xa, 0x14, 0xc, 0x14, 0xe, 0x14, 0x305, 0xb, 0x14, 0x3, 0x14, 0x3, 0x14, - 0x3, 0x14, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, - 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, - 0x5, 0x15, 0x316, 0xa, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x16, 0x3, 0x16, - 0x3, 0x16, 0x3, 0x16, 0x3, 0x16, 0x3, 0x17, 0x3, 0x17, 0x3, 0x17, 0x3, - 0x17, 0x5, 0x17, 0x323, 0xa, 0x17, 0x3, 0x18, 0x3, 0x18, 0x3, 0x18, - 0x3, 0x19, 0x3, 0x19, 0x3, 0x19, 0x3, 0x1a, 0x3, 0x1a, 0x3, 0x1a, 0x3, - 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x7, 0x1b, 0x332, 0xa, 0x1b, - 0xc, 0x1b, 0xe, 0x1b, 0x335, 0xb, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, - 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x5, 0x1b, 0x33d, 0xa, 0x1b, 0x3, 0x1c, + 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x278, 0xa, + 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x27c, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, + 0x27f, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x282, 0xa, 0xc, 0x3, 0xc, 0x5, + 0xc, 0x285, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x288, 0xa, 0xc, 0x3, 0xc, + 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x28d, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, + 0xc, 0x3, 0xc, 0x5, 0xc, 0x293, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, + 0x297, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x29a, 0xa, 0xc, 0x3, 0xc, 0x5, + 0xc, 0x29d, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x2a1, 0xa, 0xc, + 0x3, 0xd, 0x3, 0xd, 0x3, 0xd, 0x3, 0xd, 0x7, 0xd, 0x2a7, 0xa, 0xd, 0xc, + 0xd, 0xe, 0xd, 0x2aa, 0xb, 0xd, 0x3, 0xd, 0x3, 0xd, 0x3, 0xe, 0x3, 0xe, + 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, + 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, + 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x7, 0xe, 0x2c3, 0xa, + 0xe, 0xc, 0xe, 0xe, 0xe, 0x2c6, 0xb, 0xe, 0x3, 0xf, 0x5, 0xf, 0x2c9, + 0xa, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, + 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, + 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, + 0x7, 0xf, 0x2df, 0xa, 0xf, 0xc, 0xf, 0xe, 0xf, 0x2e2, 0xb, 0xf, 0x3, + 0x10, 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, 0x3, 0x11, 0x3, 0x11, 0x3, 0x11, + 0x3, 0x11, 0x5, 0x11, 0x2ec, 0xa, 0x11, 0x3, 0x11, 0x5, 0x11, 0x2ef, + 0xa, 0x11, 0x3, 0x12, 0x3, 0x12, 0x3, 0x12, 0x3, 0x12, 0x3, 0x12, 0x7, + 0x12, 0x2f6, 0xa, 0x12, 0xc, 0x12, 0xe, 0x12, 0x2f9, 0xb, 0x12, 0x3, + 0x12, 0x3, 0x12, 0x3, 0x12, 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, + 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, + 0x13, 0x5, 0x13, 0x309, 0xa, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x14, + 0x3, 0x14, 0x3, 0x14, 0x3, 0x14, 0x3, 0x14, 0x7, 0x14, 0x312, 0xa, 0x14, + 0xc, 0x14, 0xe, 0x14, 0x315, 0xb, 0x14, 0x3, 0x14, 0x3, 0x14, 0x3, 0x14, + 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, + 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x5, 0x15, + 0x326, 0xa, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x16, 0x3, 0x16, 0x3, 0x16, + 0x3, 0x16, 0x3, 0x16, 0x3, 0x17, 0x3, 0x17, 0x3, 0x17, 0x3, 0x17, 0x5, + 0x17, 0x333, 0xa, 0x17, 0x3, 0x18, 0x3, 0x18, 0x3, 0x18, 0x3, 0x19, + 0x3, 0x19, 0x3, 0x19, 0x3, 0x1a, 0x3, 0x1a, 0x3, 0x1a, 0x3, 0x1b, 0x3, + 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x7, 0x1b, 0x342, 0xa, 0x1b, 0xc, 0x1b, + 0xe, 0x1b, 0x345, 0xb, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, + 0x3, 0x1b, 0x3, 0x1b, 0x5, 0x1b, 0x34d, 0xa, 0x1b, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, - 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x7, 0x1c, 0x358, - 0xa, 0x1c, 0xc, 0x1c, 0xe, 0x1c, 0x35b, 0xb, 0x1c, 0x3, 0x1d, 0x3, 0x1d, - 0x3, 0x1d, 0x3, 0x1d, 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1e, 0x3, - 0x1f, 0x3, 0x1f, 0x3, 0x1f, 0x3, 0x1f, 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, - 0x3, 0x20, 0x7, 0x20, 0x36d, 0xa, 0x20, 0xc, 0x20, 0xe, 0x20, 0x370, - 0xb, 0x20, 0x3, 0x21, 0x3, 0x21, 0x5, 0x21, 0x374, 0xa, 0x21, 0x3, 0x21, - 0x3, 0x21, 0x3, 0x21, 0x5, 0x21, 0x379, 0xa, 0x21, 0x3, 0x21, 0x5, 0x21, - 0x37c, 0xa, 0x21, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, - 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x5, 0x22, 0x388, - 0xa, 0x22, 0x3, 0x23, 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x38d, 0xa, 0x23, - 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x391, 0xa, 0x23, 0x3, 0x23, 0x5, 0x23, - 0x394, 0xa, 0x23, 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x398, 0xa, 0x23, - 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x39c, 0xa, 0x23, 0x3, 0x23, 0x3, 0x23, - 0x3, 0x23, 0x5, 0x23, 0x3a1, 0xa, 0x23, 0x3, 0x23, 0x5, 0x23, 0x3a4, - 0xa, 0x23, 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x3a8, 0xa, 0x23, 0x5, 0x23, - 0x3aa, 0xa, 0x23, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x25, 0x3, 0x25, + 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x7, 0x1c, 0x368, 0xa, 0x1c, + 0xc, 0x1c, 0xe, 0x1c, 0x36b, 0xb, 0x1c, 0x3, 0x1d, 0x3, 0x1d, 0x3, 0x1d, + 0x3, 0x1d, 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1f, 0x3, + 0x1f, 0x3, 0x1f, 0x3, 0x1f, 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, + 0x7, 0x20, 0x37d, 0xa, 0x20, 0xc, 0x20, 0xe, 0x20, 0x380, 0xb, 0x20, + 0x3, 0x21, 0x3, 0x21, 0x5, 0x21, 0x384, 0xa, 0x21, 0x3, 0x21, 0x3, 0x21, + 0x3, 0x21, 0x5, 0x21, 0x389, 0xa, 0x21, 0x3, 0x21, 0x5, 0x21, 0x38c, + 0xa, 0x21, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, + 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x5, 0x22, 0x398, + 0xa, 0x22, 0x3, 0x23, 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x39d, 0xa, 0x23, + 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x3a1, 0xa, 0x23, 0x3, 0x23, 0x5, 0x23, + 0x3a4, 0xa, 0x23, 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x3a8, 0xa, 0x23, + 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x3ac, 0xa, 0x23, 0x3, 0x23, 0x3, 0x23, + 0x3, 0x23, 0x5, 0x23, 0x3b1, 0xa, 0x23, 0x3, 0x23, 0x5, 0x23, 0x3b4, + 0xa, 0x23, 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x3b8, 0xa, 0x23, 0x5, 0x23, + 0x3ba, 0xa, 0x23, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x26, 0x3, 0x26, 0x3, 0x26, 0x3, 0x27, 0x3, 0x27, 0x3, 0x27, 0x3, 0x27, 0x3, 0x27, - 0x7, 0x27, 0x3be, 0xa, 0x27, 0xc, 0x27, 0xe, 0x27, 0x3c1, 0xb, 0x27, - 0x3, 0x27, 0x3, 0x27, 0x3, 0x28, 0x3, 0x28, 0x3, 0x28, 0x5, 0x28, 0x3c8, - 0xa, 0x28, 0x3, 0x28, 0x5, 0x28, 0x3cb, 0xa, 0x28, 0x3, 0x29, 0x3, 0x29, + 0x7, 0x27, 0x3ce, 0xa, 0x27, 0xc, 0x27, 0xe, 0x27, 0x3d1, 0xb, 0x27, + 0x3, 0x27, 0x3, 0x27, 0x3, 0x28, 0x3, 0x28, 0x3, 0x28, 0x5, 0x28, 0x3d8, + 0xa, 0x28, 0x3, 0x28, 0x5, 0x28, 0x3db, 0xa, 0x28, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x5, - 0x29, 0x3d5, 0xa, 0x29, 0x3, 0x2a, 0x3, 0x2a, 0x5, 0x2a, 0x3d9, 0xa, + 0x29, 0x3e5, 0xa, 0x29, 0x3, 0x2a, 0x3, 0x2a, 0x5, 0x2a, 0x3e9, 0xa, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2b, - 0x5, 0x2b, 0x3e1, 0xa, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x3e5, - 0xa, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x3ea, 0xa, 0x2b, - 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x3ee, 0xa, 0x2b, 0x3, 0x2b, 0x3, 0x2b, - 0x5, 0x2b, 0x3f2, 0xa, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x3f6, - 0xa, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x3fa, 0xa, 0x2b, 0x5, 0x2b, - 0x3fc, 0xa, 0x2b, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, - 0x3, 0x2c, 0x5, 0x2c, 0x404, 0xa, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x5, 0x2c, - 0x408, 0xa, 0x2c, 0x3, 0x2c, 0x5, 0x2c, 0x40b, 0xa, 0x2c, 0x3, 0x2d, - 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x5, 0x2d, 0x413, - 0xa, 0x2d, 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x5, 0x2e, 0x418, 0xa, 0x2e, - 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x5, 0x2e, 0x41d, 0xa, 0x2e, 0x3, 0x2e, - 0x5, 0x2e, 0x420, 0xa, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2f, 0x3, 0x2f, - 0x3, 0x2f, 0x3, 0x2f, 0x7, 0x2f, 0x428, 0xa, 0x2f, 0xc, 0x2f, 0xe, 0x2f, - 0x42b, 0xb, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, - 0x3, 0x30, 0x3, 0x30, 0x5, 0x30, 0x434, 0xa, 0x30, 0x3, 0x30, 0x3, 0x30, - 0x5, 0x30, 0x438, 0xa, 0x30, 0x3, 0x31, 0x3, 0x31, 0x3, 0x31, 0x5, 0x31, - 0x43d, 0xa, 0x31, 0x3, 0x31, 0x3, 0x31, 0x5, 0x31, 0x441, 0xa, 0x31, - 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x5, 0x32, 0x447, 0xa, 0x32, - 0x3, 0x32, 0x5, 0x32, 0x44a, 0xa, 0x32, 0x3, 0x32, 0x5, 0x32, 0x44d, - 0xa, 0x32, 0x3, 0x32, 0x5, 0x32, 0x450, 0xa, 0x32, 0x3, 0x33, 0x3, 0x33, + 0x5, 0x2b, 0x3f1, 0xa, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x3f5, + 0xa, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x3fa, 0xa, 0x2b, + 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x3fe, 0xa, 0x2b, 0x3, 0x2b, 0x3, 0x2b, + 0x5, 0x2b, 0x402, 0xa, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x406, + 0xa, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x40a, 0xa, 0x2b, 0x5, 0x2b, + 0x40c, 0xa, 0x2b, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, + 0x3, 0x2c, 0x5, 0x2c, 0x414, 0xa, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x5, 0x2c, + 0x418, 0xa, 0x2c, 0x3, 0x2c, 0x5, 0x2c, 0x41b, 0xa, 0x2c, 0x3, 0x2d, + 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x5, 0x2d, 0x423, + 0xa, 0x2d, 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x5, 0x2e, 0x428, 0xa, 0x2e, + 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x5, 0x2e, 0x42d, 0xa, 0x2e, 0x3, 0x2e, + 0x5, 0x2e, 0x430, 0xa, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2f, 0x3, 0x2f, + 0x3, 0x2f, 0x3, 0x2f, 0x7, 0x2f, 0x438, 0xa, 0x2f, 0xc, 0x2f, 0xe, 0x2f, + 0x43b, 0xb, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, + 0x3, 0x30, 0x3, 0x30, 0x5, 0x30, 0x444, 0xa, 0x30, 0x3, 0x30, 0x3, 0x30, + 0x5, 0x30, 0x448, 0xa, 0x30, 0x3, 0x31, 0x3, 0x31, 0x3, 0x31, 0x5, 0x31, + 0x44d, 0xa, 0x31, 0x3, 0x31, 0x3, 0x31, 0x5, 0x31, 0x451, 0xa, 0x31, + 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x5, 0x32, 0x457, 0xa, 0x32, + 0x3, 0x32, 0x5, 0x32, 0x45a, 0xa, 0x32, 0x3, 0x32, 0x5, 0x32, 0x45d, + 0xa, 0x32, 0x3, 0x32, 0x5, 0x32, 0x460, 0xa, 0x32, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, - 0x33, 0x3, 0x33, 0x7, 0x33, 0x45c, 0xa, 0x33, 0xc, 0x33, 0xe, 0x33, - 0x45f, 0xb, 0x33, 0x3, 0x33, 0x5, 0x33, 0x462, 0xa, 0x33, 0x3, 0x34, - 0x3, 0x34, 0x5, 0x34, 0x466, 0xa, 0x34, 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, - 0x5, 0x34, 0x46b, 0xa, 0x34, 0x3, 0x34, 0x5, 0x34, 0x46e, 0xa, 0x34, + 0x33, 0x3, 0x33, 0x7, 0x33, 0x46c, 0xa, 0x33, 0xc, 0x33, 0xe, 0x33, + 0x46f, 0xb, 0x33, 0x3, 0x33, 0x5, 0x33, 0x472, 0xa, 0x33, 0x3, 0x34, + 0x3, 0x34, 0x5, 0x34, 0x476, 0xa, 0x34, 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, + 0x5, 0x34, 0x47b, 0xa, 0x34, 0x3, 0x34, 0x5, 0x34, 0x47e, 0xa, 0x34, 0x3, 0x34, 0x3, 0x34, 0x3, 0x35, 0x3, 0x35, 0x3, 0x35, 0x3, 0x35, 0x7, - 0x35, 0x476, 0xa, 0x35, 0xc, 0x35, 0xe, 0x35, 0x479, 0xb, 0x35, 0x3, - 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x5, 0x36, 0x480, - 0xa, 0x36, 0x3, 0x37, 0x5, 0x37, 0x483, 0xa, 0x37, 0x3, 0x37, 0x3, 0x37, - 0x5, 0x37, 0x487, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x48a, 0xa, 0x37, - 0x3, 0x37, 0x3, 0x37, 0x5, 0x37, 0x48e, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, - 0x491, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x494, 0xa, 0x37, 0x3, 0x37, + 0x35, 0x486, 0xa, 0x35, 0xc, 0x35, 0xe, 0x35, 0x489, 0xb, 0x35, 0x3, + 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x5, 0x36, 0x490, + 0xa, 0x36, 0x3, 0x37, 0x5, 0x37, 0x493, 0xa, 0x37, 0x3, 0x37, 0x3, 0x37, 0x5, 0x37, 0x497, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x49a, 0xa, 0x37, - 0x3, 0x37, 0x3, 0x37, 0x5, 0x37, 0x49e, 0xa, 0x37, 0x3, 0x37, 0x3, 0x37, - 0x5, 0x37, 0x4a2, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x4a5, 0xa, 0x37, - 0x3, 0x37, 0x5, 0x37, 0x4a8, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x4ab, - 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x4ae, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, - 0x4b1, 0xa, 0x37, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x39, 0x3, 0x39, - 0x3, 0x39, 0x3, 0x39, 0x5, 0x39, 0x4ba, 0xa, 0x39, 0x3, 0x3a, 0x3, 0x3a, - 0x3, 0x3a, 0x3, 0x3b, 0x5, 0x3b, 0x4c0, 0xa, 0x3b, 0x3, 0x3b, 0x3, 0x3b, + 0x3, 0x37, 0x3, 0x37, 0x5, 0x37, 0x49e, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, + 0x4a1, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x4a4, 0xa, 0x37, 0x3, 0x37, + 0x5, 0x37, 0x4a7, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x4aa, 0xa, 0x37, + 0x3, 0x37, 0x3, 0x37, 0x5, 0x37, 0x4ae, 0xa, 0x37, 0x3, 0x37, 0x3, 0x37, + 0x5, 0x37, 0x4b2, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x4b5, 0xa, 0x37, + 0x3, 0x37, 0x5, 0x37, 0x4b8, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x4bb, + 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x4be, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, + 0x4c1, 0xa, 0x37, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x39, 0x3, 0x39, + 0x3, 0x39, 0x3, 0x39, 0x5, 0x39, 0x4ca, 0xa, 0x39, 0x3, 0x3a, 0x3, 0x3a, + 0x3, 0x3a, 0x3, 0x3b, 0x5, 0x3b, 0x4d0, 0xa, 0x3b, 0x3, 0x3b, 0x3, 0x3b, 0x3, 0x3b, 0x3, 0x3b, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3d, 0x3, 0x3d, 0x3, 0x3d, 0x3, 0x3e, 0x3, 0x3e, 0x3, 0x3e, 0x3, 0x3e, 0x3, 0x3e, - 0x3, 0x3e, 0x3, 0x3e, 0x3, 0x3e, 0x5, 0x3e, 0x4d4, 0xa, 0x3e, 0x3, 0x3f, + 0x3, 0x3e, 0x3, 0x3e, 0x3, 0x3e, 0x5, 0x3e, 0x4e4, 0xa, 0x3e, 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x40, 0x3, 0x40, 0x3, 0x40, 0x3, 0x40, 0x3, 0x41, 0x3, 0x41, 0x3, 0x41, 0x3, 0x41, 0x3, 0x42, 0x3, 0x42, 0x3, 0x42, 0x3, 0x42, 0x3, 0x42, 0x3, 0x43, 0x3, 0x43, 0x3, 0x43, 0x3, 0x43, 0x5, - 0x43, 0x4ea, 0xa, 0x43, 0x3, 0x44, 0x3, 0x44, 0x3, 0x44, 0x3, 0x45, - 0x3, 0x45, 0x3, 0x45, 0x5, 0x45, 0x4f2, 0xa, 0x45, 0x3, 0x45, 0x5, 0x45, - 0x4f5, 0xa, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x5, 0x45, - 0x4fb, 0xa, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, - 0x3, 0x45, 0x5, 0x45, 0x503, 0xa, 0x45, 0x3, 0x45, 0x5, 0x45, 0x506, - 0xa, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x7, 0x45, 0x50c, - 0xa, 0x45, 0xc, 0x45, 0xe, 0x45, 0x50f, 0xb, 0x45, 0x3, 0x46, 0x5, 0x46, - 0x512, 0xa, 0x46, 0x3, 0x46, 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, 0x517, - 0xa, 0x46, 0x3, 0x46, 0x5, 0x46, 0x51a, 0xa, 0x46, 0x3, 0x46, 0x5, 0x46, - 0x51d, 0xa, 0x46, 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, 0x521, 0xa, 0x46, - 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, 0x525, 0xa, 0x46, 0x3, 0x46, 0x5, 0x46, - 0x528, 0xa, 0x46, 0x5, 0x46, 0x52a, 0xa, 0x46, 0x3, 0x46, 0x5, 0x46, + 0x43, 0x4fa, 0xa, 0x43, 0x3, 0x44, 0x3, 0x44, 0x3, 0x44, 0x3, 0x45, + 0x3, 0x45, 0x3, 0x45, 0x5, 0x45, 0x502, 0xa, 0x45, 0x3, 0x45, 0x5, 0x45, + 0x505, 0xa, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x5, 0x45, + 0x50b, 0xa, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, + 0x3, 0x45, 0x5, 0x45, 0x513, 0xa, 0x45, 0x3, 0x45, 0x5, 0x45, 0x516, + 0xa, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x7, 0x45, 0x51c, + 0xa, 0x45, 0xc, 0x45, 0xe, 0x45, 0x51f, 0xb, 0x45, 0x3, 0x46, 0x5, 0x46, + 0x522, 0xa, 0x46, 0x3, 0x46, 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, 0x527, + 0xa, 0x46, 0x3, 0x46, 0x5, 0x46, 0x52a, 0xa, 0x46, 0x3, 0x46, 0x5, 0x46, 0x52d, 0xa, 0x46, 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, 0x531, 0xa, 0x46, 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, 0x535, 0xa, 0x46, 0x3, 0x46, 0x5, 0x46, - 0x538, 0xa, 0x46, 0x5, 0x46, 0x53a, 0xa, 0x46, 0x5, 0x46, 0x53c, 0xa, - 0x46, 0x3, 0x47, 0x5, 0x47, 0x53f, 0xa, 0x47, 0x3, 0x47, 0x3, 0x47, - 0x3, 0x47, 0x5, 0x47, 0x544, 0xa, 0x47, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, + 0x538, 0xa, 0x46, 0x5, 0x46, 0x53a, 0xa, 0x46, 0x3, 0x46, 0x5, 0x46, + 0x53d, 0xa, 0x46, 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, 0x541, 0xa, 0x46, + 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, 0x545, 0xa, 0x46, 0x3, 0x46, 0x5, 0x46, + 0x548, 0xa, 0x46, 0x5, 0x46, 0x54a, 0xa, 0x46, 0x5, 0x46, 0x54c, 0xa, + 0x46, 0x3, 0x47, 0x5, 0x47, 0x54f, 0xa, 0x47, 0x3, 0x47, 0x3, 0x47, + 0x3, 0x47, 0x5, 0x47, 0x554, 0xa, 0x47, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x5, - 0x48, 0x54f, 0xa, 0x48, 0x3, 0x49, 0x3, 0x49, 0x3, 0x49, 0x3, 0x49, - 0x5, 0x49, 0x555, 0xa, 0x49, 0x3, 0x4a, 0x3, 0x4a, 0x3, 0x4a, 0x5, 0x4a, - 0x55a, 0xa, 0x4a, 0x3, 0x4b, 0x3, 0x4b, 0x3, 0x4b, 0x7, 0x4b, 0x55f, - 0xa, 0x4b, 0xc, 0x4b, 0xe, 0x4b, 0x562, 0xb, 0x4b, 0x3, 0x4c, 0x3, 0x4c, - 0x5, 0x4c, 0x566, 0xa, 0x4c, 0x3, 0x4c, 0x3, 0x4c, 0x5, 0x4c, 0x56a, - 0xa, 0x4c, 0x3, 0x4c, 0x3, 0x4c, 0x5, 0x4c, 0x56e, 0xa, 0x4c, 0x3, 0x4d, - 0x3, 0x4d, 0x3, 0x4d, 0x5, 0x4d, 0x573, 0xa, 0x4d, 0x3, 0x4e, 0x3, 0x4e, - 0x3, 0x4e, 0x7, 0x4e, 0x578, 0xa, 0x4e, 0xc, 0x4e, 0xe, 0x4e, 0x57b, + 0x48, 0x55f, 0xa, 0x48, 0x3, 0x49, 0x3, 0x49, 0x3, 0x49, 0x3, 0x49, + 0x5, 0x49, 0x565, 0xa, 0x49, 0x3, 0x4a, 0x3, 0x4a, 0x3, 0x4a, 0x5, 0x4a, + 0x56a, 0xa, 0x4a, 0x3, 0x4b, 0x3, 0x4b, 0x3, 0x4b, 0x7, 0x4b, 0x56f, + 0xa, 0x4b, 0xc, 0x4b, 0xe, 0x4b, 0x572, 0xb, 0x4b, 0x3, 0x4c, 0x3, 0x4c, + 0x5, 0x4c, 0x576, 0xa, 0x4c, 0x3, 0x4c, 0x3, 0x4c, 0x5, 0x4c, 0x57a, + 0xa, 0x4c, 0x3, 0x4c, 0x3, 0x4c, 0x5, 0x4c, 0x57e, 0xa, 0x4c, 0x3, 0x4d, + 0x3, 0x4d, 0x3, 0x4d, 0x5, 0x4d, 0x583, 0xa, 0x4d, 0x3, 0x4e, 0x3, 0x4e, + 0x3, 0x4e, 0x7, 0x4e, 0x588, 0xa, 0x4e, 0xc, 0x4e, 0xe, 0x4e, 0x58b, 0xb, 0x4e, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x5, - 0x51, 0x58f, 0xa, 0x51, 0x3, 0x51, 0x5, 0x51, 0x592, 0xa, 0x51, 0x3, + 0x51, 0x59f, 0xa, 0x51, 0x3, 0x51, 0x5, 0x51, 0x5a2, 0xa, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, - 0x5, 0x51, 0x59b, 0xa, 0x51, 0x3, 0x51, 0x3, 0x51, 0x5, 0x51, 0x59f, - 0xa, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x5, 0x51, 0x5a4, 0xa, 0x51, - 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x5, 0x51, 0x5a9, 0xa, 0x51, 0x3, 0x51, - 0x5, 0x51, 0x5ac, 0xa, 0x51, 0x5, 0x51, 0x5ae, 0xa, 0x51, 0x3, 0x52, + 0x5, 0x51, 0x5ab, 0xa, 0x51, 0x3, 0x51, 0x3, 0x51, 0x5, 0x51, 0x5af, + 0xa, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x5, 0x51, 0x5b4, 0xa, 0x51, + 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x5, 0x51, 0x5b9, 0xa, 0x51, 0x3, 0x51, + 0x5, 0x51, 0x5bc, 0xa, 0x51, 0x5, 0x51, 0x5be, 0xa, 0x51, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x5, - 0x52, 0x5c4, 0xa, 0x52, 0x3, 0x52, 0x5, 0x52, 0x5c7, 0xa, 0x52, 0x3, + 0x52, 0x5d4, 0xa, 0x52, 0x3, 0x52, 0x5, 0x52, 0x5d7, 0xa, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, - 0x3, 0x52, 0x3, 0x52, 0x5, 0x52, 0x5d2, 0xa, 0x52, 0x3, 0x53, 0x3, 0x53, - 0x5, 0x53, 0x5d6, 0xa, 0x53, 0x3, 0x53, 0x5, 0x53, 0x5d9, 0xa, 0x53, - 0x3, 0x53, 0x3, 0x53, 0x5, 0x53, 0x5dd, 0xa, 0x53, 0x3, 0x53, 0x3, 0x53, - 0x5, 0x53, 0x5e1, 0xa, 0x53, 0x3, 0x54, 0x3, 0x54, 0x3, 0x54, 0x3, 0x55, - 0x3, 0x55, 0x3, 0x55, 0x5, 0x55, 0x5e9, 0xa, 0x55, 0x3, 0x55, 0x3, 0x55, - 0x5, 0x55, 0x5ed, 0xa, 0x55, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x7, 0x56, 0x5f8, - 0xa, 0x56, 0xc, 0x56, 0xe, 0x56, 0x5fb, 0xb, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x7, 0x56, 0x604, - 0xa, 0x56, 0xc, 0x56, 0xe, 0x56, 0x607, 0xb, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x7, 0x56, 0x610, - 0xa, 0x56, 0xc, 0x56, 0xe, 0x56, 0x613, 0xb, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, 0x61a, 0xa, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x5, 0x56, 0x61e, 0xa, 0x56, 0x3, 0x57, 0x3, 0x57, 0x3, 0x57, - 0x7, 0x57, 0x623, 0xa, 0x57, 0xc, 0x57, 0xe, 0x57, 0x626, 0xb, 0x57, - 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, 0x5, 0x58, 0x62b, 0xa, 0x58, 0x3, 0x58, - 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, 0x5, 0x58, 0x633, - 0xa, 0x58, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x638, 0xa, 0x59, - 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x6, 0x59, 0x63f, - 0xa, 0x59, 0xd, 0x59, 0xe, 0x59, 0x640, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, - 0x645, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, + 0x3, 0x52, 0x3, 0x52, 0x5, 0x52, 0x5e2, 0xa, 0x52, 0x3, 0x53, 0x3, 0x53, + 0x5, 0x53, 0x5e6, 0xa, 0x53, 0x3, 0x53, 0x5, 0x53, 0x5e9, 0xa, 0x53, + 0x3, 0x53, 0x3, 0x53, 0x5, 0x53, 0x5ed, 0xa, 0x53, 0x3, 0x53, 0x3, 0x53, + 0x5, 0x53, 0x5f1, 0xa, 0x53, 0x3, 0x54, 0x3, 0x54, 0x3, 0x54, 0x3, 0x55, + 0x3, 0x55, 0x3, 0x55, 0x5, 0x55, 0x5f9, 0xa, 0x55, 0x3, 0x55, 0x3, 0x55, + 0x5, 0x55, 0x5fd, 0xa, 0x55, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, + 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x7, 0x56, 0x608, + 0xa, 0x56, 0xc, 0x56, 0xe, 0x56, 0x60b, 0xb, 0x56, 0x3, 0x56, 0x3, 0x56, + 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x7, 0x56, 0x614, + 0xa, 0x56, 0xc, 0x56, 0xe, 0x56, 0x617, 0xb, 0x56, 0x3, 0x56, 0x3, 0x56, + 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x7, 0x56, 0x620, + 0xa, 0x56, 0xc, 0x56, 0xe, 0x56, 0x623, 0xb, 0x56, 0x3, 0x56, 0x3, 0x56, + 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, 0x62a, 0xa, 0x56, 0x3, 0x56, + 0x3, 0x56, 0x5, 0x56, 0x62e, 0xa, 0x56, 0x3, 0x57, 0x3, 0x57, 0x3, 0x57, + 0x7, 0x57, 0x633, 0xa, 0x57, 0xc, 0x57, 0xe, 0x57, 0x636, 0xb, 0x57, + 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, 0x5, 0x58, 0x63b, 0xa, 0x58, 0x3, 0x58, + 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, 0x5, 0x58, 0x643, + 0xa, 0x58, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x648, 0xa, 0x59, + 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x6, 0x59, 0x64f, + 0xa, 0x59, 0xd, 0x59, 0xe, 0x59, 0x650, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, + 0x655, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, - 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x664, + 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x674, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, - 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x675, 0xa, 0x59, 0x3, 0x59, - 0x5, 0x59, 0x678, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x67c, - 0xa, 0x59, 0x3, 0x59, 0x5, 0x59, 0x67f, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, + 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x685, 0xa, 0x59, 0x3, 0x59, + 0x5, 0x59, 0x688, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x68c, + 0xa, 0x59, 0x3, 0x59, 0x5, 0x59, 0x68f, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, - 0x59, 0x3, 0x59, 0x5, 0x59, 0x68b, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, + 0x59, 0x3, 0x59, 0x5, 0x59, 0x69b, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, - 0x5, 0x59, 0x69c, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6a0, + 0x5, 0x59, 0x6ac, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6b0, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, - 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6b1, 0xa, 0x59, 0x3, 0x59, - 0x5, 0x59, 0x6b4, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6b8, - 0xa, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6bb, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, + 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6c1, 0xa, 0x59, 0x3, 0x59, + 0x5, 0x59, 0x6c4, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6c8, + 0xa, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6cb, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, - 0x59, 0x5, 0x59, 0x6c6, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, + 0x59, 0x5, 0x59, 0x6d6, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, - 0x59, 0x6de, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, - 0x3, 0x59, 0x5, 0x59, 0x6e5, 0xa, 0x59, 0x7, 0x59, 0x6e7, 0xa, 0x59, - 0xc, 0x59, 0xe, 0x59, 0x6ea, 0xb, 0x59, 0x3, 0x5a, 0x3, 0x5a, 0x3, 0x5a, - 0x7, 0x5a, 0x6ef, 0xa, 0x5a, 0xc, 0x5a, 0xe, 0x5a, 0x6f2, 0xb, 0x5a, - 0x3, 0x5b, 0x3, 0x5b, 0x5, 0x5b, 0x6f6, 0xa, 0x5b, 0x3, 0x5c, 0x3, 0x5c, - 0x3, 0x5c, 0x3, 0x5c, 0x7, 0x5c, 0x6fc, 0xa, 0x5c, 0xc, 0x5c, 0xe, 0x5c, - 0x6ff, 0xb, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, - 0x7, 0x5c, 0x706, 0xa, 0x5c, 0xc, 0x5c, 0xe, 0x5c, 0x709, 0xb, 0x5c, - 0x5, 0x5c, 0x70b, 0xa, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5d, - 0x3, 0x5d, 0x3, 0x5d, 0x5, 0x5d, 0x713, 0xa, 0x5d, 0x3, 0x5d, 0x3, 0x5d, - 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x5, 0x5e, 0x71a, 0xa, 0x5e, 0x3, 0x5f, + 0x59, 0x6ee, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, + 0x3, 0x59, 0x5, 0x59, 0x6f5, 0xa, 0x59, 0x7, 0x59, 0x6f7, 0xa, 0x59, + 0xc, 0x59, 0xe, 0x59, 0x6fa, 0xb, 0x59, 0x3, 0x5a, 0x3, 0x5a, 0x3, 0x5a, + 0x7, 0x5a, 0x6ff, 0xa, 0x5a, 0xc, 0x5a, 0xe, 0x5a, 0x702, 0xb, 0x5a, + 0x3, 0x5b, 0x3, 0x5b, 0x5, 0x5b, 0x706, 0xa, 0x5b, 0x3, 0x5c, 0x3, 0x5c, + 0x3, 0x5c, 0x3, 0x5c, 0x7, 0x5c, 0x70c, 0xa, 0x5c, 0xc, 0x5c, 0xe, 0x5c, + 0x70f, 0xb, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, + 0x7, 0x5c, 0x716, 0xa, 0x5c, 0xc, 0x5c, 0xe, 0x5c, 0x719, 0xb, 0x5c, + 0x5, 0x5c, 0x71b, 0xa, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5d, + 0x3, 0x5d, 0x3, 0x5d, 0x5, 0x5d, 0x723, 0xa, 0x5d, 0x3, 0x5d, 0x3, 0x5d, + 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x5, 0x5e, 0x72a, 0xa, 0x5e, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x5, - 0x5f, 0x723, 0xa, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, - 0x5, 0x5f, 0x729, 0xa, 0x5f, 0x7, 0x5f, 0x72b, 0xa, 0x5f, 0xc, 0x5f, - 0xe, 0x5f, 0x72e, 0xb, 0x5f, 0x3, 0x60, 0x3, 0x60, 0x3, 0x60, 0x5, 0x60, - 0x733, 0xa, 0x60, 0x3, 0x60, 0x3, 0x60, 0x3, 0x61, 0x3, 0x61, 0x3, 0x61, - 0x5, 0x61, 0x73a, 0xa, 0x61, 0x3, 0x61, 0x3, 0x61, 0x3, 0x62, 0x3, 0x62, - 0x3, 0x62, 0x7, 0x62, 0x741, 0xa, 0x62, 0xc, 0x62, 0xe, 0x62, 0x744, - 0xb, 0x62, 0x3, 0x63, 0x3, 0x63, 0x3, 0x63, 0x5, 0x63, 0x749, 0xa, 0x63, + 0x5f, 0x733, 0xa, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, + 0x5, 0x5f, 0x739, 0xa, 0x5f, 0x7, 0x5f, 0x73b, 0xa, 0x5f, 0xc, 0x5f, + 0xe, 0x5f, 0x73e, 0xb, 0x5f, 0x3, 0x60, 0x3, 0x60, 0x3, 0x60, 0x5, 0x60, + 0x743, 0xa, 0x60, 0x3, 0x60, 0x3, 0x60, 0x3, 0x61, 0x3, 0x61, 0x3, 0x61, + 0x5, 0x61, 0x74a, 0xa, 0x61, 0x3, 0x61, 0x3, 0x61, 0x3, 0x62, 0x3, 0x62, + 0x3, 0x62, 0x7, 0x62, 0x751, 0xa, 0x62, 0xc, 0x62, 0xe, 0x62, 0x754, + 0xb, 0x62, 0x3, 0x63, 0x3, 0x63, 0x3, 0x63, 0x5, 0x63, 0x759, 0xa, 0x63, 0x3, 0x64, 0x3, 0x64, 0x3, 0x65, 0x3, 0x65, 0x3, 0x65, 0x3, 0x65, 0x3, - 0x65, 0x3, 0x65, 0x5, 0x65, 0x753, 0xa, 0x65, 0x5, 0x65, 0x755, 0xa, - 0x65, 0x3, 0x66, 0x5, 0x66, 0x758, 0xa, 0x66, 0x3, 0x66, 0x3, 0x66, - 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, 0x5, 0x66, 0x760, 0xa, 0x66, - 0x3, 0x67, 0x3, 0x67, 0x3, 0x67, 0x5, 0x67, 0x765, 0xa, 0x67, 0x3, 0x68, + 0x65, 0x3, 0x65, 0x5, 0x65, 0x763, 0xa, 0x65, 0x5, 0x65, 0x765, 0xa, + 0x65, 0x3, 0x66, 0x5, 0x66, 0x768, 0xa, 0x66, 0x3, 0x66, 0x3, 0x66, + 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, 0x5, 0x66, 0x770, 0xa, 0x66, + 0x3, 0x67, 0x3, 0x67, 0x3, 0x67, 0x5, 0x67, 0x775, 0xa, 0x67, 0x3, 0x68, 0x3, 0x68, 0x3, 0x69, 0x3, 0x69, 0x3, 0x6a, 0x3, 0x6a, 0x3, 0x6b, 0x3, - 0x6b, 0x5, 0x6b, 0x76f, 0xa, 0x6b, 0x3, 0x6c, 0x3, 0x6c, 0x3, 0x6c, - 0x5, 0x6c, 0x774, 0xa, 0x6c, 0x3, 0x6d, 0x3, 0x6d, 0x5, 0x6d, 0x778, + 0x6b, 0x5, 0x6b, 0x77f, 0xa, 0x6b, 0x3, 0x6c, 0x3, 0x6c, 0x3, 0x6c, + 0x5, 0x6c, 0x784, 0xa, 0x6c, 0x3, 0x6d, 0x3, 0x6d, 0x5, 0x6d, 0x788, 0xa, 0x6d, 0x3, 0x6e, 0x3, 0x6e, 0x3, 0x6e, 0x3, 0x6e, 0x3, 0x6e, 0x2, 0x5, 0x88, 0xb0, 0xbc, 0x6f, 0x2, 0x4, 0x6, 0x8, 0xa, 0xc, 0xe, 0x10, 0x12, 0x14, 0x16, 0x18, 0x1a, 0x1c, 0x1e, 0x20, 0x22, 0x24, 0x26, 0x28, @@ -19094,57 +19160,57 @@ ClickHouseParser::Initializer::Initializer() { 0xb7, 0xe, 0x2, 0x4, 0x24, 0x26, 0x4a, 0x4c, 0x50, 0x52, 0x6a, 0x6c, 0x6c, 0x6e, 0x6f, 0x71, 0x72, 0x74, 0x7f, 0x82, 0x8b, 0x8d, 0xb2, 0xb4, 0xb6, 0xb8, 0xb9, 0x6, 0x2, 0x24, 0x24, 0x3e, 0x3e, 0x4c, 0x4c, 0x5a, - 0x5a, 0x2, 0x88f, 0x2, 0xea, 0x3, 0x2, 0x2, 0x2, 0x4, 0xfe, 0x3, 0x2, + 0x5a, 0x2, 0x8a5, 0x2, 0xea, 0x3, 0x2, 0x2, 0x2, 0x4, 0xfe, 0x3, 0x2, 0x2, 0x2, 0x6, 0x100, 0x3, 0x2, 0x2, 0x2, 0x8, 0x1e4, 0x3, 0x2, 0x2, 0x2, 0xa, 0x1e6, 0x3, 0x2, 0x2, 0x2, 0xc, 0x1ee, 0x3, 0x2, 0x2, 0x2, 0xe, 0x1f2, 0x3, 0x2, 0x2, 0x2, 0x10, 0x1f9, 0x3, 0x2, 0x2, 0x2, 0x12, - 0x1fb, 0x3, 0x2, 0x2, 0x2, 0x14, 0x201, 0x3, 0x2, 0x2, 0x2, 0x16, 0x290, - 0x3, 0x2, 0x2, 0x2, 0x18, 0x292, 0x3, 0x2, 0x2, 0x2, 0x1a, 0x29d, 0x3, - 0x2, 0x2, 0x2, 0x1c, 0x2b8, 0x3, 0x2, 0x2, 0x2, 0x1e, 0x2d3, 0x3, 0x2, - 0x2, 0x2, 0x20, 0x2d7, 0x3, 0x2, 0x2, 0x2, 0x22, 0x2e0, 0x3, 0x2, 0x2, - 0x2, 0x24, 0x2ed, 0x3, 0x2, 0x2, 0x2, 0x26, 0x2fc, 0x3, 0x2, 0x2, 0x2, - 0x28, 0x309, 0x3, 0x2, 0x2, 0x2, 0x2a, 0x319, 0x3, 0x2, 0x2, 0x2, 0x2c, - 0x31e, 0x3, 0x2, 0x2, 0x2, 0x2e, 0x324, 0x3, 0x2, 0x2, 0x2, 0x30, 0x327, - 0x3, 0x2, 0x2, 0x2, 0x32, 0x32a, 0x3, 0x2, 0x2, 0x2, 0x34, 0x33c, 0x3, - 0x2, 0x2, 0x2, 0x36, 0x33e, 0x3, 0x2, 0x2, 0x2, 0x38, 0x35c, 0x3, 0x2, - 0x2, 0x2, 0x3a, 0x360, 0x3, 0x2, 0x2, 0x2, 0x3c, 0x364, 0x3, 0x2, 0x2, - 0x2, 0x3e, 0x368, 0x3, 0x2, 0x2, 0x2, 0x40, 0x371, 0x3, 0x2, 0x2, 0x2, - 0x42, 0x387, 0x3, 0x2, 0x2, 0x2, 0x44, 0x3a9, 0x3, 0x2, 0x2, 0x2, 0x46, - 0x3ab, 0x3, 0x2, 0x2, 0x2, 0x48, 0x3ae, 0x3, 0x2, 0x2, 0x2, 0x4a, 0x3b5, - 0x3, 0x2, 0x2, 0x2, 0x4c, 0x3b8, 0x3, 0x2, 0x2, 0x2, 0x4e, 0x3c4, 0x3, - 0x2, 0x2, 0x2, 0x50, 0x3cc, 0x3, 0x2, 0x2, 0x2, 0x52, 0x3d6, 0x3, 0x2, - 0x2, 0x2, 0x54, 0x3fb, 0x3, 0x2, 0x2, 0x2, 0x56, 0x40a, 0x3, 0x2, 0x2, - 0x2, 0x58, 0x412, 0x3, 0x2, 0x2, 0x2, 0x5a, 0x414, 0x3, 0x2, 0x2, 0x2, - 0x5c, 0x423, 0x3, 0x2, 0x2, 0x2, 0x5e, 0x437, 0x3, 0x2, 0x2, 0x2, 0x60, - 0x439, 0x3, 0x2, 0x2, 0x2, 0x62, 0x442, 0x3, 0x2, 0x2, 0x2, 0x64, 0x451, - 0x3, 0x2, 0x2, 0x2, 0x66, 0x463, 0x3, 0x2, 0x2, 0x2, 0x68, 0x471, 0x3, - 0x2, 0x2, 0x2, 0x6a, 0x47f, 0x3, 0x2, 0x2, 0x2, 0x6c, 0x482, 0x3, 0x2, - 0x2, 0x2, 0x6e, 0x4b2, 0x3, 0x2, 0x2, 0x2, 0x70, 0x4b5, 0x3, 0x2, 0x2, - 0x2, 0x72, 0x4bb, 0x3, 0x2, 0x2, 0x2, 0x74, 0x4bf, 0x3, 0x2, 0x2, 0x2, - 0x76, 0x4c5, 0x3, 0x2, 0x2, 0x2, 0x78, 0x4c8, 0x3, 0x2, 0x2, 0x2, 0x7a, - 0x4cb, 0x3, 0x2, 0x2, 0x2, 0x7c, 0x4d5, 0x3, 0x2, 0x2, 0x2, 0x7e, 0x4d8, - 0x3, 0x2, 0x2, 0x2, 0x80, 0x4dc, 0x3, 0x2, 0x2, 0x2, 0x82, 0x4e0, 0x3, - 0x2, 0x2, 0x2, 0x84, 0x4e5, 0x3, 0x2, 0x2, 0x2, 0x86, 0x4eb, 0x3, 0x2, - 0x2, 0x2, 0x88, 0x4fa, 0x3, 0x2, 0x2, 0x2, 0x8a, 0x53b, 0x3, 0x2, 0x2, - 0x2, 0x8c, 0x543, 0x3, 0x2, 0x2, 0x2, 0x8e, 0x54e, 0x3, 0x2, 0x2, 0x2, - 0x90, 0x550, 0x3, 0x2, 0x2, 0x2, 0x92, 0x556, 0x3, 0x2, 0x2, 0x2, 0x94, - 0x55b, 0x3, 0x2, 0x2, 0x2, 0x96, 0x563, 0x3, 0x2, 0x2, 0x2, 0x98, 0x56f, - 0x3, 0x2, 0x2, 0x2, 0x9a, 0x574, 0x3, 0x2, 0x2, 0x2, 0x9c, 0x57c, 0x3, - 0x2, 0x2, 0x2, 0x9e, 0x580, 0x3, 0x2, 0x2, 0x2, 0xa0, 0x5ad, 0x3, 0x2, - 0x2, 0x2, 0xa2, 0x5d1, 0x3, 0x2, 0x2, 0x2, 0xa4, 0x5d3, 0x3, 0x2, 0x2, - 0x2, 0xa6, 0x5e2, 0x3, 0x2, 0x2, 0x2, 0xa8, 0x5e5, 0x3, 0x2, 0x2, 0x2, - 0xaa, 0x61d, 0x3, 0x2, 0x2, 0x2, 0xac, 0x61f, 0x3, 0x2, 0x2, 0x2, 0xae, - 0x632, 0x3, 0x2, 0x2, 0x2, 0xb0, 0x69f, 0x3, 0x2, 0x2, 0x2, 0xb2, 0x6eb, - 0x3, 0x2, 0x2, 0x2, 0xb4, 0x6f5, 0x3, 0x2, 0x2, 0x2, 0xb6, 0x70a, 0x3, - 0x2, 0x2, 0x2, 0xb8, 0x712, 0x3, 0x2, 0x2, 0x2, 0xba, 0x716, 0x3, 0x2, - 0x2, 0x2, 0xbc, 0x722, 0x3, 0x2, 0x2, 0x2, 0xbe, 0x72f, 0x3, 0x2, 0x2, - 0x2, 0xc0, 0x739, 0x3, 0x2, 0x2, 0x2, 0xc2, 0x73d, 0x3, 0x2, 0x2, 0x2, - 0xc4, 0x748, 0x3, 0x2, 0x2, 0x2, 0xc6, 0x74a, 0x3, 0x2, 0x2, 0x2, 0xc8, - 0x754, 0x3, 0x2, 0x2, 0x2, 0xca, 0x757, 0x3, 0x2, 0x2, 0x2, 0xcc, 0x764, - 0x3, 0x2, 0x2, 0x2, 0xce, 0x766, 0x3, 0x2, 0x2, 0x2, 0xd0, 0x768, 0x3, - 0x2, 0x2, 0x2, 0xd2, 0x76a, 0x3, 0x2, 0x2, 0x2, 0xd4, 0x76e, 0x3, 0x2, - 0x2, 0x2, 0xd6, 0x773, 0x3, 0x2, 0x2, 0x2, 0xd8, 0x777, 0x3, 0x2, 0x2, - 0x2, 0xda, 0x779, 0x3, 0x2, 0x2, 0x2, 0xdc, 0xe0, 0x5, 0x4, 0x3, 0x2, + 0x1fb, 0x3, 0x2, 0x2, 0x2, 0x14, 0x201, 0x3, 0x2, 0x2, 0x2, 0x16, 0x2a0, + 0x3, 0x2, 0x2, 0x2, 0x18, 0x2a2, 0x3, 0x2, 0x2, 0x2, 0x1a, 0x2ad, 0x3, + 0x2, 0x2, 0x2, 0x1c, 0x2c8, 0x3, 0x2, 0x2, 0x2, 0x1e, 0x2e3, 0x3, 0x2, + 0x2, 0x2, 0x20, 0x2e7, 0x3, 0x2, 0x2, 0x2, 0x22, 0x2f0, 0x3, 0x2, 0x2, + 0x2, 0x24, 0x2fd, 0x3, 0x2, 0x2, 0x2, 0x26, 0x30c, 0x3, 0x2, 0x2, 0x2, + 0x28, 0x319, 0x3, 0x2, 0x2, 0x2, 0x2a, 0x329, 0x3, 0x2, 0x2, 0x2, 0x2c, + 0x32e, 0x3, 0x2, 0x2, 0x2, 0x2e, 0x334, 0x3, 0x2, 0x2, 0x2, 0x30, 0x337, + 0x3, 0x2, 0x2, 0x2, 0x32, 0x33a, 0x3, 0x2, 0x2, 0x2, 0x34, 0x34c, 0x3, + 0x2, 0x2, 0x2, 0x36, 0x34e, 0x3, 0x2, 0x2, 0x2, 0x38, 0x36c, 0x3, 0x2, + 0x2, 0x2, 0x3a, 0x370, 0x3, 0x2, 0x2, 0x2, 0x3c, 0x374, 0x3, 0x2, 0x2, + 0x2, 0x3e, 0x378, 0x3, 0x2, 0x2, 0x2, 0x40, 0x381, 0x3, 0x2, 0x2, 0x2, + 0x42, 0x397, 0x3, 0x2, 0x2, 0x2, 0x44, 0x3b9, 0x3, 0x2, 0x2, 0x2, 0x46, + 0x3bb, 0x3, 0x2, 0x2, 0x2, 0x48, 0x3be, 0x3, 0x2, 0x2, 0x2, 0x4a, 0x3c5, + 0x3, 0x2, 0x2, 0x2, 0x4c, 0x3c8, 0x3, 0x2, 0x2, 0x2, 0x4e, 0x3d4, 0x3, + 0x2, 0x2, 0x2, 0x50, 0x3dc, 0x3, 0x2, 0x2, 0x2, 0x52, 0x3e6, 0x3, 0x2, + 0x2, 0x2, 0x54, 0x40b, 0x3, 0x2, 0x2, 0x2, 0x56, 0x41a, 0x3, 0x2, 0x2, + 0x2, 0x58, 0x422, 0x3, 0x2, 0x2, 0x2, 0x5a, 0x424, 0x3, 0x2, 0x2, 0x2, + 0x5c, 0x433, 0x3, 0x2, 0x2, 0x2, 0x5e, 0x447, 0x3, 0x2, 0x2, 0x2, 0x60, + 0x449, 0x3, 0x2, 0x2, 0x2, 0x62, 0x452, 0x3, 0x2, 0x2, 0x2, 0x64, 0x461, + 0x3, 0x2, 0x2, 0x2, 0x66, 0x473, 0x3, 0x2, 0x2, 0x2, 0x68, 0x481, 0x3, + 0x2, 0x2, 0x2, 0x6a, 0x48f, 0x3, 0x2, 0x2, 0x2, 0x6c, 0x492, 0x3, 0x2, + 0x2, 0x2, 0x6e, 0x4c2, 0x3, 0x2, 0x2, 0x2, 0x70, 0x4c5, 0x3, 0x2, 0x2, + 0x2, 0x72, 0x4cb, 0x3, 0x2, 0x2, 0x2, 0x74, 0x4cf, 0x3, 0x2, 0x2, 0x2, + 0x76, 0x4d5, 0x3, 0x2, 0x2, 0x2, 0x78, 0x4d8, 0x3, 0x2, 0x2, 0x2, 0x7a, + 0x4db, 0x3, 0x2, 0x2, 0x2, 0x7c, 0x4e5, 0x3, 0x2, 0x2, 0x2, 0x7e, 0x4e8, + 0x3, 0x2, 0x2, 0x2, 0x80, 0x4ec, 0x3, 0x2, 0x2, 0x2, 0x82, 0x4f0, 0x3, + 0x2, 0x2, 0x2, 0x84, 0x4f5, 0x3, 0x2, 0x2, 0x2, 0x86, 0x4fb, 0x3, 0x2, + 0x2, 0x2, 0x88, 0x50a, 0x3, 0x2, 0x2, 0x2, 0x8a, 0x54b, 0x3, 0x2, 0x2, + 0x2, 0x8c, 0x553, 0x3, 0x2, 0x2, 0x2, 0x8e, 0x55e, 0x3, 0x2, 0x2, 0x2, + 0x90, 0x560, 0x3, 0x2, 0x2, 0x2, 0x92, 0x566, 0x3, 0x2, 0x2, 0x2, 0x94, + 0x56b, 0x3, 0x2, 0x2, 0x2, 0x96, 0x573, 0x3, 0x2, 0x2, 0x2, 0x98, 0x57f, + 0x3, 0x2, 0x2, 0x2, 0x9a, 0x584, 0x3, 0x2, 0x2, 0x2, 0x9c, 0x58c, 0x3, + 0x2, 0x2, 0x2, 0x9e, 0x590, 0x3, 0x2, 0x2, 0x2, 0xa0, 0x5bd, 0x3, 0x2, + 0x2, 0x2, 0xa2, 0x5e1, 0x3, 0x2, 0x2, 0x2, 0xa4, 0x5e3, 0x3, 0x2, 0x2, + 0x2, 0xa6, 0x5f2, 0x3, 0x2, 0x2, 0x2, 0xa8, 0x5f5, 0x3, 0x2, 0x2, 0x2, + 0xaa, 0x62d, 0x3, 0x2, 0x2, 0x2, 0xac, 0x62f, 0x3, 0x2, 0x2, 0x2, 0xae, + 0x642, 0x3, 0x2, 0x2, 0x2, 0xb0, 0x6af, 0x3, 0x2, 0x2, 0x2, 0xb2, 0x6fb, + 0x3, 0x2, 0x2, 0x2, 0xb4, 0x705, 0x3, 0x2, 0x2, 0x2, 0xb6, 0x71a, 0x3, + 0x2, 0x2, 0x2, 0xb8, 0x722, 0x3, 0x2, 0x2, 0x2, 0xba, 0x726, 0x3, 0x2, + 0x2, 0x2, 0xbc, 0x732, 0x3, 0x2, 0x2, 0x2, 0xbe, 0x73f, 0x3, 0x2, 0x2, + 0x2, 0xc0, 0x749, 0x3, 0x2, 0x2, 0x2, 0xc2, 0x74d, 0x3, 0x2, 0x2, 0x2, + 0xc4, 0x758, 0x3, 0x2, 0x2, 0x2, 0xc6, 0x75a, 0x3, 0x2, 0x2, 0x2, 0xc8, + 0x764, 0x3, 0x2, 0x2, 0x2, 0xca, 0x767, 0x3, 0x2, 0x2, 0x2, 0xcc, 0x774, + 0x3, 0x2, 0x2, 0x2, 0xce, 0x776, 0x3, 0x2, 0x2, 0x2, 0xd0, 0x778, 0x3, + 0x2, 0x2, 0x2, 0xd2, 0x77a, 0x3, 0x2, 0x2, 0x2, 0xd4, 0x77e, 0x3, 0x2, + 0x2, 0x2, 0xd6, 0x783, 0x3, 0x2, 0x2, 0x2, 0xd8, 0x787, 0x3, 0x2, 0x2, + 0x2, 0xda, 0x789, 0x3, 0x2, 0x2, 0x2, 0xdc, 0xe0, 0x5, 0x4, 0x3, 0x2, 0xdd, 0xde, 0x7, 0x56, 0x2, 0x2, 0xde, 0xdf, 0x7, 0x7b, 0x2, 0x2, 0xdf, 0xe1, 0x7, 0xbf, 0x2, 0x2, 0xe0, 0xdd, 0x3, 0x2, 0x2, 0x2, 0xe0, 0xe1, 0x3, 0x2, 0x2, 0x2, 0xe1, 0xe4, 0x3, 0x2, 0x2, 0x2, 0xe2, 0xe3, 0x7, @@ -19341,870 +19407,881 @@ ClickHouseParser::Initializer::Initializer() { 0x2c, 0x17, 0x2, 0x210, 0x20f, 0x3, 0x2, 0x2, 0x2, 0x210, 0x211, 0x3, 0x2, 0x2, 0x2, 0x211, 0x213, 0x3, 0x2, 0x2, 0x2, 0x212, 0x214, 0x5, 0x40, 0x21, 0x2, 0x213, 0x212, 0x3, 0x2, 0x2, 0x2, 0x213, 0x214, 0x3, - 0x2, 0x2, 0x2, 0x214, 0x291, 0x3, 0x2, 0x2, 0x2, 0x215, 0x216, 0x9, - 0x3, 0x2, 0x2, 0x216, 0x21a, 0x7, 0x2f, 0x2, 0x2, 0x217, 0x218, 0x7, - 0x4d, 0x2, 0x2, 0x218, 0x219, 0x7, 0x72, 0x2, 0x2, 0x219, 0x21b, 0x7, - 0x38, 0x2, 0x2, 0x21a, 0x217, 0x3, 0x2, 0x2, 0x2, 0x21a, 0x21b, 0x3, - 0x2, 0x2, 0x2, 0x21b, 0x21c, 0x3, 0x2, 0x2, 0x2, 0x21c, 0x21e, 0x5, - 0xc0, 0x61, 0x2, 0x21d, 0x21f, 0x5, 0x2e, 0x18, 0x2, 0x21e, 0x21d, 0x3, - 0x2, 0x2, 0x2, 0x21e, 0x21f, 0x3, 0x2, 0x2, 0x2, 0x21f, 0x221, 0x3, - 0x2, 0x2, 0x2, 0x220, 0x222, 0x5, 0x2c, 0x17, 0x2, 0x221, 0x220, 0x3, - 0x2, 0x2, 0x2, 0x221, 0x222, 0x3, 0x2, 0x2, 0x2, 0x222, 0x223, 0x3, - 0x2, 0x2, 0x2, 0x223, 0x224, 0x5, 0x18, 0xd, 0x2, 0x224, 0x225, 0x5, - 0x1c, 0xf, 0x2, 0x225, 0x291, 0x3, 0x2, 0x2, 0x2, 0x226, 0x227, 0x9, - 0x3, 0x2, 0x2, 0x227, 0x228, 0x7, 0x63, 0x2, 0x2, 0x228, 0x22c, 0x7, - 0xb0, 0x2, 0x2, 0x229, 0x22a, 0x7, 0x4d, 0x2, 0x2, 0x22a, 0x22b, 0x7, - 0x72, 0x2, 0x2, 0x22b, 0x22d, 0x7, 0x38, 0x2, 0x2, 0x22c, 0x229, 0x3, - 0x2, 0x2, 0x2, 0x22c, 0x22d, 0x3, 0x2, 0x2, 0x2, 0x22d, 0x22e, 0x3, - 0x2, 0x2, 0x2, 0x22e, 0x230, 0x5, 0xc0, 0x61, 0x2, 0x22f, 0x231, 0x5, - 0x2e, 0x18, 0x2, 0x230, 0x22f, 0x3, 0x2, 0x2, 0x2, 0x230, 0x231, 0x3, - 0x2, 0x2, 0x2, 0x231, 0x233, 0x3, 0x2, 0x2, 0x2, 0x232, 0x234, 0x5, - 0x2c, 0x17, 0x2, 0x233, 0x232, 0x3, 0x2, 0x2, 0x2, 0x233, 0x234, 0x3, - 0x2, 0x2, 0x2, 0x234, 0x23a, 0x3, 0x2, 0x2, 0x2, 0x235, 0x236, 0x7, - 0xb6, 0x2, 0x2, 0x236, 0x238, 0x7, 0xa0, 0x2, 0x2, 0x237, 0x239, 0x7, - 0xbd, 0x2, 0x2, 0x238, 0x237, 0x3, 0x2, 0x2, 0x2, 0x238, 0x239, 0x3, - 0x2, 0x2, 0x2, 0x239, 0x23b, 0x3, 0x2, 0x2, 0x2, 0x23a, 0x235, 0x3, - 0x2, 0x2, 0x2, 0x23a, 0x23b, 0x3, 0x2, 0x2, 0x2, 0x23b, 0x23d, 0x3, - 0x2, 0x2, 0x2, 0x23c, 0x23e, 0x5, 0x30, 0x19, 0x2, 0x23d, 0x23c, 0x3, - 0x2, 0x2, 0x2, 0x23d, 0x23e, 0x3, 0x2, 0x2, 0x2, 0x23e, 0x240, 0x3, - 0x2, 0x2, 0x2, 0x23f, 0x241, 0x5, 0x34, 0x1b, 0x2, 0x240, 0x23f, 0x3, - 0x2, 0x2, 0x2, 0x240, 0x241, 0x3, 0x2, 0x2, 0x2, 0x241, 0x242, 0x3, - 0x2, 0x2, 0x2, 0x242, 0x243, 0x5, 0x32, 0x1a, 0x2, 0x243, 0x291, 0x3, - 0x2, 0x2, 0x2, 0x244, 0x245, 0x9, 0x3, 0x2, 0x2, 0x245, 0x246, 0x7, - 0x67, 0x2, 0x2, 0x246, 0x24a, 0x7, 0xb0, 0x2, 0x2, 0x247, 0x248, 0x7, - 0x4d, 0x2, 0x2, 0x248, 0x249, 0x7, 0x72, 0x2, 0x2, 0x249, 0x24b, 0x7, - 0x38, 0x2, 0x2, 0x24a, 0x247, 0x3, 0x2, 0x2, 0x2, 0x24a, 0x24b, 0x3, - 0x2, 0x2, 0x2, 0x24b, 0x24c, 0x3, 0x2, 0x2, 0x2, 0x24c, 0x24e, 0x5, - 0xc0, 0x61, 0x2, 0x24d, 0x24f, 0x5, 0x2e, 0x18, 0x2, 0x24e, 0x24d, 0x3, - 0x2, 0x2, 0x2, 0x24e, 0x24f, 0x3, 0x2, 0x2, 0x2, 0x24f, 0x251, 0x3, - 0x2, 0x2, 0x2, 0x250, 0x252, 0x5, 0x2c, 0x17, 0x2, 0x251, 0x250, 0x3, - 0x2, 0x2, 0x2, 0x251, 0x252, 0x3, 0x2, 0x2, 0x2, 0x252, 0x254, 0x3, - 0x2, 0x2, 0x2, 0x253, 0x255, 0x5, 0x34, 0x1b, 0x2, 0x254, 0x253, 0x3, - 0x2, 0x2, 0x2, 0x254, 0x255, 0x3, 0x2, 0x2, 0x2, 0x255, 0x25b, 0x3, - 0x2, 0x2, 0x2, 0x256, 0x25c, 0x5, 0x30, 0x19, 0x2, 0x257, 0x259, 0x5, - 0x36, 0x1c, 0x2, 0x258, 0x25a, 0x7, 0x7d, 0x2, 0x2, 0x259, 0x258, 0x3, - 0x2, 0x2, 0x2, 0x259, 0x25a, 0x3, 0x2, 0x2, 0x2, 0x25a, 0x25c, 0x3, - 0x2, 0x2, 0x2, 0x25b, 0x256, 0x3, 0x2, 0x2, 0x2, 0x25b, 0x257, 0x3, - 0x2, 0x2, 0x2, 0x25c, 0x25d, 0x3, 0x2, 0x2, 0x2, 0x25d, 0x25e, 0x5, - 0x32, 0x1a, 0x2, 0x25e, 0x291, 0x3, 0x2, 0x2, 0x2, 0x25f, 0x261, 0x9, - 0x3, 0x2, 0x2, 0x260, 0x262, 0x7, 0x9c, 0x2, 0x2, 0x261, 0x260, 0x3, - 0x2, 0x2, 0x2, 0x261, 0x262, 0x3, 0x2, 0x2, 0x2, 0x262, 0x263, 0x3, - 0x2, 0x2, 0x2, 0x263, 0x267, 0x7, 0x9a, 0x2, 0x2, 0x264, 0x265, 0x7, - 0x4d, 0x2, 0x2, 0x265, 0x266, 0x7, 0x72, 0x2, 0x2, 0x266, 0x268, 0x7, - 0x38, 0x2, 0x2, 0x267, 0x264, 0x3, 0x2, 0x2, 0x2, 0x267, 0x268, 0x3, - 0x2, 0x2, 0x2, 0x268, 0x269, 0x3, 0x2, 0x2, 0x2, 0x269, 0x26b, 0x5, - 0xc0, 0x61, 0x2, 0x26a, 0x26c, 0x5, 0x2e, 0x18, 0x2, 0x26b, 0x26a, 0x3, - 0x2, 0x2, 0x2, 0x26b, 0x26c, 0x3, 0x2, 0x2, 0x2, 0x26c, 0x26e, 0x3, - 0x2, 0x2, 0x2, 0x26d, 0x26f, 0x5, 0x2c, 0x17, 0x2, 0x26e, 0x26d, 0x3, - 0x2, 0x2, 0x2, 0x26e, 0x26f, 0x3, 0x2, 0x2, 0x2, 0x26f, 0x271, 0x3, - 0x2, 0x2, 0x2, 0x270, 0x272, 0x5, 0x34, 0x1b, 0x2, 0x271, 0x270, 0x3, - 0x2, 0x2, 0x2, 0x271, 0x272, 0x3, 0x2, 0x2, 0x2, 0x272, 0x274, 0x3, - 0x2, 0x2, 0x2, 0x273, 0x275, 0x5, 0x36, 0x1c, 0x2, 0x274, 0x273, 0x3, - 0x2, 0x2, 0x2, 0x274, 0x275, 0x3, 0x2, 0x2, 0x2, 0x275, 0x277, 0x3, - 0x2, 0x2, 0x2, 0x276, 0x278, 0x5, 0x32, 0x1a, 0x2, 0x277, 0x276, 0x3, - 0x2, 0x2, 0x2, 0x277, 0x278, 0x3, 0x2, 0x2, 0x2, 0x278, 0x291, 0x3, - 0x2, 0x2, 0x2, 0x279, 0x27c, 0x9, 0x3, 0x2, 0x2, 0x27a, 0x27b, 0x7, - 0x78, 0x2, 0x2, 0x27b, 0x27d, 0x7, 0x86, 0x2, 0x2, 0x27c, 0x27a, 0x3, - 0x2, 0x2, 0x2, 0x27c, 0x27d, 0x3, 0x2, 0x2, 0x2, 0x27d, 0x27e, 0x3, - 0x2, 0x2, 0x2, 0x27e, 0x282, 0x7, 0xb0, 0x2, 0x2, 0x27f, 0x280, 0x7, - 0x4d, 0x2, 0x2, 0x280, 0x281, 0x7, 0x72, 0x2, 0x2, 0x281, 0x283, 0x7, - 0x38, 0x2, 0x2, 0x282, 0x27f, 0x3, 0x2, 0x2, 0x2, 0x282, 0x283, 0x3, - 0x2, 0x2, 0x2, 0x283, 0x284, 0x3, 0x2, 0x2, 0x2, 0x284, 0x286, 0x5, - 0xc0, 0x61, 0x2, 0x285, 0x287, 0x5, 0x2e, 0x18, 0x2, 0x286, 0x285, 0x3, - 0x2, 0x2, 0x2, 0x286, 0x287, 0x3, 0x2, 0x2, 0x2, 0x287, 0x289, 0x3, - 0x2, 0x2, 0x2, 0x288, 0x28a, 0x5, 0x2c, 0x17, 0x2, 0x289, 0x288, 0x3, - 0x2, 0x2, 0x2, 0x289, 0x28a, 0x3, 0x2, 0x2, 0x2, 0x28a, 0x28c, 0x3, - 0x2, 0x2, 0x2, 0x28b, 0x28d, 0x5, 0x34, 0x1b, 0x2, 0x28c, 0x28b, 0x3, + 0x2, 0x2, 0x2, 0x214, 0x2a1, 0x3, 0x2, 0x2, 0x2, 0x215, 0x21d, 0x7, + 0x11, 0x2, 0x2, 0x216, 0x219, 0x7, 0x1f, 0x2, 0x2, 0x217, 0x218, 0x7, + 0x78, 0x2, 0x2, 0x218, 0x21a, 0x7, 0x86, 0x2, 0x2, 0x219, 0x217, 0x3, + 0x2, 0x2, 0x2, 0x219, 0x21a, 0x3, 0x2, 0x2, 0x2, 0x21a, 0x21d, 0x3, + 0x2, 0x2, 0x2, 0x21b, 0x21d, 0x7, 0x86, 0x2, 0x2, 0x21c, 0x215, 0x3, + 0x2, 0x2, 0x2, 0x21c, 0x216, 0x3, 0x2, 0x2, 0x2, 0x21c, 0x21b, 0x3, + 0x2, 0x2, 0x2, 0x21d, 0x21e, 0x3, 0x2, 0x2, 0x2, 0x21e, 0x222, 0x7, + 0x2f, 0x2, 0x2, 0x21f, 0x220, 0x7, 0x4d, 0x2, 0x2, 0x220, 0x221, 0x7, + 0x72, 0x2, 0x2, 0x221, 0x223, 0x7, 0x38, 0x2, 0x2, 0x222, 0x21f, 0x3, + 0x2, 0x2, 0x2, 0x222, 0x223, 0x3, 0x2, 0x2, 0x2, 0x223, 0x224, 0x3, + 0x2, 0x2, 0x2, 0x224, 0x226, 0x5, 0xc0, 0x61, 0x2, 0x225, 0x227, 0x5, + 0x2e, 0x18, 0x2, 0x226, 0x225, 0x3, 0x2, 0x2, 0x2, 0x226, 0x227, 0x3, + 0x2, 0x2, 0x2, 0x227, 0x229, 0x3, 0x2, 0x2, 0x2, 0x228, 0x22a, 0x5, + 0x2c, 0x17, 0x2, 0x229, 0x228, 0x3, 0x2, 0x2, 0x2, 0x229, 0x22a, 0x3, + 0x2, 0x2, 0x2, 0x22a, 0x22b, 0x3, 0x2, 0x2, 0x2, 0x22b, 0x22c, 0x5, + 0x18, 0xd, 0x2, 0x22c, 0x22d, 0x5, 0x1c, 0xf, 0x2, 0x22d, 0x2a1, 0x3, + 0x2, 0x2, 0x2, 0x22e, 0x22f, 0x9, 0x3, 0x2, 0x2, 0x22f, 0x230, 0x7, + 0x63, 0x2, 0x2, 0x230, 0x234, 0x7, 0xb0, 0x2, 0x2, 0x231, 0x232, 0x7, + 0x4d, 0x2, 0x2, 0x232, 0x233, 0x7, 0x72, 0x2, 0x2, 0x233, 0x235, 0x7, + 0x38, 0x2, 0x2, 0x234, 0x231, 0x3, 0x2, 0x2, 0x2, 0x234, 0x235, 0x3, + 0x2, 0x2, 0x2, 0x235, 0x236, 0x3, 0x2, 0x2, 0x2, 0x236, 0x238, 0x5, + 0xc0, 0x61, 0x2, 0x237, 0x239, 0x5, 0x2e, 0x18, 0x2, 0x238, 0x237, 0x3, + 0x2, 0x2, 0x2, 0x238, 0x239, 0x3, 0x2, 0x2, 0x2, 0x239, 0x23b, 0x3, + 0x2, 0x2, 0x2, 0x23a, 0x23c, 0x5, 0x2c, 0x17, 0x2, 0x23b, 0x23a, 0x3, + 0x2, 0x2, 0x2, 0x23b, 0x23c, 0x3, 0x2, 0x2, 0x2, 0x23c, 0x242, 0x3, + 0x2, 0x2, 0x2, 0x23d, 0x23e, 0x7, 0xb6, 0x2, 0x2, 0x23e, 0x240, 0x7, + 0xa0, 0x2, 0x2, 0x23f, 0x241, 0x7, 0xbd, 0x2, 0x2, 0x240, 0x23f, 0x3, + 0x2, 0x2, 0x2, 0x240, 0x241, 0x3, 0x2, 0x2, 0x2, 0x241, 0x243, 0x3, + 0x2, 0x2, 0x2, 0x242, 0x23d, 0x3, 0x2, 0x2, 0x2, 0x242, 0x243, 0x3, + 0x2, 0x2, 0x2, 0x243, 0x245, 0x3, 0x2, 0x2, 0x2, 0x244, 0x246, 0x5, + 0x30, 0x19, 0x2, 0x245, 0x244, 0x3, 0x2, 0x2, 0x2, 0x245, 0x246, 0x3, + 0x2, 0x2, 0x2, 0x246, 0x248, 0x3, 0x2, 0x2, 0x2, 0x247, 0x249, 0x5, + 0x34, 0x1b, 0x2, 0x248, 0x247, 0x3, 0x2, 0x2, 0x2, 0x248, 0x249, 0x3, + 0x2, 0x2, 0x2, 0x249, 0x24a, 0x3, 0x2, 0x2, 0x2, 0x24a, 0x24b, 0x5, + 0x32, 0x1a, 0x2, 0x24b, 0x2a1, 0x3, 0x2, 0x2, 0x2, 0x24c, 0x24d, 0x9, + 0x3, 0x2, 0x2, 0x24d, 0x24e, 0x7, 0x67, 0x2, 0x2, 0x24e, 0x252, 0x7, + 0xb0, 0x2, 0x2, 0x24f, 0x250, 0x7, 0x4d, 0x2, 0x2, 0x250, 0x251, 0x7, + 0x72, 0x2, 0x2, 0x251, 0x253, 0x7, 0x38, 0x2, 0x2, 0x252, 0x24f, 0x3, + 0x2, 0x2, 0x2, 0x252, 0x253, 0x3, 0x2, 0x2, 0x2, 0x253, 0x254, 0x3, + 0x2, 0x2, 0x2, 0x254, 0x256, 0x5, 0xc0, 0x61, 0x2, 0x255, 0x257, 0x5, + 0x2e, 0x18, 0x2, 0x256, 0x255, 0x3, 0x2, 0x2, 0x2, 0x256, 0x257, 0x3, + 0x2, 0x2, 0x2, 0x257, 0x259, 0x3, 0x2, 0x2, 0x2, 0x258, 0x25a, 0x5, + 0x2c, 0x17, 0x2, 0x259, 0x258, 0x3, 0x2, 0x2, 0x2, 0x259, 0x25a, 0x3, + 0x2, 0x2, 0x2, 0x25a, 0x25c, 0x3, 0x2, 0x2, 0x2, 0x25b, 0x25d, 0x5, + 0x34, 0x1b, 0x2, 0x25c, 0x25b, 0x3, 0x2, 0x2, 0x2, 0x25c, 0x25d, 0x3, + 0x2, 0x2, 0x2, 0x25d, 0x263, 0x3, 0x2, 0x2, 0x2, 0x25e, 0x264, 0x5, + 0x30, 0x19, 0x2, 0x25f, 0x261, 0x5, 0x36, 0x1c, 0x2, 0x260, 0x262, 0x7, + 0x7d, 0x2, 0x2, 0x261, 0x260, 0x3, 0x2, 0x2, 0x2, 0x261, 0x262, 0x3, + 0x2, 0x2, 0x2, 0x262, 0x264, 0x3, 0x2, 0x2, 0x2, 0x263, 0x25e, 0x3, + 0x2, 0x2, 0x2, 0x263, 0x25f, 0x3, 0x2, 0x2, 0x2, 0x264, 0x265, 0x3, + 0x2, 0x2, 0x2, 0x265, 0x266, 0x5, 0x32, 0x1a, 0x2, 0x266, 0x2a1, 0x3, + 0x2, 0x2, 0x2, 0x267, 0x26f, 0x7, 0x11, 0x2, 0x2, 0x268, 0x26b, 0x7, + 0x1f, 0x2, 0x2, 0x269, 0x26a, 0x7, 0x78, 0x2, 0x2, 0x26a, 0x26c, 0x7, + 0x86, 0x2, 0x2, 0x26b, 0x269, 0x3, 0x2, 0x2, 0x2, 0x26b, 0x26c, 0x3, + 0x2, 0x2, 0x2, 0x26c, 0x26f, 0x3, 0x2, 0x2, 0x2, 0x26d, 0x26f, 0x7, + 0x86, 0x2, 0x2, 0x26e, 0x267, 0x3, 0x2, 0x2, 0x2, 0x26e, 0x268, 0x3, + 0x2, 0x2, 0x2, 0x26e, 0x26d, 0x3, 0x2, 0x2, 0x2, 0x26f, 0x271, 0x3, + 0x2, 0x2, 0x2, 0x270, 0x272, 0x7, 0x9c, 0x2, 0x2, 0x271, 0x270, 0x3, + 0x2, 0x2, 0x2, 0x271, 0x272, 0x3, 0x2, 0x2, 0x2, 0x272, 0x273, 0x3, + 0x2, 0x2, 0x2, 0x273, 0x277, 0x7, 0x9a, 0x2, 0x2, 0x274, 0x275, 0x7, + 0x4d, 0x2, 0x2, 0x275, 0x276, 0x7, 0x72, 0x2, 0x2, 0x276, 0x278, 0x7, + 0x38, 0x2, 0x2, 0x277, 0x274, 0x3, 0x2, 0x2, 0x2, 0x277, 0x278, 0x3, + 0x2, 0x2, 0x2, 0x278, 0x279, 0x3, 0x2, 0x2, 0x2, 0x279, 0x27b, 0x5, + 0xc0, 0x61, 0x2, 0x27a, 0x27c, 0x5, 0x2e, 0x18, 0x2, 0x27b, 0x27a, 0x3, + 0x2, 0x2, 0x2, 0x27b, 0x27c, 0x3, 0x2, 0x2, 0x2, 0x27c, 0x27e, 0x3, + 0x2, 0x2, 0x2, 0x27d, 0x27f, 0x5, 0x2c, 0x17, 0x2, 0x27e, 0x27d, 0x3, + 0x2, 0x2, 0x2, 0x27e, 0x27f, 0x3, 0x2, 0x2, 0x2, 0x27f, 0x281, 0x3, + 0x2, 0x2, 0x2, 0x280, 0x282, 0x5, 0x34, 0x1b, 0x2, 0x281, 0x280, 0x3, + 0x2, 0x2, 0x2, 0x281, 0x282, 0x3, 0x2, 0x2, 0x2, 0x282, 0x284, 0x3, + 0x2, 0x2, 0x2, 0x283, 0x285, 0x5, 0x36, 0x1c, 0x2, 0x284, 0x283, 0x3, + 0x2, 0x2, 0x2, 0x284, 0x285, 0x3, 0x2, 0x2, 0x2, 0x285, 0x287, 0x3, + 0x2, 0x2, 0x2, 0x286, 0x288, 0x5, 0x32, 0x1a, 0x2, 0x287, 0x286, 0x3, + 0x2, 0x2, 0x2, 0x287, 0x288, 0x3, 0x2, 0x2, 0x2, 0x288, 0x2a1, 0x3, + 0x2, 0x2, 0x2, 0x289, 0x28c, 0x9, 0x3, 0x2, 0x2, 0x28a, 0x28b, 0x7, + 0x78, 0x2, 0x2, 0x28b, 0x28d, 0x7, 0x86, 0x2, 0x2, 0x28c, 0x28a, 0x3, 0x2, 0x2, 0x2, 0x28c, 0x28d, 0x3, 0x2, 0x2, 0x2, 0x28d, 0x28e, 0x3, - 0x2, 0x2, 0x2, 0x28e, 0x28f, 0x5, 0x32, 0x1a, 0x2, 0x28f, 0x291, 0x3, - 0x2, 0x2, 0x2, 0x290, 0x207, 0x3, 0x2, 0x2, 0x2, 0x290, 0x215, 0x3, - 0x2, 0x2, 0x2, 0x290, 0x226, 0x3, 0x2, 0x2, 0x2, 0x290, 0x244, 0x3, - 0x2, 0x2, 0x2, 0x290, 0x25f, 0x3, 0x2, 0x2, 0x2, 0x290, 0x279, 0x3, - 0x2, 0x2, 0x2, 0x291, 0x17, 0x3, 0x2, 0x2, 0x2, 0x292, 0x293, 0x7, 0xd0, - 0x2, 0x2, 0x293, 0x298, 0x5, 0x1a, 0xe, 0x2, 0x294, 0x295, 0x7, 0xc5, - 0x2, 0x2, 0x295, 0x297, 0x5, 0x1a, 0xe, 0x2, 0x296, 0x294, 0x3, 0x2, - 0x2, 0x2, 0x297, 0x29a, 0x3, 0x2, 0x2, 0x2, 0x298, 0x296, 0x3, 0x2, - 0x2, 0x2, 0x298, 0x299, 0x3, 0x2, 0x2, 0x2, 0x299, 0x29b, 0x3, 0x2, - 0x2, 0x2, 0x29a, 0x298, 0x3, 0x2, 0x2, 0x2, 0x29b, 0x29c, 0x7, 0xda, - 0x2, 0x2, 0x29c, 0x19, 0x3, 0x2, 0x2, 0x2, 0x29d, 0x29e, 0x5, 0xd6, - 0x6c, 0x2, 0x29e, 0x2b4, 0x5, 0xaa, 0x56, 0x2, 0x29f, 0x2a0, 0x6, 0xe, - 0x2, 0x3, 0x2a0, 0x2a1, 0x7, 0x27, 0x2, 0x2, 0x2a1, 0x2a2, 0x5, 0xcc, - 0x67, 0x2, 0x2a2, 0x2a3, 0x8, 0xe, 0x1, 0x2, 0x2a3, 0x2b3, 0x3, 0x2, - 0x2, 0x2, 0x2a4, 0x2a5, 0x6, 0xe, 0x3, 0x3, 0x2a5, 0x2a6, 0x7, 0x3a, - 0x2, 0x2, 0x2a6, 0x2a7, 0x5, 0xb0, 0x59, 0x2, 0x2a7, 0x2a8, 0x8, 0xe, - 0x1, 0x2, 0x2a8, 0x2b3, 0x3, 0x2, 0x2, 0x2, 0x2a9, 0x2aa, 0x6, 0xe, - 0x4, 0x3, 0x2aa, 0x2ab, 0x7, 0x4a, 0x2, 0x2, 0x2ab, 0x2b3, 0x8, 0xe, - 0x1, 0x2, 0x2ac, 0x2ad, 0x6, 0xe, 0x5, 0x3, 0x2ad, 0x2ae, 0x7, 0x52, - 0x2, 0x2, 0x2ae, 0x2b3, 0x8, 0xe, 0x1, 0x2, 0x2af, 0x2b0, 0x6, 0xe, - 0x6, 0x3, 0x2b0, 0x2b1, 0x7, 0x58, 0x2, 0x2, 0x2b1, 0x2b3, 0x8, 0xe, - 0x1, 0x2, 0x2b2, 0x29f, 0x3, 0x2, 0x2, 0x2, 0x2b2, 0x2a4, 0x3, 0x2, - 0x2, 0x2, 0x2b2, 0x2a9, 0x3, 0x2, 0x2, 0x2, 0x2b2, 0x2ac, 0x3, 0x2, - 0x2, 0x2, 0x2b2, 0x2af, 0x3, 0x2, 0x2, 0x2, 0x2b3, 0x2b6, 0x3, 0x2, - 0x2, 0x2, 0x2b4, 0x2b2, 0x3, 0x2, 0x2, 0x2, 0x2b4, 0x2b5, 0x3, 0x2, - 0x2, 0x2, 0x2b5, 0x1b, 0x3, 0x2, 0x2, 0x2, 0x2b6, 0x2b4, 0x3, 0x2, 0x2, - 0x2, 0x2b7, 0x2b9, 0x5, 0x1e, 0x10, 0x2, 0x2b8, 0x2b7, 0x3, 0x2, 0x2, - 0x2, 0x2b8, 0x2b9, 0x3, 0x2, 0x2, 0x2, 0x2b9, 0x2d0, 0x3, 0x2, 0x2, - 0x2, 0x2ba, 0x2bb, 0x6, 0xf, 0x7, 0x3, 0x2bb, 0x2bc, 0x5, 0x22, 0x12, - 0x2, 0x2bc, 0x2bd, 0x8, 0xf, 0x1, 0x2, 0x2bd, 0x2cf, 0x3, 0x2, 0x2, - 0x2, 0x2be, 0x2bf, 0x6, 0xf, 0x8, 0x3, 0x2bf, 0x2c0, 0x5, 0x24, 0x13, - 0x2, 0x2c0, 0x2c1, 0x8, 0xf, 0x1, 0x2, 0x2c1, 0x2cf, 0x3, 0x2, 0x2, - 0x2, 0x2c2, 0x2c3, 0x6, 0xf, 0x9, 0x3, 0x2c3, 0x2c4, 0x5, 0x26, 0x14, - 0x2, 0x2c4, 0x2c5, 0x8, 0xf, 0x1, 0x2, 0x2c5, 0x2cf, 0x3, 0x2, 0x2, - 0x2, 0x2c6, 0x2c7, 0x6, 0xf, 0xa, 0x3, 0x2c7, 0x2c8, 0x5, 0x28, 0x15, - 0x2, 0x2c8, 0x2c9, 0x8, 0xf, 0x1, 0x2, 0x2c9, 0x2cf, 0x3, 0x2, 0x2, - 0x2, 0x2ca, 0x2cb, 0x6, 0xf, 0xb, 0x3, 0x2cb, 0x2cc, 0x5, 0x2a, 0x16, - 0x2, 0x2cc, 0x2cd, 0x8, 0xf, 0x1, 0x2, 0x2cd, 0x2cf, 0x3, 0x2, 0x2, - 0x2, 0x2ce, 0x2ba, 0x3, 0x2, 0x2, 0x2, 0x2ce, 0x2be, 0x3, 0x2, 0x2, - 0x2, 0x2ce, 0x2c2, 0x3, 0x2, 0x2, 0x2, 0x2ce, 0x2c6, 0x3, 0x2, 0x2, - 0x2, 0x2ce, 0x2ca, 0x3, 0x2, 0x2, 0x2, 0x2cf, 0x2d2, 0x3, 0x2, 0x2, - 0x2, 0x2d0, 0x2ce, 0x3, 0x2, 0x2, 0x2, 0x2d0, 0x2d1, 0x3, 0x2, 0x2, - 0x2, 0x2d1, 0x1d, 0x3, 0x2, 0x2, 0x2, 0x2d2, 0x2d0, 0x3, 0x2, 0x2, 0x2, - 0x2d3, 0x2d4, 0x7, 0x7f, 0x2, 0x2, 0x2d4, 0x2d5, 0x7, 0x5a, 0x2, 0x2, - 0x2d5, 0x2d6, 0x5, 0xac, 0x57, 0x2, 0x2d6, 0x1f, 0x3, 0x2, 0x2, 0x2, - 0x2d7, 0x2de, 0x5, 0xd6, 0x6c, 0x2, 0x2d8, 0x2db, 0x5, 0xd6, 0x6c, 0x2, - 0x2d9, 0x2da, 0x7, 0xd0, 0x2, 0x2, 0x2da, 0x2dc, 0x7, 0xda, 0x2, 0x2, - 0x2db, 0x2d9, 0x3, 0x2, 0x2, 0x2, 0x2db, 0x2dc, 0x3, 0x2, 0x2, 0x2, - 0x2dc, 0x2df, 0x3, 0x2, 0x2, 0x2, 0x2dd, 0x2df, 0x5, 0xcc, 0x67, 0x2, - 0x2de, 0x2d8, 0x3, 0x2, 0x2, 0x2, 0x2de, 0x2dd, 0x3, 0x2, 0x2, 0x2, - 0x2df, 0x21, 0x3, 0x2, 0x2, 0x2, 0x2e0, 0x2e1, 0x7, 0x93, 0x2, 0x2, - 0x2e1, 0x2e2, 0x7, 0xd0, 0x2, 0x2, 0x2e2, 0x2e3, 0x5, 0xd6, 0x6c, 0x2, - 0x2e3, 0x2e7, 0x7, 0xd0, 0x2, 0x2, 0x2e4, 0x2e6, 0x5, 0x20, 0x11, 0x2, - 0x2e5, 0x2e4, 0x3, 0x2, 0x2, 0x2, 0x2e6, 0x2e9, 0x3, 0x2, 0x2, 0x2, - 0x2e7, 0x2e5, 0x3, 0x2, 0x2, 0x2, 0x2e7, 0x2e8, 0x3, 0x2, 0x2, 0x2, - 0x2e8, 0x2ea, 0x3, 0x2, 0x2, 0x2, 0x2e9, 0x2e7, 0x3, 0x2, 0x2, 0x2, - 0x2ea, 0x2eb, 0x7, 0xda, 0x2, 0x2, 0x2eb, 0x2ec, 0x7, 0xda, 0x2, 0x2, - 0x2ec, 0x23, 0x3, 0x2, 0x2, 0x2, 0x2ed, 0x2ee, 0x7, 0x60, 0x2, 0x2, - 0x2ee, 0x2f8, 0x7, 0xd0, 0x2, 0x2, 0x2ef, 0x2f9, 0x7, 0xbd, 0x2, 0x2, - 0x2f0, 0x2f1, 0x7, 0x6a, 0x2, 0x2, 0x2f1, 0x2f2, 0x7, 0xbd, 0x2, 0x2, - 0x2f2, 0x2f3, 0x7, 0x68, 0x2, 0x2, 0x2f3, 0x2f9, 0x7, 0xbd, 0x2, 0x2, - 0x2f4, 0x2f5, 0x7, 0x68, 0x2, 0x2, 0x2f5, 0x2f6, 0x7, 0xbd, 0x2, 0x2, - 0x2f6, 0x2f7, 0x7, 0x6a, 0x2, 0x2, 0x2f7, 0x2f9, 0x7, 0xbd, 0x2, 0x2, - 0x2f8, 0x2ef, 0x3, 0x2, 0x2, 0x2, 0x2f8, 0x2f0, 0x3, 0x2, 0x2, 0x2, - 0x2f8, 0x2f4, 0x3, 0x2, 0x2, 0x2, 0x2f9, 0x2fa, 0x3, 0x2, 0x2, 0x2, - 0x2fa, 0x2fb, 0x7, 0xda, 0x2, 0x2, 0x2fb, 0x25, 0x3, 0x2, 0x2, 0x2, - 0x2fc, 0x2fd, 0x7, 0x5d, 0x2, 0x2, 0x2fd, 0x2fe, 0x7, 0xd0, 0x2, 0x2, - 0x2fe, 0x2ff, 0x5, 0xd6, 0x6c, 0x2, 0x2ff, 0x303, 0x7, 0xd0, 0x2, 0x2, - 0x300, 0x302, 0x5, 0x20, 0x11, 0x2, 0x301, 0x300, 0x3, 0x2, 0x2, 0x2, - 0x302, 0x305, 0x3, 0x2, 0x2, 0x2, 0x303, 0x301, 0x3, 0x2, 0x2, 0x2, - 0x303, 0x304, 0x3, 0x2, 0x2, 0x2, 0x304, 0x306, 0x3, 0x2, 0x2, 0x2, - 0x305, 0x303, 0x3, 0x2, 0x2, 0x2, 0x306, 0x307, 0x7, 0xda, 0x2, 0x2, - 0x307, 0x308, 0x7, 0xda, 0x2, 0x2, 0x308, 0x27, 0x3, 0x2, 0x2, 0x2, - 0x309, 0x30a, 0x7, 0x82, 0x2, 0x2, 0x30a, 0x315, 0x7, 0xd0, 0x2, 0x2, - 0x30b, 0x30c, 0x7, 0x6a, 0x2, 0x2, 0x30c, 0x30d, 0x5, 0xd6, 0x6c, 0x2, - 0x30d, 0x30e, 0x7, 0x68, 0x2, 0x2, 0x30e, 0x30f, 0x5, 0xd6, 0x6c, 0x2, - 0x30f, 0x316, 0x3, 0x2, 0x2, 0x2, 0x310, 0x311, 0x7, 0x68, 0x2, 0x2, - 0x311, 0x312, 0x5, 0xd6, 0x6c, 0x2, 0x312, 0x313, 0x7, 0x6a, 0x2, 0x2, - 0x313, 0x314, 0x5, 0xd6, 0x6c, 0x2, 0x314, 0x316, 0x3, 0x2, 0x2, 0x2, - 0x315, 0x30b, 0x3, 0x2, 0x2, 0x2, 0x315, 0x310, 0x3, 0x2, 0x2, 0x2, - 0x316, 0x317, 0x3, 0x2, 0x2, 0x2, 0x317, 0x318, 0x7, 0xda, 0x2, 0x2, - 0x318, 0x29, 0x3, 0x2, 0x2, 0x2, 0x319, 0x31a, 0x7, 0x91, 0x2, 0x2, - 0x31a, 0x31b, 0x7, 0xd0, 0x2, 0x2, 0x31b, 0x31c, 0x5, 0x9a, 0x4e, 0x2, - 0x31c, 0x31d, 0x7, 0xda, 0x2, 0x2, 0x31d, 0x2b, 0x3, 0x2, 0x2, 0x2, - 0x31e, 0x31f, 0x7, 0x76, 0x2, 0x2, 0x31f, 0x322, 0x7, 0x19, 0x2, 0x2, - 0x320, 0x323, 0x5, 0xd6, 0x6c, 0x2, 0x321, 0x323, 0x7, 0xbf, 0x2, 0x2, - 0x322, 0x320, 0x3, 0x2, 0x2, 0x2, 0x322, 0x321, 0x3, 0x2, 0x2, 0x2, - 0x323, 0x2d, 0x3, 0x2, 0x2, 0x2, 0x324, 0x325, 0x7, 0xae, 0x2, 0x2, - 0x325, 0x326, 0x7, 0xbf, 0x2, 0x2, 0x326, 0x2f, 0x3, 0x2, 0x2, 0x2, - 0x327, 0x328, 0x7, 0xa2, 0x2, 0x2, 0x328, 0x329, 0x5, 0xc0, 0x61, 0x2, - 0x329, 0x31, 0x3, 0x2, 0x2, 0x2, 0x32a, 0x32b, 0x7, 0xc, 0x2, 0x2, 0x32b, - 0x32c, 0x5, 0x68, 0x35, 0x2, 0x32c, 0x33, 0x3, 0x2, 0x2, 0x2, 0x32d, - 0x32e, 0x7, 0xd0, 0x2, 0x2, 0x32e, 0x333, 0x5, 0x42, 0x22, 0x2, 0x32f, - 0x330, 0x7, 0xc5, 0x2, 0x2, 0x330, 0x332, 0x5, 0x42, 0x22, 0x2, 0x331, - 0x32f, 0x3, 0x2, 0x2, 0x2, 0x332, 0x335, 0x3, 0x2, 0x2, 0x2, 0x333, - 0x331, 0x3, 0x2, 0x2, 0x2, 0x333, 0x334, 0x3, 0x2, 0x2, 0x2, 0x334, - 0x336, 0x3, 0x2, 0x2, 0x2, 0x335, 0x333, 0x3, 0x2, 0x2, 0x2, 0x336, - 0x337, 0x7, 0xda, 0x2, 0x2, 0x337, 0x33d, 0x3, 0x2, 0x2, 0x2, 0x338, - 0x339, 0x7, 0xc, 0x2, 0x2, 0x339, 0x33d, 0x5, 0xc0, 0x61, 0x2, 0x33a, - 0x33b, 0x7, 0xc, 0x2, 0x2, 0x33b, 0x33d, 0x5, 0xbe, 0x60, 0x2, 0x33c, - 0x32d, 0x3, 0x2, 0x2, 0x2, 0x33c, 0x338, 0x3, 0x2, 0x2, 0x2, 0x33c, - 0x33a, 0x3, 0x2, 0x2, 0x2, 0x33d, 0x35, 0x3, 0x2, 0x2, 0x2, 0x33e, 0x359, - 0x5, 0x40, 0x21, 0x2, 0x33f, 0x340, 0x6, 0x1c, 0xc, 0x3, 0x340, 0x341, - 0x5, 0x7e, 0x40, 0x2, 0x341, 0x342, 0x8, 0x1c, 0x1, 0x2, 0x342, 0x358, - 0x3, 0x2, 0x2, 0x2, 0x343, 0x344, 0x6, 0x1c, 0xd, 0x3, 0x344, 0x345, - 0x5, 0x38, 0x1d, 0x2, 0x345, 0x346, 0x8, 0x1c, 0x1, 0x2, 0x346, 0x358, - 0x3, 0x2, 0x2, 0x2, 0x347, 0x348, 0x6, 0x1c, 0xe, 0x3, 0x348, 0x349, - 0x5, 0x3a, 0x1e, 0x2, 0x349, 0x34a, 0x8, 0x1c, 0x1, 0x2, 0x34a, 0x358, - 0x3, 0x2, 0x2, 0x2, 0x34b, 0x34c, 0x6, 0x1c, 0xf, 0x3, 0x34c, 0x34d, - 0x5, 0x3c, 0x1f, 0x2, 0x34d, 0x34e, 0x8, 0x1c, 0x1, 0x2, 0x34e, 0x358, - 0x3, 0x2, 0x2, 0x2, 0x34f, 0x350, 0x6, 0x1c, 0x10, 0x3, 0x350, 0x351, - 0x5, 0x3e, 0x20, 0x2, 0x351, 0x352, 0x8, 0x1c, 0x1, 0x2, 0x352, 0x358, - 0x3, 0x2, 0x2, 0x2, 0x353, 0x354, 0x6, 0x1c, 0x11, 0x3, 0x354, 0x355, - 0x5, 0x86, 0x44, 0x2, 0x355, 0x356, 0x8, 0x1c, 0x1, 0x2, 0x356, 0x358, - 0x3, 0x2, 0x2, 0x2, 0x357, 0x33f, 0x3, 0x2, 0x2, 0x2, 0x357, 0x343, - 0x3, 0x2, 0x2, 0x2, 0x357, 0x347, 0x3, 0x2, 0x2, 0x2, 0x357, 0x34b, - 0x3, 0x2, 0x2, 0x2, 0x357, 0x34f, 0x3, 0x2, 0x2, 0x2, 0x357, 0x353, - 0x3, 0x2, 0x2, 0x2, 0x358, 0x35b, 0x3, 0x2, 0x2, 0x2, 0x359, 0x357, - 0x3, 0x2, 0x2, 0x2, 0x359, 0x35a, 0x3, 0x2, 0x2, 0x2, 0x35a, 0x37, 0x3, - 0x2, 0x2, 0x2, 0x35b, 0x359, 0x3, 0x2, 0x2, 0x2, 0x35c, 0x35d, 0x7, - 0x7c, 0x2, 0x2, 0x35d, 0x35e, 0x7, 0x14, 0x2, 0x2, 0x35e, 0x35f, 0x5, - 0xb0, 0x59, 0x2, 0x35f, 0x39, 0x3, 0x2, 0x2, 0x2, 0x360, 0x361, 0x7, - 0x7f, 0x2, 0x2, 0x361, 0x362, 0x7, 0x5a, 0x2, 0x2, 0x362, 0x363, 0x5, - 0xb0, 0x59, 0x2, 0x363, 0x3b, 0x3, 0x2, 0x2, 0x2, 0x364, 0x365, 0x7, - 0x8b, 0x2, 0x2, 0x365, 0x366, 0x7, 0x14, 0x2, 0x2, 0x366, 0x367, 0x5, - 0xb0, 0x59, 0x2, 0x367, 0x3d, 0x3, 0x2, 0x2, 0x2, 0x368, 0x369, 0x7, - 0xa8, 0x2, 0x2, 0x369, 0x36e, 0x5, 0x50, 0x29, 0x2, 0x36a, 0x36b, 0x7, - 0xc5, 0x2, 0x2, 0x36b, 0x36d, 0x5, 0x50, 0x29, 0x2, 0x36c, 0x36a, 0x3, - 0x2, 0x2, 0x2, 0x36d, 0x370, 0x3, 0x2, 0x2, 0x2, 0x36e, 0x36c, 0x3, - 0x2, 0x2, 0x2, 0x36e, 0x36f, 0x3, 0x2, 0x2, 0x2, 0x36f, 0x3f, 0x3, 0x2, - 0x2, 0x2, 0x370, 0x36e, 0x3, 0x2, 0x2, 0x2, 0x371, 0x373, 0x7, 0x36, - 0x2, 0x2, 0x372, 0x374, 0x7, 0xca, 0x2, 0x2, 0x373, 0x372, 0x3, 0x2, - 0x2, 0x2, 0x373, 0x374, 0x3, 0x2, 0x2, 0x2, 0x374, 0x375, 0x3, 0x2, - 0x2, 0x2, 0x375, 0x37b, 0x5, 0xd8, 0x6d, 0x2, 0x376, 0x378, 0x7, 0xd0, - 0x2, 0x2, 0x377, 0x379, 0x5, 0xac, 0x57, 0x2, 0x378, 0x377, 0x3, 0x2, - 0x2, 0x2, 0x378, 0x379, 0x3, 0x2, 0x2, 0x2, 0x379, 0x37a, 0x3, 0x2, - 0x2, 0x2, 0x37a, 0x37c, 0x7, 0xda, 0x2, 0x2, 0x37b, 0x376, 0x3, 0x2, - 0x2, 0x2, 0x37b, 0x37c, 0x3, 0x2, 0x2, 0x2, 0x37c, 0x41, 0x3, 0x2, 0x2, - 0x2, 0x37d, 0x388, 0x5, 0x44, 0x23, 0x2, 0x37e, 0x37f, 0x7, 0x1e, 0x2, - 0x2, 0x37f, 0x380, 0x5, 0xd6, 0x6c, 0x2, 0x380, 0x381, 0x7, 0x17, 0x2, - 0x2, 0x381, 0x382, 0x5, 0xb0, 0x59, 0x2, 0x382, 0x388, 0x3, 0x2, 0x2, - 0x2, 0x383, 0x384, 0x7, 0x50, 0x2, 0x2, 0x384, 0x388, 0x5, 0x48, 0x25, - 0x2, 0x385, 0x386, 0x7, 0x80, 0x2, 0x2, 0x386, 0x388, 0x5, 0x4a, 0x26, - 0x2, 0x387, 0x37d, 0x3, 0x2, 0x2, 0x2, 0x387, 0x37e, 0x3, 0x2, 0x2, - 0x2, 0x387, 0x383, 0x3, 0x2, 0x2, 0x2, 0x387, 0x385, 0x3, 0x2, 0x2, - 0x2, 0x388, 0x43, 0x3, 0x2, 0x2, 0x2, 0x389, 0x38a, 0x5, 0xba, 0x5e, - 0x2, 0x38a, 0x38c, 0x5, 0xaa, 0x56, 0x2, 0x38b, 0x38d, 0x5, 0x46, 0x24, - 0x2, 0x38c, 0x38b, 0x3, 0x2, 0x2, 0x2, 0x38c, 0x38d, 0x3, 0x2, 0x2, - 0x2, 0x38d, 0x390, 0x3, 0x2, 0x2, 0x2, 0x38e, 0x38f, 0x7, 0x1d, 0x2, - 0x2, 0x38f, 0x391, 0x7, 0xbf, 0x2, 0x2, 0x390, 0x38e, 0x3, 0x2, 0x2, - 0x2, 0x390, 0x391, 0x3, 0x2, 0x2, 0x2, 0x391, 0x393, 0x3, 0x2, 0x2, - 0x2, 0x392, 0x394, 0x5, 0x4c, 0x27, 0x2, 0x393, 0x392, 0x3, 0x2, 0x2, - 0x2, 0x393, 0x394, 0x3, 0x2, 0x2, 0x2, 0x394, 0x397, 0x3, 0x2, 0x2, - 0x2, 0x395, 0x396, 0x7, 0xa8, 0x2, 0x2, 0x396, 0x398, 0x5, 0xb0, 0x59, - 0x2, 0x397, 0x395, 0x3, 0x2, 0x2, 0x2, 0x397, 0x398, 0x3, 0x2, 0x2, - 0x2, 0x398, 0x3aa, 0x3, 0x2, 0x2, 0x2, 0x399, 0x39b, 0x5, 0xba, 0x5e, - 0x2, 0x39a, 0x39c, 0x5, 0xaa, 0x56, 0x2, 0x39b, 0x39a, 0x3, 0x2, 0x2, - 0x2, 0x39b, 0x39c, 0x3, 0x2, 0x2, 0x2, 0x39c, 0x39d, 0x3, 0x2, 0x2, - 0x2, 0x39d, 0x3a0, 0x5, 0x46, 0x24, 0x2, 0x39e, 0x39f, 0x7, 0x1d, 0x2, + 0x2, 0x2, 0x2, 0x28e, 0x292, 0x7, 0xb0, 0x2, 0x2, 0x28f, 0x290, 0x7, + 0x4d, 0x2, 0x2, 0x290, 0x291, 0x7, 0x72, 0x2, 0x2, 0x291, 0x293, 0x7, + 0x38, 0x2, 0x2, 0x292, 0x28f, 0x3, 0x2, 0x2, 0x2, 0x292, 0x293, 0x3, + 0x2, 0x2, 0x2, 0x293, 0x294, 0x3, 0x2, 0x2, 0x2, 0x294, 0x296, 0x5, + 0xc0, 0x61, 0x2, 0x295, 0x297, 0x5, 0x2e, 0x18, 0x2, 0x296, 0x295, 0x3, + 0x2, 0x2, 0x2, 0x296, 0x297, 0x3, 0x2, 0x2, 0x2, 0x297, 0x299, 0x3, + 0x2, 0x2, 0x2, 0x298, 0x29a, 0x5, 0x2c, 0x17, 0x2, 0x299, 0x298, 0x3, + 0x2, 0x2, 0x2, 0x299, 0x29a, 0x3, 0x2, 0x2, 0x2, 0x29a, 0x29c, 0x3, + 0x2, 0x2, 0x2, 0x29b, 0x29d, 0x5, 0x34, 0x1b, 0x2, 0x29c, 0x29b, 0x3, + 0x2, 0x2, 0x2, 0x29c, 0x29d, 0x3, 0x2, 0x2, 0x2, 0x29d, 0x29e, 0x3, + 0x2, 0x2, 0x2, 0x29e, 0x29f, 0x5, 0x32, 0x1a, 0x2, 0x29f, 0x2a1, 0x3, + 0x2, 0x2, 0x2, 0x2a0, 0x207, 0x3, 0x2, 0x2, 0x2, 0x2a0, 0x21c, 0x3, + 0x2, 0x2, 0x2, 0x2a0, 0x22e, 0x3, 0x2, 0x2, 0x2, 0x2a0, 0x24c, 0x3, + 0x2, 0x2, 0x2, 0x2a0, 0x26e, 0x3, 0x2, 0x2, 0x2, 0x2a0, 0x289, 0x3, + 0x2, 0x2, 0x2, 0x2a1, 0x17, 0x3, 0x2, 0x2, 0x2, 0x2a2, 0x2a3, 0x7, 0xd0, + 0x2, 0x2, 0x2a3, 0x2a8, 0x5, 0x1a, 0xe, 0x2, 0x2a4, 0x2a5, 0x7, 0xc5, + 0x2, 0x2, 0x2a5, 0x2a7, 0x5, 0x1a, 0xe, 0x2, 0x2a6, 0x2a4, 0x3, 0x2, + 0x2, 0x2, 0x2a7, 0x2aa, 0x3, 0x2, 0x2, 0x2, 0x2a8, 0x2a6, 0x3, 0x2, + 0x2, 0x2, 0x2a8, 0x2a9, 0x3, 0x2, 0x2, 0x2, 0x2a9, 0x2ab, 0x3, 0x2, + 0x2, 0x2, 0x2aa, 0x2a8, 0x3, 0x2, 0x2, 0x2, 0x2ab, 0x2ac, 0x7, 0xda, + 0x2, 0x2, 0x2ac, 0x19, 0x3, 0x2, 0x2, 0x2, 0x2ad, 0x2ae, 0x5, 0xd6, + 0x6c, 0x2, 0x2ae, 0x2c4, 0x5, 0xaa, 0x56, 0x2, 0x2af, 0x2b0, 0x6, 0xe, + 0x2, 0x3, 0x2b0, 0x2b1, 0x7, 0x27, 0x2, 0x2, 0x2b1, 0x2b2, 0x5, 0xcc, + 0x67, 0x2, 0x2b2, 0x2b3, 0x8, 0xe, 0x1, 0x2, 0x2b3, 0x2c3, 0x3, 0x2, + 0x2, 0x2, 0x2b4, 0x2b5, 0x6, 0xe, 0x3, 0x3, 0x2b5, 0x2b6, 0x7, 0x3a, + 0x2, 0x2, 0x2b6, 0x2b7, 0x5, 0xb0, 0x59, 0x2, 0x2b7, 0x2b8, 0x8, 0xe, + 0x1, 0x2, 0x2b8, 0x2c3, 0x3, 0x2, 0x2, 0x2, 0x2b9, 0x2ba, 0x6, 0xe, + 0x4, 0x3, 0x2ba, 0x2bb, 0x7, 0x4a, 0x2, 0x2, 0x2bb, 0x2c3, 0x8, 0xe, + 0x1, 0x2, 0x2bc, 0x2bd, 0x6, 0xe, 0x5, 0x3, 0x2bd, 0x2be, 0x7, 0x52, + 0x2, 0x2, 0x2be, 0x2c3, 0x8, 0xe, 0x1, 0x2, 0x2bf, 0x2c0, 0x6, 0xe, + 0x6, 0x3, 0x2c0, 0x2c1, 0x7, 0x58, 0x2, 0x2, 0x2c1, 0x2c3, 0x8, 0xe, + 0x1, 0x2, 0x2c2, 0x2af, 0x3, 0x2, 0x2, 0x2, 0x2c2, 0x2b4, 0x3, 0x2, + 0x2, 0x2, 0x2c2, 0x2b9, 0x3, 0x2, 0x2, 0x2, 0x2c2, 0x2bc, 0x3, 0x2, + 0x2, 0x2, 0x2c2, 0x2bf, 0x3, 0x2, 0x2, 0x2, 0x2c3, 0x2c6, 0x3, 0x2, + 0x2, 0x2, 0x2c4, 0x2c2, 0x3, 0x2, 0x2, 0x2, 0x2c4, 0x2c5, 0x3, 0x2, + 0x2, 0x2, 0x2c5, 0x1b, 0x3, 0x2, 0x2, 0x2, 0x2c6, 0x2c4, 0x3, 0x2, 0x2, + 0x2, 0x2c7, 0x2c9, 0x5, 0x1e, 0x10, 0x2, 0x2c8, 0x2c7, 0x3, 0x2, 0x2, + 0x2, 0x2c8, 0x2c9, 0x3, 0x2, 0x2, 0x2, 0x2c9, 0x2e0, 0x3, 0x2, 0x2, + 0x2, 0x2ca, 0x2cb, 0x6, 0xf, 0x7, 0x3, 0x2cb, 0x2cc, 0x5, 0x22, 0x12, + 0x2, 0x2cc, 0x2cd, 0x8, 0xf, 0x1, 0x2, 0x2cd, 0x2df, 0x3, 0x2, 0x2, + 0x2, 0x2ce, 0x2cf, 0x6, 0xf, 0x8, 0x3, 0x2cf, 0x2d0, 0x5, 0x24, 0x13, + 0x2, 0x2d0, 0x2d1, 0x8, 0xf, 0x1, 0x2, 0x2d1, 0x2df, 0x3, 0x2, 0x2, + 0x2, 0x2d2, 0x2d3, 0x6, 0xf, 0x9, 0x3, 0x2d3, 0x2d4, 0x5, 0x26, 0x14, + 0x2, 0x2d4, 0x2d5, 0x8, 0xf, 0x1, 0x2, 0x2d5, 0x2df, 0x3, 0x2, 0x2, + 0x2, 0x2d6, 0x2d7, 0x6, 0xf, 0xa, 0x3, 0x2d7, 0x2d8, 0x5, 0x28, 0x15, + 0x2, 0x2d8, 0x2d9, 0x8, 0xf, 0x1, 0x2, 0x2d9, 0x2df, 0x3, 0x2, 0x2, + 0x2, 0x2da, 0x2db, 0x6, 0xf, 0xb, 0x3, 0x2db, 0x2dc, 0x5, 0x2a, 0x16, + 0x2, 0x2dc, 0x2dd, 0x8, 0xf, 0x1, 0x2, 0x2dd, 0x2df, 0x3, 0x2, 0x2, + 0x2, 0x2de, 0x2ca, 0x3, 0x2, 0x2, 0x2, 0x2de, 0x2ce, 0x3, 0x2, 0x2, + 0x2, 0x2de, 0x2d2, 0x3, 0x2, 0x2, 0x2, 0x2de, 0x2d6, 0x3, 0x2, 0x2, + 0x2, 0x2de, 0x2da, 0x3, 0x2, 0x2, 0x2, 0x2df, 0x2e2, 0x3, 0x2, 0x2, + 0x2, 0x2e0, 0x2de, 0x3, 0x2, 0x2, 0x2, 0x2e0, 0x2e1, 0x3, 0x2, 0x2, + 0x2, 0x2e1, 0x1d, 0x3, 0x2, 0x2, 0x2, 0x2e2, 0x2e0, 0x3, 0x2, 0x2, 0x2, + 0x2e3, 0x2e4, 0x7, 0x7f, 0x2, 0x2, 0x2e4, 0x2e5, 0x7, 0x5a, 0x2, 0x2, + 0x2e5, 0x2e6, 0x5, 0xac, 0x57, 0x2, 0x2e6, 0x1f, 0x3, 0x2, 0x2, 0x2, + 0x2e7, 0x2ee, 0x5, 0xd6, 0x6c, 0x2, 0x2e8, 0x2eb, 0x5, 0xd6, 0x6c, 0x2, + 0x2e9, 0x2ea, 0x7, 0xd0, 0x2, 0x2, 0x2ea, 0x2ec, 0x7, 0xda, 0x2, 0x2, + 0x2eb, 0x2e9, 0x3, 0x2, 0x2, 0x2, 0x2eb, 0x2ec, 0x3, 0x2, 0x2, 0x2, + 0x2ec, 0x2ef, 0x3, 0x2, 0x2, 0x2, 0x2ed, 0x2ef, 0x5, 0xcc, 0x67, 0x2, + 0x2ee, 0x2e8, 0x3, 0x2, 0x2, 0x2, 0x2ee, 0x2ed, 0x3, 0x2, 0x2, 0x2, + 0x2ef, 0x21, 0x3, 0x2, 0x2, 0x2, 0x2f0, 0x2f1, 0x7, 0x93, 0x2, 0x2, + 0x2f1, 0x2f2, 0x7, 0xd0, 0x2, 0x2, 0x2f2, 0x2f3, 0x5, 0xd6, 0x6c, 0x2, + 0x2f3, 0x2f7, 0x7, 0xd0, 0x2, 0x2, 0x2f4, 0x2f6, 0x5, 0x20, 0x11, 0x2, + 0x2f5, 0x2f4, 0x3, 0x2, 0x2, 0x2, 0x2f6, 0x2f9, 0x3, 0x2, 0x2, 0x2, + 0x2f7, 0x2f5, 0x3, 0x2, 0x2, 0x2, 0x2f7, 0x2f8, 0x3, 0x2, 0x2, 0x2, + 0x2f8, 0x2fa, 0x3, 0x2, 0x2, 0x2, 0x2f9, 0x2f7, 0x3, 0x2, 0x2, 0x2, + 0x2fa, 0x2fb, 0x7, 0xda, 0x2, 0x2, 0x2fb, 0x2fc, 0x7, 0xda, 0x2, 0x2, + 0x2fc, 0x23, 0x3, 0x2, 0x2, 0x2, 0x2fd, 0x2fe, 0x7, 0x60, 0x2, 0x2, + 0x2fe, 0x308, 0x7, 0xd0, 0x2, 0x2, 0x2ff, 0x309, 0x7, 0xbd, 0x2, 0x2, + 0x300, 0x301, 0x7, 0x6a, 0x2, 0x2, 0x301, 0x302, 0x7, 0xbd, 0x2, 0x2, + 0x302, 0x303, 0x7, 0x68, 0x2, 0x2, 0x303, 0x309, 0x7, 0xbd, 0x2, 0x2, + 0x304, 0x305, 0x7, 0x68, 0x2, 0x2, 0x305, 0x306, 0x7, 0xbd, 0x2, 0x2, + 0x306, 0x307, 0x7, 0x6a, 0x2, 0x2, 0x307, 0x309, 0x7, 0xbd, 0x2, 0x2, + 0x308, 0x2ff, 0x3, 0x2, 0x2, 0x2, 0x308, 0x300, 0x3, 0x2, 0x2, 0x2, + 0x308, 0x304, 0x3, 0x2, 0x2, 0x2, 0x309, 0x30a, 0x3, 0x2, 0x2, 0x2, + 0x30a, 0x30b, 0x7, 0xda, 0x2, 0x2, 0x30b, 0x25, 0x3, 0x2, 0x2, 0x2, + 0x30c, 0x30d, 0x7, 0x5d, 0x2, 0x2, 0x30d, 0x30e, 0x7, 0xd0, 0x2, 0x2, + 0x30e, 0x30f, 0x5, 0xd6, 0x6c, 0x2, 0x30f, 0x313, 0x7, 0xd0, 0x2, 0x2, + 0x310, 0x312, 0x5, 0x20, 0x11, 0x2, 0x311, 0x310, 0x3, 0x2, 0x2, 0x2, + 0x312, 0x315, 0x3, 0x2, 0x2, 0x2, 0x313, 0x311, 0x3, 0x2, 0x2, 0x2, + 0x313, 0x314, 0x3, 0x2, 0x2, 0x2, 0x314, 0x316, 0x3, 0x2, 0x2, 0x2, + 0x315, 0x313, 0x3, 0x2, 0x2, 0x2, 0x316, 0x317, 0x7, 0xda, 0x2, 0x2, + 0x317, 0x318, 0x7, 0xda, 0x2, 0x2, 0x318, 0x27, 0x3, 0x2, 0x2, 0x2, + 0x319, 0x31a, 0x7, 0x82, 0x2, 0x2, 0x31a, 0x325, 0x7, 0xd0, 0x2, 0x2, + 0x31b, 0x31c, 0x7, 0x6a, 0x2, 0x2, 0x31c, 0x31d, 0x5, 0xd6, 0x6c, 0x2, + 0x31d, 0x31e, 0x7, 0x68, 0x2, 0x2, 0x31e, 0x31f, 0x5, 0xd6, 0x6c, 0x2, + 0x31f, 0x326, 0x3, 0x2, 0x2, 0x2, 0x320, 0x321, 0x7, 0x68, 0x2, 0x2, + 0x321, 0x322, 0x5, 0xd6, 0x6c, 0x2, 0x322, 0x323, 0x7, 0x6a, 0x2, 0x2, + 0x323, 0x324, 0x5, 0xd6, 0x6c, 0x2, 0x324, 0x326, 0x3, 0x2, 0x2, 0x2, + 0x325, 0x31b, 0x3, 0x2, 0x2, 0x2, 0x325, 0x320, 0x3, 0x2, 0x2, 0x2, + 0x326, 0x327, 0x3, 0x2, 0x2, 0x2, 0x327, 0x328, 0x7, 0xda, 0x2, 0x2, + 0x328, 0x29, 0x3, 0x2, 0x2, 0x2, 0x329, 0x32a, 0x7, 0x91, 0x2, 0x2, + 0x32a, 0x32b, 0x7, 0xd0, 0x2, 0x2, 0x32b, 0x32c, 0x5, 0x9a, 0x4e, 0x2, + 0x32c, 0x32d, 0x7, 0xda, 0x2, 0x2, 0x32d, 0x2b, 0x3, 0x2, 0x2, 0x2, + 0x32e, 0x32f, 0x7, 0x76, 0x2, 0x2, 0x32f, 0x332, 0x7, 0x19, 0x2, 0x2, + 0x330, 0x333, 0x5, 0xd6, 0x6c, 0x2, 0x331, 0x333, 0x7, 0xbf, 0x2, 0x2, + 0x332, 0x330, 0x3, 0x2, 0x2, 0x2, 0x332, 0x331, 0x3, 0x2, 0x2, 0x2, + 0x333, 0x2d, 0x3, 0x2, 0x2, 0x2, 0x334, 0x335, 0x7, 0xae, 0x2, 0x2, + 0x335, 0x336, 0x7, 0xbf, 0x2, 0x2, 0x336, 0x2f, 0x3, 0x2, 0x2, 0x2, + 0x337, 0x338, 0x7, 0xa2, 0x2, 0x2, 0x338, 0x339, 0x5, 0xc0, 0x61, 0x2, + 0x339, 0x31, 0x3, 0x2, 0x2, 0x2, 0x33a, 0x33b, 0x7, 0xc, 0x2, 0x2, 0x33b, + 0x33c, 0x5, 0x68, 0x35, 0x2, 0x33c, 0x33, 0x3, 0x2, 0x2, 0x2, 0x33d, + 0x33e, 0x7, 0xd0, 0x2, 0x2, 0x33e, 0x343, 0x5, 0x42, 0x22, 0x2, 0x33f, + 0x340, 0x7, 0xc5, 0x2, 0x2, 0x340, 0x342, 0x5, 0x42, 0x22, 0x2, 0x341, + 0x33f, 0x3, 0x2, 0x2, 0x2, 0x342, 0x345, 0x3, 0x2, 0x2, 0x2, 0x343, + 0x341, 0x3, 0x2, 0x2, 0x2, 0x343, 0x344, 0x3, 0x2, 0x2, 0x2, 0x344, + 0x346, 0x3, 0x2, 0x2, 0x2, 0x345, 0x343, 0x3, 0x2, 0x2, 0x2, 0x346, + 0x347, 0x7, 0xda, 0x2, 0x2, 0x347, 0x34d, 0x3, 0x2, 0x2, 0x2, 0x348, + 0x349, 0x7, 0xc, 0x2, 0x2, 0x349, 0x34d, 0x5, 0xc0, 0x61, 0x2, 0x34a, + 0x34b, 0x7, 0xc, 0x2, 0x2, 0x34b, 0x34d, 0x5, 0xbe, 0x60, 0x2, 0x34c, + 0x33d, 0x3, 0x2, 0x2, 0x2, 0x34c, 0x348, 0x3, 0x2, 0x2, 0x2, 0x34c, + 0x34a, 0x3, 0x2, 0x2, 0x2, 0x34d, 0x35, 0x3, 0x2, 0x2, 0x2, 0x34e, 0x369, + 0x5, 0x40, 0x21, 0x2, 0x34f, 0x350, 0x6, 0x1c, 0xc, 0x3, 0x350, 0x351, + 0x5, 0x7e, 0x40, 0x2, 0x351, 0x352, 0x8, 0x1c, 0x1, 0x2, 0x352, 0x368, + 0x3, 0x2, 0x2, 0x2, 0x353, 0x354, 0x6, 0x1c, 0xd, 0x3, 0x354, 0x355, + 0x5, 0x38, 0x1d, 0x2, 0x355, 0x356, 0x8, 0x1c, 0x1, 0x2, 0x356, 0x368, + 0x3, 0x2, 0x2, 0x2, 0x357, 0x358, 0x6, 0x1c, 0xe, 0x3, 0x358, 0x359, + 0x5, 0x3a, 0x1e, 0x2, 0x359, 0x35a, 0x8, 0x1c, 0x1, 0x2, 0x35a, 0x368, + 0x3, 0x2, 0x2, 0x2, 0x35b, 0x35c, 0x6, 0x1c, 0xf, 0x3, 0x35c, 0x35d, + 0x5, 0x3c, 0x1f, 0x2, 0x35d, 0x35e, 0x8, 0x1c, 0x1, 0x2, 0x35e, 0x368, + 0x3, 0x2, 0x2, 0x2, 0x35f, 0x360, 0x6, 0x1c, 0x10, 0x3, 0x360, 0x361, + 0x5, 0x3e, 0x20, 0x2, 0x361, 0x362, 0x8, 0x1c, 0x1, 0x2, 0x362, 0x368, + 0x3, 0x2, 0x2, 0x2, 0x363, 0x364, 0x6, 0x1c, 0x11, 0x3, 0x364, 0x365, + 0x5, 0x86, 0x44, 0x2, 0x365, 0x366, 0x8, 0x1c, 0x1, 0x2, 0x366, 0x368, + 0x3, 0x2, 0x2, 0x2, 0x367, 0x34f, 0x3, 0x2, 0x2, 0x2, 0x367, 0x353, + 0x3, 0x2, 0x2, 0x2, 0x367, 0x357, 0x3, 0x2, 0x2, 0x2, 0x367, 0x35b, + 0x3, 0x2, 0x2, 0x2, 0x367, 0x35f, 0x3, 0x2, 0x2, 0x2, 0x367, 0x363, + 0x3, 0x2, 0x2, 0x2, 0x368, 0x36b, 0x3, 0x2, 0x2, 0x2, 0x369, 0x367, + 0x3, 0x2, 0x2, 0x2, 0x369, 0x36a, 0x3, 0x2, 0x2, 0x2, 0x36a, 0x37, 0x3, + 0x2, 0x2, 0x2, 0x36b, 0x369, 0x3, 0x2, 0x2, 0x2, 0x36c, 0x36d, 0x7, + 0x7c, 0x2, 0x2, 0x36d, 0x36e, 0x7, 0x14, 0x2, 0x2, 0x36e, 0x36f, 0x5, + 0xb0, 0x59, 0x2, 0x36f, 0x39, 0x3, 0x2, 0x2, 0x2, 0x370, 0x371, 0x7, + 0x7f, 0x2, 0x2, 0x371, 0x372, 0x7, 0x5a, 0x2, 0x2, 0x372, 0x373, 0x5, + 0xb0, 0x59, 0x2, 0x373, 0x3b, 0x3, 0x2, 0x2, 0x2, 0x374, 0x375, 0x7, + 0x8b, 0x2, 0x2, 0x375, 0x376, 0x7, 0x14, 0x2, 0x2, 0x376, 0x377, 0x5, + 0xb0, 0x59, 0x2, 0x377, 0x3d, 0x3, 0x2, 0x2, 0x2, 0x378, 0x379, 0x7, + 0xa8, 0x2, 0x2, 0x379, 0x37e, 0x5, 0x50, 0x29, 0x2, 0x37a, 0x37b, 0x7, + 0xc5, 0x2, 0x2, 0x37b, 0x37d, 0x5, 0x50, 0x29, 0x2, 0x37c, 0x37a, 0x3, + 0x2, 0x2, 0x2, 0x37d, 0x380, 0x3, 0x2, 0x2, 0x2, 0x37e, 0x37c, 0x3, + 0x2, 0x2, 0x2, 0x37e, 0x37f, 0x3, 0x2, 0x2, 0x2, 0x37f, 0x3f, 0x3, 0x2, + 0x2, 0x2, 0x380, 0x37e, 0x3, 0x2, 0x2, 0x2, 0x381, 0x383, 0x7, 0x36, + 0x2, 0x2, 0x382, 0x384, 0x7, 0xca, 0x2, 0x2, 0x383, 0x382, 0x3, 0x2, + 0x2, 0x2, 0x383, 0x384, 0x3, 0x2, 0x2, 0x2, 0x384, 0x385, 0x3, 0x2, + 0x2, 0x2, 0x385, 0x38b, 0x5, 0xd8, 0x6d, 0x2, 0x386, 0x388, 0x7, 0xd0, + 0x2, 0x2, 0x387, 0x389, 0x5, 0xac, 0x57, 0x2, 0x388, 0x387, 0x3, 0x2, + 0x2, 0x2, 0x388, 0x389, 0x3, 0x2, 0x2, 0x2, 0x389, 0x38a, 0x3, 0x2, + 0x2, 0x2, 0x38a, 0x38c, 0x7, 0xda, 0x2, 0x2, 0x38b, 0x386, 0x3, 0x2, + 0x2, 0x2, 0x38b, 0x38c, 0x3, 0x2, 0x2, 0x2, 0x38c, 0x41, 0x3, 0x2, 0x2, + 0x2, 0x38d, 0x398, 0x5, 0x44, 0x23, 0x2, 0x38e, 0x38f, 0x7, 0x1e, 0x2, + 0x2, 0x38f, 0x390, 0x5, 0xd6, 0x6c, 0x2, 0x390, 0x391, 0x7, 0x17, 0x2, + 0x2, 0x391, 0x392, 0x5, 0xb0, 0x59, 0x2, 0x392, 0x398, 0x3, 0x2, 0x2, + 0x2, 0x393, 0x394, 0x7, 0x50, 0x2, 0x2, 0x394, 0x398, 0x5, 0x48, 0x25, + 0x2, 0x395, 0x396, 0x7, 0x80, 0x2, 0x2, 0x396, 0x398, 0x5, 0x4a, 0x26, + 0x2, 0x397, 0x38d, 0x3, 0x2, 0x2, 0x2, 0x397, 0x38e, 0x3, 0x2, 0x2, + 0x2, 0x397, 0x393, 0x3, 0x2, 0x2, 0x2, 0x397, 0x395, 0x3, 0x2, 0x2, + 0x2, 0x398, 0x43, 0x3, 0x2, 0x2, 0x2, 0x399, 0x39a, 0x5, 0xba, 0x5e, + 0x2, 0x39a, 0x39c, 0x5, 0xaa, 0x56, 0x2, 0x39b, 0x39d, 0x5, 0x46, 0x24, + 0x2, 0x39c, 0x39b, 0x3, 0x2, 0x2, 0x2, 0x39c, 0x39d, 0x3, 0x2, 0x2, + 0x2, 0x39d, 0x3a0, 0x3, 0x2, 0x2, 0x2, 0x39e, 0x39f, 0x7, 0x1d, 0x2, 0x2, 0x39f, 0x3a1, 0x7, 0xbf, 0x2, 0x2, 0x3a0, 0x39e, 0x3, 0x2, 0x2, 0x2, 0x3a0, 0x3a1, 0x3, 0x2, 0x2, 0x2, 0x3a1, 0x3a3, 0x3, 0x2, 0x2, 0x2, 0x3a2, 0x3a4, 0x5, 0x4c, 0x27, 0x2, 0x3a3, 0x3a2, 0x3, 0x2, 0x2, 0x2, 0x3a3, 0x3a4, 0x3, 0x2, 0x2, 0x2, 0x3a4, 0x3a7, 0x3, 0x2, 0x2, 0x2, 0x3a5, 0x3a6, 0x7, 0xa8, 0x2, 0x2, 0x3a6, 0x3a8, 0x5, 0xb0, 0x59, 0x2, 0x3a7, 0x3a5, 0x3, 0x2, 0x2, 0x2, 0x3a7, 0x3a8, 0x3, 0x2, 0x2, - 0x2, 0x3a8, 0x3aa, 0x3, 0x2, 0x2, 0x2, 0x3a9, 0x389, 0x3, 0x2, 0x2, - 0x2, 0x3a9, 0x399, 0x3, 0x2, 0x2, 0x2, 0x3aa, 0x45, 0x3, 0x2, 0x2, 0x2, - 0x3ab, 0x3ac, 0x9, 0x4, 0x2, 0x2, 0x3ac, 0x3ad, 0x5, 0xb0, 0x59, 0x2, - 0x3ad, 0x47, 0x3, 0x2, 0x2, 0x2, 0x3ae, 0x3af, 0x5, 0xba, 0x5e, 0x2, - 0x3af, 0x3b0, 0x5, 0xb0, 0x59, 0x2, 0x3b0, 0x3b1, 0x7, 0xa9, 0x2, 0x2, - 0x3b1, 0x3b2, 0x5, 0xaa, 0x56, 0x2, 0x3b2, 0x3b3, 0x7, 0x47, 0x2, 0x2, - 0x3b3, 0x3b4, 0x7, 0xbd, 0x2, 0x2, 0x3b4, 0x49, 0x3, 0x2, 0x2, 0x2, - 0x3b5, 0x3b6, 0x5, 0xba, 0x5e, 0x2, 0x3b6, 0x3b7, 0x5, 0x66, 0x34, 0x2, - 0x3b7, 0x4b, 0x3, 0x2, 0x2, 0x2, 0x3b8, 0x3b9, 0x7, 0x1a, 0x2, 0x2, - 0x3b9, 0x3ba, 0x7, 0xd0, 0x2, 0x2, 0x3ba, 0x3bf, 0x5, 0x4e, 0x28, 0x2, - 0x3bb, 0x3bc, 0x7, 0xc5, 0x2, 0x2, 0x3bc, 0x3be, 0x5, 0x4e, 0x28, 0x2, - 0x3bd, 0x3bb, 0x3, 0x2, 0x2, 0x2, 0x3be, 0x3c1, 0x3, 0x2, 0x2, 0x2, - 0x3bf, 0x3bd, 0x3, 0x2, 0x2, 0x2, 0x3bf, 0x3c0, 0x3, 0x2, 0x2, 0x2, - 0x3c0, 0x3c2, 0x3, 0x2, 0x2, 0x2, 0x3c1, 0x3bf, 0x3, 0x2, 0x2, 0x2, - 0x3c2, 0x3c3, 0x7, 0xda, 0x2, 0x2, 0x3c3, 0x4d, 0x3, 0x2, 0x2, 0x2, - 0x3c4, 0x3ca, 0x5, 0xd6, 0x6c, 0x2, 0x3c5, 0x3c7, 0x7, 0xd0, 0x2, 0x2, - 0x3c6, 0x3c8, 0x5, 0xac, 0x57, 0x2, 0x3c7, 0x3c6, 0x3, 0x2, 0x2, 0x2, - 0x3c7, 0x3c8, 0x3, 0x2, 0x2, 0x2, 0x3c8, 0x3c9, 0x3, 0x2, 0x2, 0x2, - 0x3c9, 0x3cb, 0x7, 0xda, 0x2, 0x2, 0x3ca, 0x3c5, 0x3, 0x2, 0x2, 0x2, - 0x3ca, 0x3cb, 0x3, 0x2, 0x2, 0x2, 0x3cb, 0x4f, 0x3, 0x2, 0x2, 0x2, 0x3cc, - 0x3d4, 0x5, 0xb0, 0x59, 0x2, 0x3cd, 0x3d5, 0x7, 0x29, 0x2, 0x2, 0x3ce, - 0x3cf, 0x7, 0xa2, 0x2, 0x2, 0x3cf, 0x3d0, 0x7, 0x30, 0x2, 0x2, 0x3d0, - 0x3d5, 0x7, 0xbf, 0x2, 0x2, 0x3d1, 0x3d2, 0x7, 0xa2, 0x2, 0x2, 0x3d2, - 0x3d3, 0x7, 0xb1, 0x2, 0x2, 0x3d3, 0x3d5, 0x7, 0xbf, 0x2, 0x2, 0x3d4, - 0x3cd, 0x3, 0x2, 0x2, 0x2, 0x3d4, 0x3ce, 0x3, 0x2, 0x2, 0x2, 0x3d4, - 0x3d1, 0x3, 0x2, 0x2, 0x2, 0x3d4, 0x3d5, 0x3, 0x2, 0x2, 0x2, 0x3d5, - 0x51, 0x3, 0x2, 0x2, 0x2, 0x3d6, 0x3d8, 0x9, 0x5, 0x2, 0x2, 0x3d7, 0x3d9, - 0x7, 0x9a, 0x2, 0x2, 0x3d8, 0x3d7, 0x3, 0x2, 0x2, 0x2, 0x3d8, 0x3d9, - 0x3, 0x2, 0x2, 0x2, 0x3d9, 0x3da, 0x3, 0x2, 0x2, 0x2, 0x3da, 0x3db, - 0x5, 0xbc, 0x5f, 0x2, 0x3db, 0x53, 0x3, 0x2, 0x2, 0x2, 0x3dc, 0x3dd, - 0x9, 0x6, 0x2, 0x2, 0x3dd, 0x3e0, 0x7, 0x22, 0x2, 0x2, 0x3de, 0x3df, - 0x7, 0x4d, 0x2, 0x2, 0x3df, 0x3e1, 0x7, 0x38, 0x2, 0x2, 0x3e0, 0x3de, - 0x3, 0x2, 0x2, 0x2, 0x3e0, 0x3e1, 0x3, 0x2, 0x2, 0x2, 0x3e1, 0x3e2, - 0x3, 0x2, 0x2, 0x2, 0x3e2, 0x3e4, 0x5, 0xc6, 0x64, 0x2, 0x3e3, 0x3e5, - 0x5, 0x2c, 0x17, 0x2, 0x3e4, 0x3e3, 0x3, 0x2, 0x2, 0x2, 0x3e4, 0x3e5, - 0x3, 0x2, 0x2, 0x2, 0x3e5, 0x3fc, 0x3, 0x2, 0x2, 0x2, 0x3e6, 0x3ed, - 0x9, 0x6, 0x2, 0x2, 0x3e7, 0x3ee, 0x7, 0x2f, 0x2, 0x2, 0x3e8, 0x3ea, - 0x7, 0x9c, 0x2, 0x2, 0x3e9, 0x3e8, 0x3, 0x2, 0x2, 0x2, 0x3e9, 0x3ea, - 0x3, 0x2, 0x2, 0x2, 0x3ea, 0x3eb, 0x3, 0x2, 0x2, 0x2, 0x3eb, 0x3ee, - 0x7, 0x9a, 0x2, 0x2, 0x3ec, 0x3ee, 0x7, 0xb0, 0x2, 0x2, 0x3ed, 0x3e7, - 0x3, 0x2, 0x2, 0x2, 0x3ed, 0x3e9, 0x3, 0x2, 0x2, 0x2, 0x3ed, 0x3ec, - 0x3, 0x2, 0x2, 0x2, 0x3ee, 0x3f1, 0x3, 0x2, 0x2, 0x2, 0x3ef, 0x3f0, - 0x7, 0x4d, 0x2, 0x2, 0x3f0, 0x3f2, 0x7, 0x38, 0x2, 0x2, 0x3f1, 0x3ef, - 0x3, 0x2, 0x2, 0x2, 0x3f1, 0x3f2, 0x3, 0x2, 0x2, 0x2, 0x3f2, 0x3f3, - 0x3, 0x2, 0x2, 0x2, 0x3f3, 0x3f5, 0x5, 0xc0, 0x61, 0x2, 0x3f4, 0x3f6, - 0x5, 0x2c, 0x17, 0x2, 0x3f5, 0x3f4, 0x3, 0x2, 0x2, 0x2, 0x3f5, 0x3f6, - 0x3, 0x2, 0x2, 0x2, 0x3f6, 0x3f9, 0x3, 0x2, 0x2, 0x2, 0x3f7, 0x3f8, - 0x7, 0x71, 0x2, 0x2, 0x3f8, 0x3fa, 0x7, 0x28, 0x2, 0x2, 0x3f9, 0x3f7, - 0x3, 0x2, 0x2, 0x2, 0x3f9, 0x3fa, 0x3, 0x2, 0x2, 0x2, 0x3fa, 0x3fc, - 0x3, 0x2, 0x2, 0x2, 0x3fb, 0x3dc, 0x3, 0x2, 0x2, 0x2, 0x3fb, 0x3e6, - 0x3, 0x2, 0x2, 0x2, 0x3fc, 0x55, 0x3, 0x2, 0x2, 0x2, 0x3fd, 0x3fe, 0x7, - 0x38, 0x2, 0x2, 0x3fe, 0x3ff, 0x7, 0x22, 0x2, 0x2, 0x3ff, 0x40b, 0x5, - 0xc6, 0x64, 0x2, 0x400, 0x407, 0x7, 0x38, 0x2, 0x2, 0x401, 0x408, 0x7, - 0x2f, 0x2, 0x2, 0x402, 0x404, 0x7, 0x9c, 0x2, 0x2, 0x403, 0x402, 0x3, - 0x2, 0x2, 0x2, 0x403, 0x404, 0x3, 0x2, 0x2, 0x2, 0x404, 0x405, 0x3, - 0x2, 0x2, 0x2, 0x405, 0x408, 0x7, 0x9a, 0x2, 0x2, 0x406, 0x408, 0x7, - 0xb0, 0x2, 0x2, 0x407, 0x401, 0x3, 0x2, 0x2, 0x2, 0x407, 0x403, 0x3, - 0x2, 0x2, 0x2, 0x407, 0x406, 0x3, 0x2, 0x2, 0x2, 0x407, 0x408, 0x3, - 0x2, 0x2, 0x2, 0x408, 0x409, 0x3, 0x2, 0x2, 0x2, 0x409, 0x40b, 0x5, - 0xc0, 0x61, 0x2, 0x40a, 0x3fd, 0x3, 0x2, 0x2, 0x2, 0x40a, 0x400, 0x3, - 0x2, 0x2, 0x2, 0x40b, 0x57, 0x3, 0x2, 0x2, 0x2, 0x40c, 0x40d, 0x7, 0x39, - 0x2, 0x2, 0x40d, 0x40e, 0x7, 0xf, 0x2, 0x2, 0x40e, 0x413, 0x5, 0x4, - 0x3, 0x2, 0x40f, 0x410, 0x7, 0x39, 0x2, 0x2, 0x410, 0x411, 0x7, 0x98, - 0x2, 0x2, 0x411, 0x413, 0x5, 0x4, 0x3, 0x2, 0x412, 0x40c, 0x3, 0x2, - 0x2, 0x2, 0x412, 0x40f, 0x3, 0x2, 0x2, 0x2, 0x413, 0x59, 0x3, 0x2, 0x2, - 0x2, 0x414, 0x415, 0x7, 0x54, 0x2, 0x2, 0x415, 0x417, 0x7, 0x56, 0x2, - 0x2, 0x416, 0x418, 0x7, 0x9a, 0x2, 0x2, 0x417, 0x416, 0x3, 0x2, 0x2, - 0x2, 0x417, 0x418, 0x3, 0x2, 0x2, 0x2, 0x418, 0x41c, 0x3, 0x2, 0x2, - 0x2, 0x419, 0x41d, 0x5, 0xc0, 0x61, 0x2, 0x41a, 0x41b, 0x7, 0x45, 0x2, - 0x2, 0x41b, 0x41d, 0x5, 0xbe, 0x60, 0x2, 0x41c, 0x419, 0x3, 0x2, 0x2, - 0x2, 0x41c, 0x41a, 0x3, 0x2, 0x2, 0x2, 0x41d, 0x41f, 0x3, 0x2, 0x2, - 0x2, 0x41e, 0x420, 0x5, 0x5c, 0x2f, 0x2, 0x41f, 0x41e, 0x3, 0x2, 0x2, - 0x2, 0x41f, 0x420, 0x3, 0x2, 0x2, 0x2, 0x420, 0x421, 0x3, 0x2, 0x2, - 0x2, 0x421, 0x422, 0x5, 0x5e, 0x30, 0x2, 0x422, 0x5b, 0x3, 0x2, 0x2, - 0x2, 0x423, 0x424, 0x7, 0xd0, 0x2, 0x2, 0x424, 0x429, 0x5, 0xba, 0x5e, - 0x2, 0x425, 0x426, 0x7, 0xc5, 0x2, 0x2, 0x426, 0x428, 0x5, 0xba, 0x5e, - 0x2, 0x427, 0x425, 0x3, 0x2, 0x2, 0x2, 0x428, 0x42b, 0x3, 0x2, 0x2, - 0x2, 0x429, 0x427, 0x3, 0x2, 0x2, 0x2, 0x429, 0x42a, 0x3, 0x2, 0x2, - 0x2, 0x42a, 0x42c, 0x3, 0x2, 0x2, 0x2, 0x42b, 0x429, 0x3, 0x2, 0x2, - 0x2, 0x42c, 0x42d, 0x7, 0xda, 0x2, 0x2, 0x42d, 0x5d, 0x3, 0x2, 0x2, - 0x2, 0x42e, 0x42f, 0x7, 0x41, 0x2, 0x2, 0x42f, 0x438, 0x5, 0xd6, 0x6c, - 0x2, 0x430, 0x438, 0x7, 0xaf, 0x2, 0x2, 0x431, 0x433, 0x5, 0x68, 0x35, - 0x2, 0x432, 0x434, 0x7, 0xdb, 0x2, 0x2, 0x433, 0x432, 0x3, 0x2, 0x2, - 0x2, 0x433, 0x434, 0x3, 0x2, 0x2, 0x2, 0x434, 0x435, 0x3, 0x2, 0x2, - 0x2, 0x435, 0x436, 0x7, 0x2, 0x2, 0x3, 0x436, 0x438, 0x3, 0x2, 0x2, - 0x2, 0x437, 0x42e, 0x3, 0x2, 0x2, 0x2, 0x437, 0x430, 0x3, 0x2, 0x2, - 0x2, 0x437, 0x431, 0x3, 0x2, 0x2, 0x2, 0x438, 0x5f, 0x3, 0x2, 0x2, 0x2, - 0x439, 0x43a, 0x7, 0x5b, 0x2, 0x2, 0x43a, 0x43c, 0x7, 0x6f, 0x2, 0x2, - 0x43b, 0x43d, 0x5, 0x2c, 0x17, 0x2, 0x43c, 0x43b, 0x3, 0x2, 0x2, 0x2, - 0x43c, 0x43d, 0x3, 0x2, 0x2, 0x2, 0x43d, 0x43e, 0x3, 0x2, 0x2, 0x2, - 0x43e, 0x440, 0x5, 0x78, 0x3d, 0x2, 0x43f, 0x441, 0x9, 0x7, 0x2, 0x2, - 0x440, 0x43f, 0x3, 0x2, 0x2, 0x2, 0x440, 0x441, 0x3, 0x2, 0x2, 0x2, - 0x441, 0x61, 0x3, 0x2, 0x2, 0x2, 0x442, 0x443, 0x7, 0x77, 0x2, 0x2, - 0x443, 0x444, 0x7, 0x9a, 0x2, 0x2, 0x444, 0x446, 0x5, 0xc0, 0x61, 0x2, - 0x445, 0x447, 0x5, 0x2c, 0x17, 0x2, 0x446, 0x445, 0x3, 0x2, 0x2, 0x2, - 0x446, 0x447, 0x3, 0x2, 0x2, 0x2, 0x447, 0x449, 0x3, 0x2, 0x2, 0x2, - 0x448, 0x44a, 0x5, 0x10, 0x9, 0x2, 0x449, 0x448, 0x3, 0x2, 0x2, 0x2, - 0x449, 0x44a, 0x3, 0x2, 0x2, 0x2, 0x44a, 0x44c, 0x3, 0x2, 0x2, 0x2, - 0x44b, 0x44d, 0x7, 0x3d, 0x2, 0x2, 0x44c, 0x44b, 0x3, 0x2, 0x2, 0x2, - 0x44c, 0x44d, 0x3, 0x2, 0x2, 0x2, 0x44d, 0x44f, 0x3, 0x2, 0x2, 0x2, - 0x44e, 0x450, 0x7, 0x26, 0x2, 0x2, 0x44f, 0x44e, 0x3, 0x2, 0x2, 0x2, - 0x44f, 0x450, 0x3, 0x2, 0x2, 0x2, 0x450, 0x63, 0x3, 0x2, 0x2, 0x2, 0x451, - 0x452, 0x7, 0x85, 0x2, 0x2, 0x452, 0x453, 0x7, 0x9a, 0x2, 0x2, 0x453, - 0x454, 0x5, 0xc0, 0x61, 0x2, 0x454, 0x455, 0x7, 0xa2, 0x2, 0x2, 0x455, - 0x45d, 0x5, 0xc0, 0x61, 0x2, 0x456, 0x457, 0x7, 0xc5, 0x2, 0x2, 0x457, - 0x458, 0x5, 0xc0, 0x61, 0x2, 0x458, 0x459, 0x7, 0xa2, 0x2, 0x2, 0x459, - 0x45a, 0x5, 0xc0, 0x61, 0x2, 0x45a, 0x45c, 0x3, 0x2, 0x2, 0x2, 0x45b, - 0x456, 0x3, 0x2, 0x2, 0x2, 0x45c, 0x45f, 0x3, 0x2, 0x2, 0x2, 0x45d, - 0x45b, 0x3, 0x2, 0x2, 0x2, 0x45d, 0x45e, 0x3, 0x2, 0x2, 0x2, 0x45e, - 0x461, 0x3, 0x2, 0x2, 0x2, 0x45f, 0x45d, 0x3, 0x2, 0x2, 0x2, 0x460, - 0x462, 0x5, 0x2c, 0x17, 0x2, 0x461, 0x460, 0x3, 0x2, 0x2, 0x2, 0x461, - 0x462, 0x3, 0x2, 0x2, 0x2, 0x462, 0x65, 0x3, 0x2, 0x2, 0x2, 0x463, 0x465, - 0x7, 0xd0, 0x2, 0x2, 0x464, 0x466, 0x5, 0x6e, 0x38, 0x2, 0x465, 0x464, - 0x3, 0x2, 0x2, 0x2, 0x465, 0x466, 0x3, 0x2, 0x2, 0x2, 0x466, 0x467, - 0x3, 0x2, 0x2, 0x2, 0x467, 0x468, 0x7, 0x8d, 0x2, 0x2, 0x468, 0x46a, - 0x5, 0xac, 0x57, 0x2, 0x469, 0x46b, 0x5, 0x7a, 0x3e, 0x2, 0x46a, 0x469, - 0x3, 0x2, 0x2, 0x2, 0x46a, 0x46b, 0x3, 0x2, 0x2, 0x2, 0x46b, 0x46d, - 0x3, 0x2, 0x2, 0x2, 0x46c, 0x46e, 0x5, 0x80, 0x41, 0x2, 0x46d, 0x46c, - 0x3, 0x2, 0x2, 0x2, 0x46d, 0x46e, 0x3, 0x2, 0x2, 0x2, 0x46e, 0x46f, - 0x3, 0x2, 0x2, 0x2, 0x46f, 0x470, 0x7, 0xda, 0x2, 0x2, 0x470, 0x67, - 0x3, 0x2, 0x2, 0x2, 0x471, 0x477, 0x5, 0x6a, 0x36, 0x2, 0x472, 0x473, - 0x7, 0xaa, 0x2, 0x2, 0x473, 0x474, 0x7, 0x6, 0x2, 0x2, 0x474, 0x476, - 0x5, 0x6a, 0x36, 0x2, 0x475, 0x472, 0x3, 0x2, 0x2, 0x2, 0x476, 0x479, - 0x3, 0x2, 0x2, 0x2, 0x477, 0x475, 0x3, 0x2, 0x2, 0x2, 0x477, 0x478, - 0x3, 0x2, 0x2, 0x2, 0x478, 0x69, 0x3, 0x2, 0x2, 0x2, 0x479, 0x477, 0x3, - 0x2, 0x2, 0x2, 0x47a, 0x480, 0x5, 0x6c, 0x37, 0x2, 0x47b, 0x47c, 0x7, - 0xd0, 0x2, 0x2, 0x47c, 0x47d, 0x5, 0x68, 0x35, 0x2, 0x47d, 0x47e, 0x7, - 0xda, 0x2, 0x2, 0x47e, 0x480, 0x3, 0x2, 0x2, 0x2, 0x47f, 0x47a, 0x3, - 0x2, 0x2, 0x2, 0x47f, 0x47b, 0x3, 0x2, 0x2, 0x2, 0x480, 0x6b, 0x3, 0x2, - 0x2, 0x2, 0x481, 0x483, 0x5, 0x6e, 0x38, 0x2, 0x482, 0x481, 0x3, 0x2, - 0x2, 0x2, 0x482, 0x483, 0x3, 0x2, 0x2, 0x2, 0x483, 0x484, 0x3, 0x2, - 0x2, 0x2, 0x484, 0x486, 0x7, 0x8d, 0x2, 0x2, 0x485, 0x487, 0x7, 0x31, - 0x2, 0x2, 0x486, 0x485, 0x3, 0x2, 0x2, 0x2, 0x486, 0x487, 0x3, 0x2, - 0x2, 0x2, 0x487, 0x489, 0x3, 0x2, 0x2, 0x2, 0x488, 0x48a, 0x5, 0x70, - 0x39, 0x2, 0x489, 0x488, 0x3, 0x2, 0x2, 0x2, 0x489, 0x48a, 0x3, 0x2, - 0x2, 0x2, 0x48a, 0x48b, 0x3, 0x2, 0x2, 0x2, 0x48b, 0x48d, 0x5, 0xac, - 0x57, 0x2, 0x48c, 0x48e, 0x5, 0x72, 0x3a, 0x2, 0x48d, 0x48c, 0x3, 0x2, - 0x2, 0x2, 0x48d, 0x48e, 0x3, 0x2, 0x2, 0x2, 0x48e, 0x490, 0x3, 0x2, - 0x2, 0x2, 0x48f, 0x491, 0x5, 0x74, 0x3b, 0x2, 0x490, 0x48f, 0x3, 0x2, - 0x2, 0x2, 0x490, 0x491, 0x3, 0x2, 0x2, 0x2, 0x491, 0x493, 0x3, 0x2, - 0x2, 0x2, 0x492, 0x494, 0x5, 0x76, 0x3c, 0x2, 0x493, 0x492, 0x3, 0x2, - 0x2, 0x2, 0x493, 0x494, 0x3, 0x2, 0x2, 0x2, 0x494, 0x496, 0x3, 0x2, - 0x2, 0x2, 0x495, 0x497, 0x5, 0x78, 0x3d, 0x2, 0x496, 0x495, 0x3, 0x2, - 0x2, 0x2, 0x496, 0x497, 0x3, 0x2, 0x2, 0x2, 0x497, 0x499, 0x3, 0x2, - 0x2, 0x2, 0x498, 0x49a, 0x5, 0x7a, 0x3e, 0x2, 0x499, 0x498, 0x3, 0x2, - 0x2, 0x2, 0x499, 0x49a, 0x3, 0x2, 0x2, 0x2, 0x49a, 0x49d, 0x3, 0x2, - 0x2, 0x2, 0x49b, 0x49c, 0x7, 0xb6, 0x2, 0x2, 0x49c, 0x49e, 0x9, 0x8, - 0x2, 0x2, 0x49d, 0x49b, 0x3, 0x2, 0x2, 0x2, 0x49d, 0x49e, 0x3, 0x2, - 0x2, 0x2, 0x49e, 0x4a1, 0x3, 0x2, 0x2, 0x2, 0x49f, 0x4a0, 0x7, 0xb6, - 0x2, 0x2, 0x4a0, 0x4a2, 0x7, 0xa4, 0x2, 0x2, 0x4a1, 0x49f, 0x3, 0x2, - 0x2, 0x2, 0x4a1, 0x4a2, 0x3, 0x2, 0x2, 0x2, 0x4a2, 0x4a4, 0x3, 0x2, - 0x2, 0x2, 0x4a3, 0x4a5, 0x5, 0x7c, 0x3f, 0x2, 0x4a4, 0x4a3, 0x3, 0x2, - 0x2, 0x2, 0x4a4, 0x4a5, 0x3, 0x2, 0x2, 0x2, 0x4a5, 0x4a7, 0x3, 0x2, - 0x2, 0x2, 0x4a6, 0x4a8, 0x5, 0x7e, 0x40, 0x2, 0x4a7, 0x4a6, 0x3, 0x2, - 0x2, 0x2, 0x4a7, 0x4a8, 0x3, 0x2, 0x2, 0x2, 0x4a8, 0x4aa, 0x3, 0x2, - 0x2, 0x2, 0x4a9, 0x4ab, 0x5, 0x82, 0x42, 0x2, 0x4aa, 0x4a9, 0x3, 0x2, - 0x2, 0x2, 0x4aa, 0x4ab, 0x3, 0x2, 0x2, 0x2, 0x4ab, 0x4ad, 0x3, 0x2, - 0x2, 0x2, 0x4ac, 0x4ae, 0x5, 0x84, 0x43, 0x2, 0x4ad, 0x4ac, 0x3, 0x2, - 0x2, 0x2, 0x4ad, 0x4ae, 0x3, 0x2, 0x2, 0x2, 0x4ae, 0x4b0, 0x3, 0x2, - 0x2, 0x2, 0x4af, 0x4b1, 0x5, 0x86, 0x44, 0x2, 0x4b0, 0x4af, 0x3, 0x2, - 0x2, 0x2, 0x4b0, 0x4b1, 0x3, 0x2, 0x2, 0x2, 0x4b1, 0x6d, 0x3, 0x2, 0x2, - 0x2, 0x4b2, 0x4b3, 0x7, 0xb6, 0x2, 0x2, 0x4b3, 0x4b4, 0x5, 0xac, 0x57, - 0x2, 0x4b4, 0x6f, 0x3, 0x2, 0x2, 0x2, 0x4b5, 0x4b6, 0x7, 0xa3, 0x2, - 0x2, 0x4b6, 0x4b9, 0x7, 0xbd, 0x2, 0x2, 0x4b7, 0x4b8, 0x7, 0xb6, 0x2, - 0x2, 0x4b8, 0x4ba, 0x7, 0x9f, 0x2, 0x2, 0x4b9, 0x4b7, 0x3, 0x2, 0x2, - 0x2, 0x4b9, 0x4ba, 0x3, 0x2, 0x2, 0x2, 0x4ba, 0x71, 0x3, 0x2, 0x2, 0x2, - 0x4bb, 0x4bc, 0x7, 0x43, 0x2, 0x2, 0x4bc, 0x4bd, 0x5, 0x88, 0x45, 0x2, - 0x4bd, 0x73, 0x3, 0x2, 0x2, 0x2, 0x4be, 0x4c0, 0x9, 0x9, 0x2, 0x2, 0x4bf, - 0x4be, 0x3, 0x2, 0x2, 0x2, 0x4bf, 0x4c0, 0x3, 0x2, 0x2, 0x2, 0x4c0, - 0x4c1, 0x3, 0x2, 0x2, 0x2, 0x4c1, 0x4c2, 0x7, 0xb, 0x2, 0x2, 0x4c2, - 0x4c3, 0x7, 0x59, 0x2, 0x2, 0x4c3, 0x4c4, 0x5, 0xac, 0x57, 0x2, 0x4c4, - 0x75, 0x3, 0x2, 0x2, 0x2, 0x4c5, 0x4c6, 0x7, 0x7e, 0x2, 0x2, 0x4c6, - 0x4c7, 0x5, 0xb0, 0x59, 0x2, 0x4c7, 0x77, 0x3, 0x2, 0x2, 0x2, 0x4c8, - 0x4c9, 0x7, 0xb5, 0x2, 0x2, 0x4c9, 0x4ca, 0x5, 0xb0, 0x59, 0x2, 0x4ca, - 0x79, 0x3, 0x2, 0x2, 0x2, 0x4cb, 0x4cc, 0x7, 0x48, 0x2, 0x2, 0x4cc, - 0x4d3, 0x7, 0x14, 0x2, 0x2, 0x4cd, 0x4ce, 0x9, 0x8, 0x2, 0x2, 0x4ce, - 0x4cf, 0x7, 0xd0, 0x2, 0x2, 0x4cf, 0x4d0, 0x5, 0xac, 0x57, 0x2, 0x4d0, - 0x4d1, 0x7, 0xda, 0x2, 0x2, 0x4d1, 0x4d4, 0x3, 0x2, 0x2, 0x2, 0x4d2, - 0x4d4, 0x5, 0xac, 0x57, 0x2, 0x4d3, 0x4cd, 0x3, 0x2, 0x2, 0x2, 0x4d3, - 0x4d2, 0x3, 0x2, 0x2, 0x2, 0x4d4, 0x7b, 0x3, 0x2, 0x2, 0x2, 0x4d5, 0x4d6, - 0x7, 0x49, 0x2, 0x2, 0x4d6, 0x4d7, 0x5, 0xb0, 0x59, 0x2, 0x4d7, 0x7d, - 0x3, 0x2, 0x2, 0x2, 0x4d8, 0x4d9, 0x7, 0x79, 0x2, 0x2, 0x4d9, 0x4da, - 0x7, 0x14, 0x2, 0x2, 0x4da, 0x4db, 0x5, 0x94, 0x4b, 0x2, 0x4db, 0x7f, - 0x3, 0x2, 0x2, 0x2, 0x4dc, 0x4dd, 0x7, 0x79, 0x2, 0x2, 0x4dd, 0x4de, - 0x7, 0x14, 0x2, 0x2, 0x4de, 0x4df, 0x5, 0xac, 0x57, 0x2, 0x4df, 0x81, - 0x3, 0x2, 0x2, 0x2, 0x4e0, 0x4e1, 0x7, 0x62, 0x2, 0x2, 0x4e1, 0x4e2, - 0x5, 0x92, 0x4a, 0x2, 0x4e2, 0x4e3, 0x7, 0x14, 0x2, 0x2, 0x4e3, 0x4e4, - 0x5, 0xac, 0x57, 0x2, 0x4e4, 0x83, 0x3, 0x2, 0x2, 0x2, 0x4e5, 0x4e6, - 0x7, 0x62, 0x2, 0x2, 0x4e6, 0x4e9, 0x5, 0x92, 0x4a, 0x2, 0x4e7, 0x4e8, - 0x7, 0xb6, 0x2, 0x2, 0x4e8, 0x4ea, 0x7, 0x9f, 0x2, 0x2, 0x4e9, 0x4e7, - 0x3, 0x2, 0x2, 0x2, 0x4e9, 0x4ea, 0x3, 0x2, 0x2, 0x2, 0x4ea, 0x85, 0x3, - 0x2, 0x2, 0x2, 0x4eb, 0x4ec, 0x7, 0x91, 0x2, 0x2, 0x4ec, 0x4ed, 0x5, - 0x9a, 0x4e, 0x2, 0x4ed, 0x87, 0x3, 0x2, 0x2, 0x2, 0x4ee, 0x4ef, 0x8, - 0x45, 0x1, 0x2, 0x4ef, 0x4f1, 0x5, 0xbc, 0x5f, 0x2, 0x4f0, 0x4f2, 0x7, - 0x3d, 0x2, 0x2, 0x4f1, 0x4f0, 0x3, 0x2, 0x2, 0x2, 0x4f1, 0x4f2, 0x3, - 0x2, 0x2, 0x2, 0x4f2, 0x4f4, 0x3, 0x2, 0x2, 0x2, 0x4f3, 0x4f5, 0x5, - 0x90, 0x49, 0x2, 0x4f4, 0x4f3, 0x3, 0x2, 0x2, 0x2, 0x4f4, 0x4f5, 0x3, - 0x2, 0x2, 0x2, 0x4f5, 0x4fb, 0x3, 0x2, 0x2, 0x2, 0x4f6, 0x4f7, 0x7, - 0xd0, 0x2, 0x2, 0x4f7, 0x4f8, 0x5, 0x88, 0x45, 0x2, 0x4f8, 0x4f9, 0x7, - 0xda, 0x2, 0x2, 0x4f9, 0x4fb, 0x3, 0x2, 0x2, 0x2, 0x4fa, 0x4ee, 0x3, - 0x2, 0x2, 0x2, 0x4fa, 0x4f6, 0x3, 0x2, 0x2, 0x2, 0x4fb, 0x50d, 0x3, - 0x2, 0x2, 0x2, 0x4fc, 0x4fd, 0xc, 0x5, 0x2, 0x2, 0x4fd, 0x4fe, 0x5, - 0x8c, 0x47, 0x2, 0x4fe, 0x4ff, 0x5, 0x88, 0x45, 0x6, 0x4ff, 0x50c, 0x3, - 0x2, 0x2, 0x2, 0x500, 0x502, 0xc, 0x6, 0x2, 0x2, 0x501, 0x503, 0x9, - 0xa, 0x2, 0x2, 0x502, 0x501, 0x3, 0x2, 0x2, 0x2, 0x502, 0x503, 0x3, - 0x2, 0x2, 0x2, 0x503, 0x505, 0x3, 0x2, 0x2, 0x2, 0x504, 0x506, 0x5, - 0x8a, 0x46, 0x2, 0x505, 0x504, 0x3, 0x2, 0x2, 0x2, 0x505, 0x506, 0x3, - 0x2, 0x2, 0x2, 0x506, 0x507, 0x3, 0x2, 0x2, 0x2, 0x507, 0x508, 0x7, - 0x59, 0x2, 0x2, 0x508, 0x509, 0x5, 0x88, 0x45, 0x2, 0x509, 0x50a, 0x5, - 0x8e, 0x48, 0x2, 0x50a, 0x50c, 0x3, 0x2, 0x2, 0x2, 0x50b, 0x4fc, 0x3, - 0x2, 0x2, 0x2, 0x50b, 0x500, 0x3, 0x2, 0x2, 0x2, 0x50c, 0x50f, 0x3, - 0x2, 0x2, 0x2, 0x50d, 0x50b, 0x3, 0x2, 0x2, 0x2, 0x50d, 0x50e, 0x3, - 0x2, 0x2, 0x2, 0x50e, 0x89, 0x3, 0x2, 0x2, 0x2, 0x50f, 0x50d, 0x3, 0x2, - 0x2, 0x2, 0x510, 0x512, 0x9, 0xb, 0x2, 0x2, 0x511, 0x510, 0x3, 0x2, - 0x2, 0x2, 0x511, 0x512, 0x3, 0x2, 0x2, 0x2, 0x512, 0x513, 0x3, 0x2, - 0x2, 0x2, 0x513, 0x51a, 0x7, 0x53, 0x2, 0x2, 0x514, 0x516, 0x7, 0x53, - 0x2, 0x2, 0x515, 0x517, 0x9, 0xb, 0x2, 0x2, 0x516, 0x515, 0x3, 0x2, - 0x2, 0x2, 0x516, 0x517, 0x3, 0x2, 0x2, 0x2, 0x517, 0x51a, 0x3, 0x2, - 0x2, 0x2, 0x518, 0x51a, 0x9, 0xb, 0x2, 0x2, 0x519, 0x511, 0x3, 0x2, - 0x2, 0x2, 0x519, 0x514, 0x3, 0x2, 0x2, 0x2, 0x519, 0x518, 0x3, 0x2, - 0x2, 0x2, 0x51a, 0x53c, 0x3, 0x2, 0x2, 0x2, 0x51b, 0x51d, 0x9, 0xc, - 0x2, 0x2, 0x51c, 0x51b, 0x3, 0x2, 0x2, 0x2, 0x51c, 0x51d, 0x3, 0x2, - 0x2, 0x2, 0x51d, 0x51e, 0x3, 0x2, 0x2, 0x2, 0x51e, 0x520, 0x9, 0xd, - 0x2, 0x2, 0x51f, 0x521, 0x7, 0x7a, 0x2, 0x2, 0x520, 0x51f, 0x3, 0x2, - 0x2, 0x2, 0x520, 0x521, 0x3, 0x2, 0x2, 0x2, 0x521, 0x52a, 0x3, 0x2, - 0x2, 0x2, 0x522, 0x524, 0x9, 0xd, 0x2, 0x2, 0x523, 0x525, 0x7, 0x7a, - 0x2, 0x2, 0x524, 0x523, 0x3, 0x2, 0x2, 0x2, 0x524, 0x525, 0x3, 0x2, - 0x2, 0x2, 0x525, 0x527, 0x3, 0x2, 0x2, 0x2, 0x526, 0x528, 0x9, 0xc, - 0x2, 0x2, 0x527, 0x526, 0x3, 0x2, 0x2, 0x2, 0x527, 0x528, 0x3, 0x2, - 0x2, 0x2, 0x528, 0x52a, 0x3, 0x2, 0x2, 0x2, 0x529, 0x51c, 0x3, 0x2, - 0x2, 0x2, 0x529, 0x522, 0x3, 0x2, 0x2, 0x2, 0x52a, 0x53c, 0x3, 0x2, - 0x2, 0x2, 0x52b, 0x52d, 0x9, 0xe, 0x2, 0x2, 0x52c, 0x52b, 0x3, 0x2, - 0x2, 0x2, 0x52c, 0x52d, 0x3, 0x2, 0x2, 0x2, 0x52d, 0x52e, 0x3, 0x2, - 0x2, 0x2, 0x52e, 0x530, 0x7, 0x44, 0x2, 0x2, 0x52f, 0x531, 0x7, 0x7a, - 0x2, 0x2, 0x530, 0x52f, 0x3, 0x2, 0x2, 0x2, 0x530, 0x531, 0x3, 0x2, - 0x2, 0x2, 0x531, 0x53a, 0x3, 0x2, 0x2, 0x2, 0x532, 0x534, 0x7, 0x44, - 0x2, 0x2, 0x533, 0x535, 0x7, 0x7a, 0x2, 0x2, 0x534, 0x533, 0x3, 0x2, - 0x2, 0x2, 0x534, 0x535, 0x3, 0x2, 0x2, 0x2, 0x535, 0x537, 0x3, 0x2, - 0x2, 0x2, 0x536, 0x538, 0x9, 0xe, 0x2, 0x2, 0x537, 0x536, 0x3, 0x2, - 0x2, 0x2, 0x537, 0x538, 0x3, 0x2, 0x2, 0x2, 0x538, 0x53a, 0x3, 0x2, - 0x2, 0x2, 0x539, 0x52c, 0x3, 0x2, 0x2, 0x2, 0x539, 0x532, 0x3, 0x2, - 0x2, 0x2, 0x53a, 0x53c, 0x3, 0x2, 0x2, 0x2, 0x53b, 0x519, 0x3, 0x2, - 0x2, 0x2, 0x53b, 0x529, 0x3, 0x2, 0x2, 0x2, 0x53b, 0x539, 0x3, 0x2, - 0x2, 0x2, 0x53c, 0x8b, 0x3, 0x2, 0x2, 0x2, 0x53d, 0x53f, 0x9, 0xa, 0x2, - 0x2, 0x53e, 0x53d, 0x3, 0x2, 0x2, 0x2, 0x53e, 0x53f, 0x3, 0x2, 0x2, - 0x2, 0x53f, 0x540, 0x3, 0x2, 0x2, 0x2, 0x540, 0x541, 0x7, 0x20, 0x2, - 0x2, 0x541, 0x544, 0x7, 0x59, 0x2, 0x2, 0x542, 0x544, 0x7, 0xc5, 0x2, - 0x2, 0x543, 0x53e, 0x3, 0x2, 0x2, 0x2, 0x543, 0x542, 0x3, 0x2, 0x2, - 0x2, 0x544, 0x8d, 0x3, 0x2, 0x2, 0x2, 0x545, 0x546, 0x7, 0x76, 0x2, - 0x2, 0x546, 0x54f, 0x5, 0xac, 0x57, 0x2, 0x547, 0x548, 0x7, 0xad, 0x2, - 0x2, 0x548, 0x549, 0x7, 0xd0, 0x2, 0x2, 0x549, 0x54a, 0x5, 0xac, 0x57, - 0x2, 0x54a, 0x54b, 0x7, 0xda, 0x2, 0x2, 0x54b, 0x54f, 0x3, 0x2, 0x2, - 0x2, 0x54c, 0x54d, 0x7, 0xad, 0x2, 0x2, 0x54d, 0x54f, 0x5, 0xac, 0x57, - 0x2, 0x54e, 0x545, 0x3, 0x2, 0x2, 0x2, 0x54e, 0x547, 0x3, 0x2, 0x2, - 0x2, 0x54e, 0x54c, 0x3, 0x2, 0x2, 0x2, 0x54f, 0x8f, 0x3, 0x2, 0x2, 0x2, - 0x550, 0x551, 0x7, 0x8b, 0x2, 0x2, 0x551, 0x554, 0x5, 0x98, 0x4d, 0x2, - 0x552, 0x553, 0x7, 0x75, 0x2, 0x2, 0x553, 0x555, 0x5, 0x98, 0x4d, 0x2, - 0x554, 0x552, 0x3, 0x2, 0x2, 0x2, 0x554, 0x555, 0x3, 0x2, 0x2, 0x2, - 0x555, 0x91, 0x3, 0x2, 0x2, 0x2, 0x556, 0x559, 0x5, 0xb0, 0x59, 0x2, - 0x557, 0x558, 0x9, 0xf, 0x2, 0x2, 0x558, 0x55a, 0x5, 0xb0, 0x59, 0x2, - 0x559, 0x557, 0x3, 0x2, 0x2, 0x2, 0x559, 0x55a, 0x3, 0x2, 0x2, 0x2, - 0x55a, 0x93, 0x3, 0x2, 0x2, 0x2, 0x55b, 0x560, 0x5, 0x96, 0x4c, 0x2, - 0x55c, 0x55d, 0x7, 0xc5, 0x2, 0x2, 0x55d, 0x55f, 0x5, 0x96, 0x4c, 0x2, - 0x55e, 0x55c, 0x3, 0x2, 0x2, 0x2, 0x55f, 0x562, 0x3, 0x2, 0x2, 0x2, - 0x560, 0x55e, 0x3, 0x2, 0x2, 0x2, 0x560, 0x561, 0x3, 0x2, 0x2, 0x2, - 0x561, 0x95, 0x3, 0x2, 0x2, 0x2, 0x562, 0x560, 0x3, 0x2, 0x2, 0x2, 0x563, - 0x565, 0x5, 0xb0, 0x59, 0x2, 0x564, 0x566, 0x9, 0x10, 0x2, 0x2, 0x565, - 0x564, 0x3, 0x2, 0x2, 0x2, 0x565, 0x566, 0x3, 0x2, 0x2, 0x2, 0x566, - 0x569, 0x3, 0x2, 0x2, 0x2, 0x567, 0x568, 0x7, 0x74, 0x2, 0x2, 0x568, - 0x56a, 0x9, 0x11, 0x2, 0x2, 0x569, 0x567, 0x3, 0x2, 0x2, 0x2, 0x569, - 0x56a, 0x3, 0x2, 0x2, 0x2, 0x56a, 0x56d, 0x3, 0x2, 0x2, 0x2, 0x56b, - 0x56c, 0x7, 0x1b, 0x2, 0x2, 0x56c, 0x56e, 0x7, 0xbf, 0x2, 0x2, 0x56d, - 0x56b, 0x3, 0x2, 0x2, 0x2, 0x56d, 0x56e, 0x3, 0x2, 0x2, 0x2, 0x56e, - 0x97, 0x3, 0x2, 0x2, 0x2, 0x56f, 0x572, 0x5, 0xca, 0x66, 0x2, 0x570, - 0x571, 0x7, 0xdc, 0x2, 0x2, 0x571, 0x573, 0x5, 0xca, 0x66, 0x2, 0x572, - 0x570, 0x3, 0x2, 0x2, 0x2, 0x572, 0x573, 0x3, 0x2, 0x2, 0x2, 0x573, - 0x99, 0x3, 0x2, 0x2, 0x2, 0x574, 0x579, 0x5, 0x9c, 0x4f, 0x2, 0x575, - 0x576, 0x7, 0xc5, 0x2, 0x2, 0x576, 0x578, 0x5, 0x9c, 0x4f, 0x2, 0x577, - 0x575, 0x3, 0x2, 0x2, 0x2, 0x578, 0x57b, 0x3, 0x2, 0x2, 0x2, 0x579, - 0x577, 0x3, 0x2, 0x2, 0x2, 0x579, 0x57a, 0x3, 0x2, 0x2, 0x2, 0x57a, - 0x9b, 0x3, 0x2, 0x2, 0x2, 0x57b, 0x579, 0x3, 0x2, 0x2, 0x2, 0x57c, 0x57d, - 0x5, 0xd6, 0x6c, 0x2, 0x57d, 0x57e, 0x7, 0xca, 0x2, 0x2, 0x57e, 0x57f, - 0x5, 0xcc, 0x67, 0x2, 0x57f, 0x9d, 0x3, 0x2, 0x2, 0x2, 0x580, 0x581, - 0x7, 0x90, 0x2, 0x2, 0x581, 0x582, 0x5, 0x9a, 0x4e, 0x2, 0x582, 0x9f, - 0x3, 0x2, 0x2, 0x2, 0x583, 0x584, 0x7, 0x92, 0x2, 0x2, 0x584, 0x585, - 0x7, 0x1f, 0x2, 0x2, 0x585, 0x586, 0x7, 0x22, 0x2, 0x2, 0x586, 0x5ae, - 0x5, 0xc6, 0x64, 0x2, 0x587, 0x588, 0x7, 0x92, 0x2, 0x2, 0x588, 0x589, - 0x7, 0x1f, 0x2, 0x2, 0x589, 0x58a, 0x7, 0x2f, 0x2, 0x2, 0x58a, 0x5ae, - 0x5, 0xc0, 0x61, 0x2, 0x58b, 0x58c, 0x7, 0x92, 0x2, 0x2, 0x58c, 0x58e, - 0x7, 0x1f, 0x2, 0x2, 0x58d, 0x58f, 0x7, 0x9c, 0x2, 0x2, 0x58e, 0x58d, - 0x3, 0x2, 0x2, 0x2, 0x58e, 0x58f, 0x3, 0x2, 0x2, 0x2, 0x58f, 0x591, - 0x3, 0x2, 0x2, 0x2, 0x590, 0x592, 0x7, 0x9a, 0x2, 0x2, 0x591, 0x590, - 0x3, 0x2, 0x2, 0x2, 0x591, 0x592, 0x3, 0x2, 0x2, 0x2, 0x592, 0x593, - 0x3, 0x2, 0x2, 0x2, 0x593, 0x5ae, 0x5, 0xc0, 0x61, 0x2, 0x594, 0x595, - 0x7, 0x92, 0x2, 0x2, 0x595, 0x5ae, 0x7, 0x23, 0x2, 0x2, 0x596, 0x597, - 0x7, 0x92, 0x2, 0x2, 0x597, 0x59a, 0x7, 0x2e, 0x2, 0x2, 0x598, 0x599, - 0x7, 0x43, 0x2, 0x2, 0x599, 0x59b, 0x5, 0xc6, 0x64, 0x2, 0x59a, 0x598, - 0x3, 0x2, 0x2, 0x2, 0x59a, 0x59b, 0x3, 0x2, 0x2, 0x2, 0x59b, 0x5ae, - 0x3, 0x2, 0x2, 0x2, 0x59c, 0x59e, 0x7, 0x92, 0x2, 0x2, 0x59d, 0x59f, - 0x7, 0x9c, 0x2, 0x2, 0x59e, 0x59d, 0x3, 0x2, 0x2, 0x2, 0x59e, 0x59f, - 0x3, 0x2, 0x2, 0x2, 0x59f, 0x5a0, 0x3, 0x2, 0x2, 0x2, 0x5a0, 0x5a3, - 0x7, 0x9b, 0x2, 0x2, 0x5a1, 0x5a2, 0x9, 0x12, 0x2, 0x2, 0x5a2, 0x5a4, - 0x5, 0xc6, 0x64, 0x2, 0x5a3, 0x5a1, 0x3, 0x2, 0x2, 0x2, 0x5a3, 0x5a4, - 0x3, 0x2, 0x2, 0x2, 0x5a4, 0x5a8, 0x3, 0x2, 0x2, 0x2, 0x5a5, 0x5a6, - 0x7, 0x61, 0x2, 0x2, 0x5a6, 0x5a9, 0x7, 0xbf, 0x2, 0x2, 0x5a7, 0x5a9, - 0x5, 0x78, 0x3d, 0x2, 0x5a8, 0x5a5, 0x3, 0x2, 0x2, 0x2, 0x5a8, 0x5a7, - 0x3, 0x2, 0x2, 0x2, 0x5a8, 0x5a9, 0x3, 0x2, 0x2, 0x2, 0x5a9, 0x5ab, - 0x3, 0x2, 0x2, 0x2, 0x5aa, 0x5ac, 0x5, 0x84, 0x43, 0x2, 0x5ab, 0x5aa, - 0x3, 0x2, 0x2, 0x2, 0x5ab, 0x5ac, 0x3, 0x2, 0x2, 0x2, 0x5ac, 0x5ae, - 0x3, 0x2, 0x2, 0x2, 0x5ad, 0x583, 0x3, 0x2, 0x2, 0x2, 0x5ad, 0x587, - 0x3, 0x2, 0x2, 0x2, 0x5ad, 0x58b, 0x3, 0x2, 0x2, 0x2, 0x5ad, 0x594, - 0x3, 0x2, 0x2, 0x2, 0x5ad, 0x596, 0x3, 0x2, 0x2, 0x2, 0x5ad, 0x59c, - 0x3, 0x2, 0x2, 0x2, 0x5ae, 0xa1, 0x3, 0x2, 0x2, 0x2, 0x5af, 0x5b0, 0x7, - 0x99, 0x2, 0x2, 0x5b0, 0x5b1, 0x7, 0x3f, 0x2, 0x2, 0x5b1, 0x5b2, 0x7, - 0x32, 0x2, 0x2, 0x5b2, 0x5d2, 0x5, 0xc0, 0x61, 0x2, 0x5b3, 0x5b4, 0x7, - 0x99, 0x2, 0x2, 0x5b4, 0x5b5, 0x7, 0x3f, 0x2, 0x2, 0x5b5, 0x5d2, 0x7, - 0x65, 0x2, 0x2, 0x5b6, 0x5b7, 0x7, 0x99, 0x2, 0x2, 0x5b7, 0x5b8, 0x7, - 0x83, 0x2, 0x2, 0x5b8, 0x5d2, 0x7, 0x2e, 0x2, 0x2, 0x5b9, 0x5ba, 0x7, - 0x99, 0x2, 0x2, 0x5ba, 0x5bb, 0x7, 0x83, 0x2, 0x2, 0x5bb, 0x5bc, 0x7, - 0x2f, 0x2, 0x2, 0x5bc, 0x5d2, 0x5, 0xc0, 0x61, 0x2, 0x5bd, 0x5be, 0x7, - 0x99, 0x2, 0x2, 0x5be, 0x5c6, 0x9, 0x13, 0x2, 0x2, 0x5bf, 0x5c0, 0x7, - 0x32, 0x2, 0x2, 0x5c0, 0x5c7, 0x7, 0x8f, 0x2, 0x2, 0x5c1, 0x5c7, 0x7, - 0x3c, 0x2, 0x2, 0x5c2, 0x5c4, 0x7, 0xa8, 0x2, 0x2, 0x5c3, 0x5c2, 0x3, - 0x2, 0x2, 0x2, 0x5c3, 0x5c4, 0x3, 0x2, 0x2, 0x2, 0x5c4, 0x5c5, 0x3, - 0x2, 0x2, 0x2, 0x5c5, 0x5c7, 0x7, 0x69, 0x2, 0x2, 0x5c6, 0x5bf, 0x3, - 0x2, 0x2, 0x2, 0x5c6, 0x5c1, 0x3, 0x2, 0x2, 0x2, 0x5c6, 0x5c3, 0x3, - 0x2, 0x2, 0x2, 0x5c7, 0x5c8, 0x3, 0x2, 0x2, 0x2, 0x5c8, 0x5d2, 0x5, - 0xc0, 0x61, 0x2, 0x5c9, 0x5ca, 0x7, 0x99, 0x2, 0x2, 0x5ca, 0x5cb, 0x9, - 0x13, 0x2, 0x2, 0x5cb, 0x5cc, 0x7, 0x88, 0x2, 0x2, 0x5cc, 0x5d2, 0x7, - 0x8f, 0x2, 0x2, 0x5cd, 0x5ce, 0x7, 0x99, 0x2, 0x2, 0x5ce, 0x5cf, 0x7, - 0x97, 0x2, 0x2, 0x5cf, 0x5d0, 0x7, 0x87, 0x2, 0x2, 0x5d0, 0x5d2, 0x5, - 0xc0, 0x61, 0x2, 0x5d1, 0x5af, 0x3, 0x2, 0x2, 0x2, 0x5d1, 0x5b3, 0x3, - 0x2, 0x2, 0x2, 0x5d1, 0x5b6, 0x3, 0x2, 0x2, 0x2, 0x5d1, 0x5b9, 0x3, - 0x2, 0x2, 0x2, 0x5d1, 0x5bd, 0x3, 0x2, 0x2, 0x2, 0x5d1, 0x5c9, 0x3, - 0x2, 0x2, 0x2, 0x5d1, 0x5cd, 0x3, 0x2, 0x2, 0x2, 0x5d2, 0xa3, 0x3, 0x2, - 0x2, 0x2, 0x5d3, 0x5d5, 0x7, 0xa7, 0x2, 0x2, 0x5d4, 0x5d6, 0x7, 0x9c, - 0x2, 0x2, 0x5d5, 0x5d4, 0x3, 0x2, 0x2, 0x2, 0x5d5, 0x5d6, 0x3, 0x2, - 0x2, 0x2, 0x5d6, 0x5d8, 0x3, 0x2, 0x2, 0x2, 0x5d7, 0x5d9, 0x7, 0x9a, - 0x2, 0x2, 0x5d8, 0x5d7, 0x3, 0x2, 0x2, 0x2, 0x5d8, 0x5d9, 0x3, 0x2, - 0x2, 0x2, 0x5d9, 0x5dc, 0x3, 0x2, 0x2, 0x2, 0x5da, 0x5db, 0x7, 0x4d, - 0x2, 0x2, 0x5db, 0x5dd, 0x7, 0x38, 0x2, 0x2, 0x5dc, 0x5da, 0x3, 0x2, - 0x2, 0x2, 0x5dc, 0x5dd, 0x3, 0x2, 0x2, 0x2, 0x5dd, 0x5de, 0x3, 0x2, - 0x2, 0x2, 0x5de, 0x5e0, 0x5, 0xc0, 0x61, 0x2, 0x5df, 0x5e1, 0x5, 0x2c, - 0x17, 0x2, 0x5e0, 0x5df, 0x3, 0x2, 0x2, 0x2, 0x5e0, 0x5e1, 0x3, 0x2, - 0x2, 0x2, 0x5e1, 0xa5, 0x3, 0x2, 0x2, 0x2, 0x5e2, 0x5e3, 0x7, 0xac, - 0x2, 0x2, 0x5e3, 0x5e4, 0x5, 0xc6, 0x64, 0x2, 0x5e4, 0xa7, 0x3, 0x2, - 0x2, 0x2, 0x5e5, 0x5e6, 0x7, 0xb2, 0x2, 0x2, 0x5e6, 0x5e8, 0x5, 0xc0, - 0x61, 0x2, 0x5e7, 0x5e9, 0x7, 0x37, 0x2, 0x2, 0x5e8, 0x5e7, 0x3, 0x2, - 0x2, 0x2, 0x5e8, 0x5e9, 0x3, 0x2, 0x2, 0x2, 0x5e9, 0x5ec, 0x3, 0x2, - 0x2, 0x2, 0x5ea, 0x5eb, 0x7, 0x62, 0x2, 0x2, 0x5eb, 0x5ed, 0x7, 0xbd, - 0x2, 0x2, 0x5ec, 0x5ea, 0x3, 0x2, 0x2, 0x2, 0x5ec, 0x5ed, 0x3, 0x2, - 0x2, 0x2, 0x5ed, 0xa9, 0x3, 0x2, 0x2, 0x2, 0x5ee, 0x61e, 0x5, 0xd6, - 0x6c, 0x2, 0x5ef, 0x5f0, 0x5, 0xd6, 0x6c, 0x2, 0x5f0, 0x5f1, 0x7, 0xd0, - 0x2, 0x2, 0x5f1, 0x5f2, 0x5, 0xd6, 0x6c, 0x2, 0x5f2, 0x5f9, 0x5, 0xaa, - 0x56, 0x2, 0x5f3, 0x5f4, 0x7, 0xc5, 0x2, 0x2, 0x5f4, 0x5f5, 0x5, 0xd6, - 0x6c, 0x2, 0x5f5, 0x5f6, 0x5, 0xaa, 0x56, 0x2, 0x5f6, 0x5f8, 0x3, 0x2, - 0x2, 0x2, 0x5f7, 0x5f3, 0x3, 0x2, 0x2, 0x2, 0x5f8, 0x5fb, 0x3, 0x2, - 0x2, 0x2, 0x5f9, 0x5f7, 0x3, 0x2, 0x2, 0x2, 0x5f9, 0x5fa, 0x3, 0x2, - 0x2, 0x2, 0x5fa, 0x5fc, 0x3, 0x2, 0x2, 0x2, 0x5fb, 0x5f9, 0x3, 0x2, - 0x2, 0x2, 0x5fc, 0x5fd, 0x7, 0xda, 0x2, 0x2, 0x5fd, 0x61e, 0x3, 0x2, - 0x2, 0x2, 0x5fe, 0x5ff, 0x5, 0xd6, 0x6c, 0x2, 0x5ff, 0x600, 0x7, 0xd0, - 0x2, 0x2, 0x600, 0x605, 0x5, 0xda, 0x6e, 0x2, 0x601, 0x602, 0x7, 0xc5, - 0x2, 0x2, 0x602, 0x604, 0x5, 0xda, 0x6e, 0x2, 0x603, 0x601, 0x3, 0x2, - 0x2, 0x2, 0x604, 0x607, 0x3, 0x2, 0x2, 0x2, 0x605, 0x603, 0x3, 0x2, - 0x2, 0x2, 0x605, 0x606, 0x3, 0x2, 0x2, 0x2, 0x606, 0x608, 0x3, 0x2, - 0x2, 0x2, 0x607, 0x605, 0x3, 0x2, 0x2, 0x2, 0x608, 0x609, 0x7, 0xda, - 0x2, 0x2, 0x609, 0x61e, 0x3, 0x2, 0x2, 0x2, 0x60a, 0x60b, 0x5, 0xd6, - 0x6c, 0x2, 0x60b, 0x60c, 0x7, 0xd0, 0x2, 0x2, 0x60c, 0x611, 0x5, 0xaa, - 0x56, 0x2, 0x60d, 0x60e, 0x7, 0xc5, 0x2, 0x2, 0x60e, 0x610, 0x5, 0xaa, - 0x56, 0x2, 0x60f, 0x60d, 0x3, 0x2, 0x2, 0x2, 0x610, 0x613, 0x3, 0x2, - 0x2, 0x2, 0x611, 0x60f, 0x3, 0x2, 0x2, 0x2, 0x611, 0x612, 0x3, 0x2, - 0x2, 0x2, 0x612, 0x614, 0x3, 0x2, 0x2, 0x2, 0x613, 0x611, 0x3, 0x2, - 0x2, 0x2, 0x614, 0x615, 0x7, 0xda, 0x2, 0x2, 0x615, 0x61e, 0x3, 0x2, - 0x2, 0x2, 0x616, 0x617, 0x5, 0xd6, 0x6c, 0x2, 0x617, 0x619, 0x7, 0xd0, - 0x2, 0x2, 0x618, 0x61a, 0x5, 0xac, 0x57, 0x2, 0x619, 0x618, 0x3, 0x2, - 0x2, 0x2, 0x619, 0x61a, 0x3, 0x2, 0x2, 0x2, 0x61a, 0x61b, 0x3, 0x2, - 0x2, 0x2, 0x61b, 0x61c, 0x7, 0xda, 0x2, 0x2, 0x61c, 0x61e, 0x3, 0x2, - 0x2, 0x2, 0x61d, 0x5ee, 0x3, 0x2, 0x2, 0x2, 0x61d, 0x5ef, 0x3, 0x2, - 0x2, 0x2, 0x61d, 0x5fe, 0x3, 0x2, 0x2, 0x2, 0x61d, 0x60a, 0x3, 0x2, - 0x2, 0x2, 0x61d, 0x616, 0x3, 0x2, 0x2, 0x2, 0x61e, 0xab, 0x3, 0x2, 0x2, - 0x2, 0x61f, 0x624, 0x5, 0xae, 0x58, 0x2, 0x620, 0x621, 0x7, 0xc5, 0x2, - 0x2, 0x621, 0x623, 0x5, 0xae, 0x58, 0x2, 0x622, 0x620, 0x3, 0x2, 0x2, - 0x2, 0x623, 0x626, 0x3, 0x2, 0x2, 0x2, 0x624, 0x622, 0x3, 0x2, 0x2, - 0x2, 0x624, 0x625, 0x3, 0x2, 0x2, 0x2, 0x625, 0xad, 0x3, 0x2, 0x2, 0x2, - 0x626, 0x624, 0x3, 0x2, 0x2, 0x2, 0x627, 0x628, 0x5, 0xc0, 0x61, 0x2, - 0x628, 0x629, 0x7, 0xc8, 0x2, 0x2, 0x629, 0x62b, 0x3, 0x2, 0x2, 0x2, - 0x62a, 0x627, 0x3, 0x2, 0x2, 0x2, 0x62a, 0x62b, 0x3, 0x2, 0x2, 0x2, - 0x62b, 0x62c, 0x3, 0x2, 0x2, 0x2, 0x62c, 0x633, 0x7, 0xc1, 0x2, 0x2, - 0x62d, 0x62e, 0x7, 0xd0, 0x2, 0x2, 0x62e, 0x62f, 0x5, 0x68, 0x35, 0x2, - 0x62f, 0x630, 0x7, 0xda, 0x2, 0x2, 0x630, 0x633, 0x3, 0x2, 0x2, 0x2, - 0x631, 0x633, 0x5, 0xb0, 0x59, 0x2, 0x632, 0x62a, 0x3, 0x2, 0x2, 0x2, - 0x632, 0x62d, 0x3, 0x2, 0x2, 0x2, 0x632, 0x631, 0x3, 0x2, 0x2, 0x2, - 0x633, 0xaf, 0x3, 0x2, 0x2, 0x2, 0x634, 0x635, 0x8, 0x59, 0x1, 0x2, - 0x635, 0x637, 0x7, 0x15, 0x2, 0x2, 0x636, 0x638, 0x5, 0xb0, 0x59, 0x2, - 0x637, 0x636, 0x3, 0x2, 0x2, 0x2, 0x637, 0x638, 0x3, 0x2, 0x2, 0x2, - 0x638, 0x63e, 0x3, 0x2, 0x2, 0x2, 0x639, 0x63a, 0x7, 0xb4, 0x2, 0x2, - 0x63a, 0x63b, 0x5, 0xb0, 0x59, 0x2, 0x63b, 0x63c, 0x7, 0x9e, 0x2, 0x2, - 0x63c, 0x63d, 0x5, 0xb0, 0x59, 0x2, 0x63d, 0x63f, 0x3, 0x2, 0x2, 0x2, - 0x63e, 0x639, 0x3, 0x2, 0x2, 0x2, 0x63f, 0x640, 0x3, 0x2, 0x2, 0x2, - 0x640, 0x63e, 0x3, 0x2, 0x2, 0x2, 0x640, 0x641, 0x3, 0x2, 0x2, 0x2, - 0x641, 0x644, 0x3, 0x2, 0x2, 0x2, 0x642, 0x643, 0x7, 0x34, 0x2, 0x2, - 0x643, 0x645, 0x5, 0xb0, 0x59, 0x2, 0x644, 0x642, 0x3, 0x2, 0x2, 0x2, - 0x644, 0x645, 0x3, 0x2, 0x2, 0x2, 0x645, 0x646, 0x3, 0x2, 0x2, 0x2, - 0x646, 0x647, 0x7, 0x35, 0x2, 0x2, 0x647, 0x6a0, 0x3, 0x2, 0x2, 0x2, - 0x648, 0x649, 0x7, 0x16, 0x2, 0x2, 0x649, 0x64a, 0x7, 0xd0, 0x2, 0x2, - 0x64a, 0x64b, 0x5, 0xb0, 0x59, 0x2, 0x64b, 0x64c, 0x7, 0xc, 0x2, 0x2, - 0x64c, 0x64d, 0x5, 0xaa, 0x56, 0x2, 0x64d, 0x64e, 0x7, 0xda, 0x2, 0x2, - 0x64e, 0x6a0, 0x3, 0x2, 0x2, 0x2, 0x64f, 0x650, 0x7, 0x24, 0x2, 0x2, - 0x650, 0x6a0, 0x7, 0xbf, 0x2, 0x2, 0x651, 0x652, 0x7, 0x3b, 0x2, 0x2, - 0x652, 0x653, 0x7, 0xd0, 0x2, 0x2, 0x653, 0x654, 0x5, 0xce, 0x68, 0x2, - 0x654, 0x655, 0x7, 0x43, 0x2, 0x2, 0x655, 0x656, 0x5, 0xb0, 0x59, 0x2, - 0x656, 0x657, 0x7, 0xda, 0x2, 0x2, 0x657, 0x6a0, 0x3, 0x2, 0x2, 0x2, - 0x658, 0x659, 0x7, 0x55, 0x2, 0x2, 0x659, 0x65a, 0x5, 0xb0, 0x59, 0x2, - 0x65a, 0x65b, 0x5, 0xce, 0x68, 0x2, 0x65b, 0x6a0, 0x3, 0x2, 0x2, 0x2, - 0x65c, 0x65d, 0x7, 0x96, 0x2, 0x2, 0x65d, 0x65e, 0x7, 0xd0, 0x2, 0x2, - 0x65e, 0x65f, 0x5, 0xb0, 0x59, 0x2, 0x65f, 0x660, 0x7, 0x43, 0x2, 0x2, - 0x660, 0x663, 0x5, 0xb0, 0x59, 0x2, 0x661, 0x662, 0x7, 0x40, 0x2, 0x2, - 0x662, 0x664, 0x5, 0xb0, 0x59, 0x2, 0x663, 0x661, 0x3, 0x2, 0x2, 0x2, - 0x663, 0x664, 0x3, 0x2, 0x2, 0x2, 0x664, 0x665, 0x3, 0x2, 0x2, 0x2, - 0x665, 0x666, 0x7, 0xda, 0x2, 0x2, 0x666, 0x6a0, 0x3, 0x2, 0x2, 0x2, - 0x667, 0x668, 0x7, 0xa1, 0x2, 0x2, 0x668, 0x6a0, 0x7, 0xbf, 0x2, 0x2, - 0x669, 0x66a, 0x7, 0xa6, 0x2, 0x2, 0x66a, 0x66b, 0x7, 0xd0, 0x2, 0x2, - 0x66b, 0x66c, 0x9, 0x14, 0x2, 0x2, 0x66c, 0x66d, 0x7, 0xbf, 0x2, 0x2, - 0x66d, 0x66e, 0x7, 0x43, 0x2, 0x2, 0x66e, 0x66f, 0x5, 0xb0, 0x59, 0x2, - 0x66f, 0x670, 0x7, 0xda, 0x2, 0x2, 0x670, 0x6a0, 0x3, 0x2, 0x2, 0x2, - 0x671, 0x677, 0x5, 0xd6, 0x6c, 0x2, 0x672, 0x674, 0x7, 0xd0, 0x2, 0x2, - 0x673, 0x675, 0x5, 0xac, 0x57, 0x2, 0x674, 0x673, 0x3, 0x2, 0x2, 0x2, - 0x674, 0x675, 0x3, 0x2, 0x2, 0x2, 0x675, 0x676, 0x3, 0x2, 0x2, 0x2, - 0x676, 0x678, 0x7, 0xda, 0x2, 0x2, 0x677, 0x672, 0x3, 0x2, 0x2, 0x2, - 0x677, 0x678, 0x3, 0x2, 0x2, 0x2, 0x678, 0x679, 0x3, 0x2, 0x2, 0x2, - 0x679, 0x67b, 0x7, 0xd0, 0x2, 0x2, 0x67a, 0x67c, 0x7, 0x31, 0x2, 0x2, - 0x67b, 0x67a, 0x3, 0x2, 0x2, 0x2, 0x67b, 0x67c, 0x3, 0x2, 0x2, 0x2, - 0x67c, 0x67e, 0x3, 0x2, 0x2, 0x2, 0x67d, 0x67f, 0x5, 0xb2, 0x5a, 0x2, - 0x67e, 0x67d, 0x3, 0x2, 0x2, 0x2, 0x67e, 0x67f, 0x3, 0x2, 0x2, 0x2, - 0x67f, 0x680, 0x3, 0x2, 0x2, 0x2, 0x680, 0x681, 0x7, 0xda, 0x2, 0x2, - 0x681, 0x6a0, 0x3, 0x2, 0x2, 0x2, 0x682, 0x6a0, 0x5, 0xcc, 0x67, 0x2, - 0x683, 0x684, 0x7, 0xc7, 0x2, 0x2, 0x684, 0x6a0, 0x5, 0xb0, 0x59, 0x13, - 0x685, 0x686, 0x7, 0x72, 0x2, 0x2, 0x686, 0x6a0, 0x5, 0xb0, 0x59, 0xe, - 0x687, 0x688, 0x5, 0xc0, 0x61, 0x2, 0x688, 0x689, 0x7, 0xc8, 0x2, 0x2, - 0x689, 0x68b, 0x3, 0x2, 0x2, 0x2, 0x68a, 0x687, 0x3, 0x2, 0x2, 0x2, - 0x68a, 0x68b, 0x3, 0x2, 0x2, 0x2, 0x68b, 0x68c, 0x3, 0x2, 0x2, 0x2, - 0x68c, 0x6a0, 0x7, 0xc1, 0x2, 0x2, 0x68d, 0x68e, 0x7, 0xd0, 0x2, 0x2, - 0x68e, 0x68f, 0x5, 0x68, 0x35, 0x2, 0x68f, 0x690, 0x7, 0xda, 0x2, 0x2, - 0x690, 0x6a0, 0x3, 0x2, 0x2, 0x2, 0x691, 0x692, 0x7, 0xd0, 0x2, 0x2, - 0x692, 0x693, 0x5, 0xb0, 0x59, 0x2, 0x693, 0x694, 0x7, 0xda, 0x2, 0x2, - 0x694, 0x6a0, 0x3, 0x2, 0x2, 0x2, 0x695, 0x696, 0x7, 0xd0, 0x2, 0x2, - 0x696, 0x697, 0x5, 0xac, 0x57, 0x2, 0x697, 0x698, 0x7, 0xda, 0x2, 0x2, - 0x698, 0x6a0, 0x3, 0x2, 0x2, 0x2, 0x699, 0x69b, 0x7, 0xce, 0x2, 0x2, - 0x69a, 0x69c, 0x5, 0xac, 0x57, 0x2, 0x69b, 0x69a, 0x3, 0x2, 0x2, 0x2, - 0x69b, 0x69c, 0x3, 0x2, 0x2, 0x2, 0x69c, 0x69d, 0x3, 0x2, 0x2, 0x2, - 0x69d, 0x6a0, 0x7, 0xd9, 0x2, 0x2, 0x69e, 0x6a0, 0x5, 0xb8, 0x5d, 0x2, - 0x69f, 0x634, 0x3, 0x2, 0x2, 0x2, 0x69f, 0x648, 0x3, 0x2, 0x2, 0x2, - 0x69f, 0x64f, 0x3, 0x2, 0x2, 0x2, 0x69f, 0x651, 0x3, 0x2, 0x2, 0x2, - 0x69f, 0x658, 0x3, 0x2, 0x2, 0x2, 0x69f, 0x65c, 0x3, 0x2, 0x2, 0x2, - 0x69f, 0x667, 0x3, 0x2, 0x2, 0x2, 0x69f, 0x669, 0x3, 0x2, 0x2, 0x2, - 0x69f, 0x671, 0x3, 0x2, 0x2, 0x2, 0x69f, 0x682, 0x3, 0x2, 0x2, 0x2, - 0x69f, 0x683, 0x3, 0x2, 0x2, 0x2, 0x69f, 0x685, 0x3, 0x2, 0x2, 0x2, - 0x69f, 0x68a, 0x3, 0x2, 0x2, 0x2, 0x69f, 0x68d, 0x3, 0x2, 0x2, 0x2, - 0x69f, 0x691, 0x3, 0x2, 0x2, 0x2, 0x69f, 0x695, 0x3, 0x2, 0x2, 0x2, - 0x69f, 0x699, 0x3, 0x2, 0x2, 0x2, 0x69f, 0x69e, 0x3, 0x2, 0x2, 0x2, - 0x6a0, 0x6e8, 0x3, 0x2, 0x2, 0x2, 0x6a1, 0x6a2, 0xc, 0x12, 0x2, 0x2, - 0x6a2, 0x6a3, 0x9, 0x15, 0x2, 0x2, 0x6a3, 0x6e7, 0x5, 0xb0, 0x59, 0x13, - 0x6a4, 0x6a5, 0xc, 0x11, 0x2, 0x2, 0x6a5, 0x6a6, 0x9, 0x16, 0x2, 0x2, - 0x6a6, 0x6e7, 0x5, 0xb0, 0x59, 0x12, 0x6a7, 0x6ba, 0xc, 0x10, 0x2, 0x2, - 0x6a8, 0x6bb, 0x7, 0xc9, 0x2, 0x2, 0x6a9, 0x6bb, 0x7, 0xca, 0x2, 0x2, - 0x6aa, 0x6bb, 0x7, 0xd2, 0x2, 0x2, 0x6ab, 0x6bb, 0x7, 0xcf, 0x2, 0x2, - 0x6ac, 0x6bb, 0x7, 0xcb, 0x2, 0x2, 0x6ad, 0x6bb, 0x7, 0xd1, 0x2, 0x2, - 0x6ae, 0x6bb, 0x7, 0xcc, 0x2, 0x2, 0x6af, 0x6b1, 0x7, 0x46, 0x2, 0x2, - 0x6b0, 0x6af, 0x3, 0x2, 0x2, 0x2, 0x6b0, 0x6b1, 0x3, 0x2, 0x2, 0x2, - 0x6b1, 0x6b3, 0x3, 0x2, 0x2, 0x2, 0x6b2, 0x6b4, 0x7, 0x72, 0x2, 0x2, - 0x6b3, 0x6b2, 0x3, 0x2, 0x2, 0x2, 0x6b3, 0x6b4, 0x3, 0x2, 0x2, 0x2, - 0x6b4, 0x6b5, 0x3, 0x2, 0x2, 0x2, 0x6b5, 0x6bb, 0x7, 0x4f, 0x2, 0x2, - 0x6b6, 0x6b8, 0x7, 0x72, 0x2, 0x2, 0x6b7, 0x6b6, 0x3, 0x2, 0x2, 0x2, - 0x6b7, 0x6b8, 0x3, 0x2, 0x2, 0x2, 0x6b8, 0x6b9, 0x3, 0x2, 0x2, 0x2, - 0x6b9, 0x6bb, 0x9, 0x17, 0x2, 0x2, 0x6ba, 0x6a8, 0x3, 0x2, 0x2, 0x2, - 0x6ba, 0x6a9, 0x3, 0x2, 0x2, 0x2, 0x6ba, 0x6aa, 0x3, 0x2, 0x2, 0x2, - 0x6ba, 0x6ab, 0x3, 0x2, 0x2, 0x2, 0x6ba, 0x6ac, 0x3, 0x2, 0x2, 0x2, - 0x6ba, 0x6ad, 0x3, 0x2, 0x2, 0x2, 0x6ba, 0x6ae, 0x3, 0x2, 0x2, 0x2, - 0x6ba, 0x6b0, 0x3, 0x2, 0x2, 0x2, 0x6ba, 0x6b7, 0x3, 0x2, 0x2, 0x2, - 0x6bb, 0x6bc, 0x3, 0x2, 0x2, 0x2, 0x6bc, 0x6e7, 0x5, 0xb0, 0x59, 0x11, - 0x6bd, 0x6be, 0xc, 0xd, 0x2, 0x2, 0x6be, 0x6bf, 0x7, 0x8, 0x2, 0x2, - 0x6bf, 0x6e7, 0x5, 0xb0, 0x59, 0xe, 0x6c0, 0x6c1, 0xc, 0xc, 0x2, 0x2, - 0x6c1, 0x6c2, 0x7, 0x78, 0x2, 0x2, 0x6c2, 0x6e7, 0x5, 0xb0, 0x59, 0xd, - 0x6c3, 0x6c5, 0xc, 0xb, 0x2, 0x2, 0x6c4, 0x6c6, 0x7, 0x72, 0x2, 0x2, - 0x6c5, 0x6c4, 0x3, 0x2, 0x2, 0x2, 0x6c5, 0x6c6, 0x3, 0x2, 0x2, 0x2, - 0x6c6, 0x6c7, 0x3, 0x2, 0x2, 0x2, 0x6c7, 0x6c8, 0x7, 0x12, 0x2, 0x2, - 0x6c8, 0x6c9, 0x5, 0xb0, 0x59, 0x2, 0x6c9, 0x6ca, 0x7, 0x8, 0x2, 0x2, - 0x6ca, 0x6cb, 0x5, 0xb0, 0x59, 0xc, 0x6cb, 0x6e7, 0x3, 0x2, 0x2, 0x2, - 0x6cc, 0x6cd, 0xc, 0xa, 0x2, 0x2, 0x6cd, 0x6ce, 0x7, 0xd5, 0x2, 0x2, - 0x6ce, 0x6cf, 0x5, 0xb0, 0x59, 0x2, 0x6cf, 0x6d0, 0x7, 0xc4, 0x2, 0x2, - 0x6d0, 0x6d1, 0x5, 0xb0, 0x59, 0xa, 0x6d1, 0x6e7, 0x3, 0x2, 0x2, 0x2, - 0x6d2, 0x6d3, 0xc, 0x15, 0x2, 0x2, 0x6d3, 0x6d4, 0x7, 0xce, 0x2, 0x2, - 0x6d4, 0x6d5, 0x5, 0xb0, 0x59, 0x2, 0x6d5, 0x6d6, 0x7, 0xd9, 0x2, 0x2, - 0x6d6, 0x6e7, 0x3, 0x2, 0x2, 0x2, 0x6d7, 0x6d8, 0xc, 0x14, 0x2, 0x2, - 0x6d8, 0x6d9, 0x7, 0xc8, 0x2, 0x2, 0x6d9, 0x6e7, 0x7, 0xbd, 0x2, 0x2, - 0x6da, 0x6db, 0xc, 0xf, 0x2, 0x2, 0x6db, 0x6dd, 0x7, 0x57, 0x2, 0x2, - 0x6dc, 0x6de, 0x7, 0x72, 0x2, 0x2, 0x6dd, 0x6dc, 0x3, 0x2, 0x2, 0x2, - 0x6dd, 0x6de, 0x3, 0x2, 0x2, 0x2, 0x6de, 0x6df, 0x3, 0x2, 0x2, 0x2, - 0x6df, 0x6e7, 0x7, 0x73, 0x2, 0x2, 0x6e0, 0x6e4, 0xc, 0x9, 0x2, 0x2, - 0x6e1, 0x6e5, 0x5, 0xd4, 0x6b, 0x2, 0x6e2, 0x6e3, 0x7, 0xc, 0x2, 0x2, - 0x6e3, 0x6e5, 0x5, 0xd6, 0x6c, 0x2, 0x6e4, 0x6e1, 0x3, 0x2, 0x2, 0x2, - 0x6e4, 0x6e2, 0x3, 0x2, 0x2, 0x2, 0x6e5, 0x6e7, 0x3, 0x2, 0x2, 0x2, - 0x6e6, 0x6a1, 0x3, 0x2, 0x2, 0x2, 0x6e6, 0x6a4, 0x3, 0x2, 0x2, 0x2, - 0x6e6, 0x6a7, 0x3, 0x2, 0x2, 0x2, 0x6e6, 0x6bd, 0x3, 0x2, 0x2, 0x2, - 0x6e6, 0x6c0, 0x3, 0x2, 0x2, 0x2, 0x6e6, 0x6c3, 0x3, 0x2, 0x2, 0x2, - 0x6e6, 0x6cc, 0x3, 0x2, 0x2, 0x2, 0x6e6, 0x6d2, 0x3, 0x2, 0x2, 0x2, - 0x6e6, 0x6d7, 0x3, 0x2, 0x2, 0x2, 0x6e6, 0x6da, 0x3, 0x2, 0x2, 0x2, - 0x6e6, 0x6e0, 0x3, 0x2, 0x2, 0x2, 0x6e7, 0x6ea, 0x3, 0x2, 0x2, 0x2, - 0x6e8, 0x6e6, 0x3, 0x2, 0x2, 0x2, 0x6e8, 0x6e9, 0x3, 0x2, 0x2, 0x2, - 0x6e9, 0xb1, 0x3, 0x2, 0x2, 0x2, 0x6ea, 0x6e8, 0x3, 0x2, 0x2, 0x2, 0x6eb, - 0x6f0, 0x5, 0xb4, 0x5b, 0x2, 0x6ec, 0x6ed, 0x7, 0xc5, 0x2, 0x2, 0x6ed, - 0x6ef, 0x5, 0xb4, 0x5b, 0x2, 0x6ee, 0x6ec, 0x3, 0x2, 0x2, 0x2, 0x6ef, - 0x6f2, 0x3, 0x2, 0x2, 0x2, 0x6f0, 0x6ee, 0x3, 0x2, 0x2, 0x2, 0x6f0, - 0x6f1, 0x3, 0x2, 0x2, 0x2, 0x6f1, 0xb3, 0x3, 0x2, 0x2, 0x2, 0x6f2, 0x6f0, - 0x3, 0x2, 0x2, 0x2, 0x6f3, 0x6f6, 0x5, 0xb6, 0x5c, 0x2, 0x6f4, 0x6f6, - 0x5, 0xb0, 0x59, 0x2, 0x6f5, 0x6f3, 0x3, 0x2, 0x2, 0x2, 0x6f5, 0x6f4, - 0x3, 0x2, 0x2, 0x2, 0x6f6, 0xb5, 0x3, 0x2, 0x2, 0x2, 0x6f7, 0x6f8, 0x7, - 0xd0, 0x2, 0x2, 0x6f8, 0x6fd, 0x5, 0xd6, 0x6c, 0x2, 0x6f9, 0x6fa, 0x7, - 0xc5, 0x2, 0x2, 0x6fa, 0x6fc, 0x5, 0xd6, 0x6c, 0x2, 0x6fb, 0x6f9, 0x3, - 0x2, 0x2, 0x2, 0x6fc, 0x6ff, 0x3, 0x2, 0x2, 0x2, 0x6fd, 0x6fb, 0x3, - 0x2, 0x2, 0x2, 0x6fd, 0x6fe, 0x3, 0x2, 0x2, 0x2, 0x6fe, 0x700, 0x3, - 0x2, 0x2, 0x2, 0x6ff, 0x6fd, 0x3, 0x2, 0x2, 0x2, 0x700, 0x701, 0x7, - 0xda, 0x2, 0x2, 0x701, 0x70b, 0x3, 0x2, 0x2, 0x2, 0x702, 0x707, 0x5, - 0xd6, 0x6c, 0x2, 0x703, 0x704, 0x7, 0xc5, 0x2, 0x2, 0x704, 0x706, 0x5, - 0xd6, 0x6c, 0x2, 0x705, 0x703, 0x3, 0x2, 0x2, 0x2, 0x706, 0x709, 0x3, - 0x2, 0x2, 0x2, 0x707, 0x705, 0x3, 0x2, 0x2, 0x2, 0x707, 0x708, 0x3, - 0x2, 0x2, 0x2, 0x708, 0x70b, 0x3, 0x2, 0x2, 0x2, 0x709, 0x707, 0x3, - 0x2, 0x2, 0x2, 0x70a, 0x6f7, 0x3, 0x2, 0x2, 0x2, 0x70a, 0x702, 0x3, - 0x2, 0x2, 0x2, 0x70b, 0x70c, 0x3, 0x2, 0x2, 0x2, 0x70c, 0x70d, 0x7, - 0xc0, 0x2, 0x2, 0x70d, 0x70e, 0x5, 0xb0, 0x59, 0x2, 0x70e, 0xb7, 0x3, - 0x2, 0x2, 0x2, 0x70f, 0x710, 0x5, 0xc0, 0x61, 0x2, 0x710, 0x711, 0x7, - 0xc8, 0x2, 0x2, 0x711, 0x713, 0x3, 0x2, 0x2, 0x2, 0x712, 0x70f, 0x3, - 0x2, 0x2, 0x2, 0x712, 0x713, 0x3, 0x2, 0x2, 0x2, 0x713, 0x714, 0x3, - 0x2, 0x2, 0x2, 0x714, 0x715, 0x5, 0xba, 0x5e, 0x2, 0x715, 0xb9, 0x3, - 0x2, 0x2, 0x2, 0x716, 0x719, 0x5, 0xd6, 0x6c, 0x2, 0x717, 0x718, 0x7, - 0xc8, 0x2, 0x2, 0x718, 0x71a, 0x5, 0xd6, 0x6c, 0x2, 0x719, 0x717, 0x3, - 0x2, 0x2, 0x2, 0x719, 0x71a, 0x3, 0x2, 0x2, 0x2, 0x71a, 0xbb, 0x3, 0x2, - 0x2, 0x2, 0x71b, 0x71c, 0x8, 0x5f, 0x1, 0x2, 0x71c, 0x723, 0x5, 0xc0, - 0x61, 0x2, 0x71d, 0x723, 0x5, 0xbe, 0x60, 0x2, 0x71e, 0x71f, 0x7, 0xd0, - 0x2, 0x2, 0x71f, 0x720, 0x5, 0x68, 0x35, 0x2, 0x720, 0x721, 0x7, 0xda, - 0x2, 0x2, 0x721, 0x723, 0x3, 0x2, 0x2, 0x2, 0x722, 0x71b, 0x3, 0x2, - 0x2, 0x2, 0x722, 0x71d, 0x3, 0x2, 0x2, 0x2, 0x722, 0x71e, 0x3, 0x2, - 0x2, 0x2, 0x723, 0x72c, 0x3, 0x2, 0x2, 0x2, 0x724, 0x728, 0xc, 0x3, - 0x2, 0x2, 0x725, 0x729, 0x5, 0xd4, 0x6b, 0x2, 0x726, 0x727, 0x7, 0xc, - 0x2, 0x2, 0x727, 0x729, 0x5, 0xd6, 0x6c, 0x2, 0x728, 0x725, 0x3, 0x2, - 0x2, 0x2, 0x728, 0x726, 0x3, 0x2, 0x2, 0x2, 0x729, 0x72b, 0x3, 0x2, - 0x2, 0x2, 0x72a, 0x724, 0x3, 0x2, 0x2, 0x2, 0x72b, 0x72e, 0x3, 0x2, - 0x2, 0x2, 0x72c, 0x72a, 0x3, 0x2, 0x2, 0x2, 0x72c, 0x72d, 0x3, 0x2, - 0x2, 0x2, 0x72d, 0xbd, 0x3, 0x2, 0x2, 0x2, 0x72e, 0x72c, 0x3, 0x2, 0x2, - 0x2, 0x72f, 0x730, 0x5, 0xd6, 0x6c, 0x2, 0x730, 0x732, 0x7, 0xd0, 0x2, - 0x2, 0x731, 0x733, 0x5, 0xc2, 0x62, 0x2, 0x732, 0x731, 0x3, 0x2, 0x2, - 0x2, 0x732, 0x733, 0x3, 0x2, 0x2, 0x2, 0x733, 0x734, 0x3, 0x2, 0x2, - 0x2, 0x734, 0x735, 0x7, 0xda, 0x2, 0x2, 0x735, 0xbf, 0x3, 0x2, 0x2, - 0x2, 0x736, 0x737, 0x5, 0xc6, 0x64, 0x2, 0x737, 0x738, 0x7, 0xc8, 0x2, - 0x2, 0x738, 0x73a, 0x3, 0x2, 0x2, 0x2, 0x739, 0x736, 0x3, 0x2, 0x2, - 0x2, 0x739, 0x73a, 0x3, 0x2, 0x2, 0x2, 0x73a, 0x73b, 0x3, 0x2, 0x2, - 0x2, 0x73b, 0x73c, 0x5, 0xd6, 0x6c, 0x2, 0x73c, 0xc1, 0x3, 0x2, 0x2, - 0x2, 0x73d, 0x742, 0x5, 0xc4, 0x63, 0x2, 0x73e, 0x73f, 0x7, 0xc5, 0x2, - 0x2, 0x73f, 0x741, 0x5, 0xc4, 0x63, 0x2, 0x740, 0x73e, 0x3, 0x2, 0x2, - 0x2, 0x741, 0x744, 0x3, 0x2, 0x2, 0x2, 0x742, 0x740, 0x3, 0x2, 0x2, - 0x2, 0x742, 0x743, 0x3, 0x2, 0x2, 0x2, 0x743, 0xc3, 0x3, 0x2, 0x2, 0x2, - 0x744, 0x742, 0x3, 0x2, 0x2, 0x2, 0x745, 0x749, 0x5, 0xba, 0x5e, 0x2, - 0x746, 0x749, 0x5, 0xbe, 0x60, 0x2, 0x747, 0x749, 0x5, 0xcc, 0x67, 0x2, - 0x748, 0x745, 0x3, 0x2, 0x2, 0x2, 0x748, 0x746, 0x3, 0x2, 0x2, 0x2, - 0x748, 0x747, 0x3, 0x2, 0x2, 0x2, 0x749, 0xc5, 0x3, 0x2, 0x2, 0x2, 0x74a, - 0x74b, 0x5, 0xd6, 0x6c, 0x2, 0x74b, 0xc7, 0x3, 0x2, 0x2, 0x2, 0x74c, - 0x755, 0x7, 0xbb, 0x2, 0x2, 0x74d, 0x74e, 0x7, 0xc8, 0x2, 0x2, 0x74e, - 0x755, 0x9, 0x18, 0x2, 0x2, 0x74f, 0x750, 0x7, 0xbd, 0x2, 0x2, 0x750, - 0x752, 0x7, 0xc8, 0x2, 0x2, 0x751, 0x753, 0x9, 0x18, 0x2, 0x2, 0x752, - 0x751, 0x3, 0x2, 0x2, 0x2, 0x752, 0x753, 0x3, 0x2, 0x2, 0x2, 0x753, - 0x755, 0x3, 0x2, 0x2, 0x2, 0x754, 0x74c, 0x3, 0x2, 0x2, 0x2, 0x754, - 0x74d, 0x3, 0x2, 0x2, 0x2, 0x754, 0x74f, 0x3, 0x2, 0x2, 0x2, 0x755, - 0xc9, 0x3, 0x2, 0x2, 0x2, 0x756, 0x758, 0x9, 0x19, 0x2, 0x2, 0x757, - 0x756, 0x3, 0x2, 0x2, 0x2, 0x757, 0x758, 0x3, 0x2, 0x2, 0x2, 0x758, - 0x75f, 0x3, 0x2, 0x2, 0x2, 0x759, 0x760, 0x5, 0xc8, 0x65, 0x2, 0x75a, - 0x760, 0x7, 0xbc, 0x2, 0x2, 0x75b, 0x760, 0x7, 0xbd, 0x2, 0x2, 0x75c, - 0x760, 0x7, 0xbe, 0x2, 0x2, 0x75d, 0x760, 0x7, 0x51, 0x2, 0x2, 0x75e, - 0x760, 0x7, 0x70, 0x2, 0x2, 0x75f, 0x759, 0x3, 0x2, 0x2, 0x2, 0x75f, - 0x75a, 0x3, 0x2, 0x2, 0x2, 0x75f, 0x75b, 0x3, 0x2, 0x2, 0x2, 0x75f, - 0x75c, 0x3, 0x2, 0x2, 0x2, 0x75f, 0x75d, 0x3, 0x2, 0x2, 0x2, 0x75f, - 0x75e, 0x3, 0x2, 0x2, 0x2, 0x760, 0xcb, 0x3, 0x2, 0x2, 0x2, 0x761, 0x765, - 0x5, 0xca, 0x66, 0x2, 0x762, 0x765, 0x7, 0xbf, 0x2, 0x2, 0x763, 0x765, - 0x7, 0x73, 0x2, 0x2, 0x764, 0x761, 0x3, 0x2, 0x2, 0x2, 0x764, 0x762, - 0x3, 0x2, 0x2, 0x2, 0x764, 0x763, 0x3, 0x2, 0x2, 0x2, 0x765, 0xcd, 0x3, - 0x2, 0x2, 0x2, 0x766, 0x767, 0x9, 0x1a, 0x2, 0x2, 0x767, 0xcf, 0x3, - 0x2, 0x2, 0x2, 0x768, 0x769, 0x9, 0x1b, 0x2, 0x2, 0x769, 0xd1, 0x3, - 0x2, 0x2, 0x2, 0x76a, 0x76b, 0x9, 0x1c, 0x2, 0x2, 0x76b, 0xd3, 0x3, - 0x2, 0x2, 0x2, 0x76c, 0x76f, 0x7, 0xba, 0x2, 0x2, 0x76d, 0x76f, 0x5, - 0xd2, 0x6a, 0x2, 0x76e, 0x76c, 0x3, 0x2, 0x2, 0x2, 0x76e, 0x76d, 0x3, - 0x2, 0x2, 0x2, 0x76f, 0xd5, 0x3, 0x2, 0x2, 0x2, 0x770, 0x774, 0x7, 0xba, - 0x2, 0x2, 0x771, 0x774, 0x5, 0xce, 0x68, 0x2, 0x772, 0x774, 0x5, 0xd0, - 0x69, 0x2, 0x773, 0x770, 0x3, 0x2, 0x2, 0x2, 0x773, 0x771, 0x3, 0x2, - 0x2, 0x2, 0x773, 0x772, 0x3, 0x2, 0x2, 0x2, 0x774, 0xd7, 0x3, 0x2, 0x2, - 0x2, 0x775, 0x778, 0x5, 0xd6, 0x6c, 0x2, 0x776, 0x778, 0x7, 0x73, 0x2, - 0x2, 0x777, 0x775, 0x3, 0x2, 0x2, 0x2, 0x777, 0x776, 0x3, 0x2, 0x2, - 0x2, 0x778, 0xd9, 0x3, 0x2, 0x2, 0x2, 0x779, 0x77a, 0x7, 0xbf, 0x2, - 0x2, 0x77a, 0x77b, 0x7, 0xca, 0x2, 0x2, 0x77b, 0x77c, 0x5, 0xca, 0x66, - 0x2, 0x77c, 0xdb, 0x3, 0x2, 0x2, 0x2, 0x103, 0xe0, 0xe4, 0xe7, 0xea, + 0x2, 0x3a8, 0x3ba, 0x3, 0x2, 0x2, 0x2, 0x3a9, 0x3ab, 0x5, 0xba, 0x5e, + 0x2, 0x3aa, 0x3ac, 0x5, 0xaa, 0x56, 0x2, 0x3ab, 0x3aa, 0x3, 0x2, 0x2, + 0x2, 0x3ab, 0x3ac, 0x3, 0x2, 0x2, 0x2, 0x3ac, 0x3ad, 0x3, 0x2, 0x2, + 0x2, 0x3ad, 0x3b0, 0x5, 0x46, 0x24, 0x2, 0x3ae, 0x3af, 0x7, 0x1d, 0x2, + 0x2, 0x3af, 0x3b1, 0x7, 0xbf, 0x2, 0x2, 0x3b0, 0x3ae, 0x3, 0x2, 0x2, + 0x2, 0x3b0, 0x3b1, 0x3, 0x2, 0x2, 0x2, 0x3b1, 0x3b3, 0x3, 0x2, 0x2, + 0x2, 0x3b2, 0x3b4, 0x5, 0x4c, 0x27, 0x2, 0x3b3, 0x3b2, 0x3, 0x2, 0x2, + 0x2, 0x3b3, 0x3b4, 0x3, 0x2, 0x2, 0x2, 0x3b4, 0x3b7, 0x3, 0x2, 0x2, + 0x2, 0x3b5, 0x3b6, 0x7, 0xa8, 0x2, 0x2, 0x3b6, 0x3b8, 0x5, 0xb0, 0x59, + 0x2, 0x3b7, 0x3b5, 0x3, 0x2, 0x2, 0x2, 0x3b7, 0x3b8, 0x3, 0x2, 0x2, + 0x2, 0x3b8, 0x3ba, 0x3, 0x2, 0x2, 0x2, 0x3b9, 0x399, 0x3, 0x2, 0x2, + 0x2, 0x3b9, 0x3a9, 0x3, 0x2, 0x2, 0x2, 0x3ba, 0x45, 0x3, 0x2, 0x2, 0x2, + 0x3bb, 0x3bc, 0x9, 0x4, 0x2, 0x2, 0x3bc, 0x3bd, 0x5, 0xb0, 0x59, 0x2, + 0x3bd, 0x47, 0x3, 0x2, 0x2, 0x2, 0x3be, 0x3bf, 0x5, 0xba, 0x5e, 0x2, + 0x3bf, 0x3c0, 0x5, 0xb0, 0x59, 0x2, 0x3c0, 0x3c1, 0x7, 0xa9, 0x2, 0x2, + 0x3c1, 0x3c2, 0x5, 0xaa, 0x56, 0x2, 0x3c2, 0x3c3, 0x7, 0x47, 0x2, 0x2, + 0x3c3, 0x3c4, 0x7, 0xbd, 0x2, 0x2, 0x3c4, 0x49, 0x3, 0x2, 0x2, 0x2, + 0x3c5, 0x3c6, 0x5, 0xba, 0x5e, 0x2, 0x3c6, 0x3c7, 0x5, 0x66, 0x34, 0x2, + 0x3c7, 0x4b, 0x3, 0x2, 0x2, 0x2, 0x3c8, 0x3c9, 0x7, 0x1a, 0x2, 0x2, + 0x3c9, 0x3ca, 0x7, 0xd0, 0x2, 0x2, 0x3ca, 0x3cf, 0x5, 0x4e, 0x28, 0x2, + 0x3cb, 0x3cc, 0x7, 0xc5, 0x2, 0x2, 0x3cc, 0x3ce, 0x5, 0x4e, 0x28, 0x2, + 0x3cd, 0x3cb, 0x3, 0x2, 0x2, 0x2, 0x3ce, 0x3d1, 0x3, 0x2, 0x2, 0x2, + 0x3cf, 0x3cd, 0x3, 0x2, 0x2, 0x2, 0x3cf, 0x3d0, 0x3, 0x2, 0x2, 0x2, + 0x3d0, 0x3d2, 0x3, 0x2, 0x2, 0x2, 0x3d1, 0x3cf, 0x3, 0x2, 0x2, 0x2, + 0x3d2, 0x3d3, 0x7, 0xda, 0x2, 0x2, 0x3d3, 0x4d, 0x3, 0x2, 0x2, 0x2, + 0x3d4, 0x3da, 0x5, 0xd6, 0x6c, 0x2, 0x3d5, 0x3d7, 0x7, 0xd0, 0x2, 0x2, + 0x3d6, 0x3d8, 0x5, 0xac, 0x57, 0x2, 0x3d7, 0x3d6, 0x3, 0x2, 0x2, 0x2, + 0x3d7, 0x3d8, 0x3, 0x2, 0x2, 0x2, 0x3d8, 0x3d9, 0x3, 0x2, 0x2, 0x2, + 0x3d9, 0x3db, 0x7, 0xda, 0x2, 0x2, 0x3da, 0x3d5, 0x3, 0x2, 0x2, 0x2, + 0x3da, 0x3db, 0x3, 0x2, 0x2, 0x2, 0x3db, 0x4f, 0x3, 0x2, 0x2, 0x2, 0x3dc, + 0x3e4, 0x5, 0xb0, 0x59, 0x2, 0x3dd, 0x3e5, 0x7, 0x29, 0x2, 0x2, 0x3de, + 0x3df, 0x7, 0xa2, 0x2, 0x2, 0x3df, 0x3e0, 0x7, 0x30, 0x2, 0x2, 0x3e0, + 0x3e5, 0x7, 0xbf, 0x2, 0x2, 0x3e1, 0x3e2, 0x7, 0xa2, 0x2, 0x2, 0x3e2, + 0x3e3, 0x7, 0xb1, 0x2, 0x2, 0x3e3, 0x3e5, 0x7, 0xbf, 0x2, 0x2, 0x3e4, + 0x3dd, 0x3, 0x2, 0x2, 0x2, 0x3e4, 0x3de, 0x3, 0x2, 0x2, 0x2, 0x3e4, + 0x3e1, 0x3, 0x2, 0x2, 0x2, 0x3e4, 0x3e5, 0x3, 0x2, 0x2, 0x2, 0x3e5, + 0x51, 0x3, 0x2, 0x2, 0x2, 0x3e6, 0x3e8, 0x9, 0x5, 0x2, 0x2, 0x3e7, 0x3e9, + 0x7, 0x9a, 0x2, 0x2, 0x3e8, 0x3e7, 0x3, 0x2, 0x2, 0x2, 0x3e8, 0x3e9, + 0x3, 0x2, 0x2, 0x2, 0x3e9, 0x3ea, 0x3, 0x2, 0x2, 0x2, 0x3ea, 0x3eb, + 0x5, 0xbc, 0x5f, 0x2, 0x3eb, 0x53, 0x3, 0x2, 0x2, 0x2, 0x3ec, 0x3ed, + 0x9, 0x6, 0x2, 0x2, 0x3ed, 0x3f0, 0x7, 0x22, 0x2, 0x2, 0x3ee, 0x3ef, + 0x7, 0x4d, 0x2, 0x2, 0x3ef, 0x3f1, 0x7, 0x38, 0x2, 0x2, 0x3f0, 0x3ee, + 0x3, 0x2, 0x2, 0x2, 0x3f0, 0x3f1, 0x3, 0x2, 0x2, 0x2, 0x3f1, 0x3f2, + 0x3, 0x2, 0x2, 0x2, 0x3f2, 0x3f4, 0x5, 0xc6, 0x64, 0x2, 0x3f3, 0x3f5, + 0x5, 0x2c, 0x17, 0x2, 0x3f4, 0x3f3, 0x3, 0x2, 0x2, 0x2, 0x3f4, 0x3f5, + 0x3, 0x2, 0x2, 0x2, 0x3f5, 0x40c, 0x3, 0x2, 0x2, 0x2, 0x3f6, 0x3fd, + 0x9, 0x6, 0x2, 0x2, 0x3f7, 0x3fe, 0x7, 0x2f, 0x2, 0x2, 0x3f8, 0x3fa, + 0x7, 0x9c, 0x2, 0x2, 0x3f9, 0x3f8, 0x3, 0x2, 0x2, 0x2, 0x3f9, 0x3fa, + 0x3, 0x2, 0x2, 0x2, 0x3fa, 0x3fb, 0x3, 0x2, 0x2, 0x2, 0x3fb, 0x3fe, + 0x7, 0x9a, 0x2, 0x2, 0x3fc, 0x3fe, 0x7, 0xb0, 0x2, 0x2, 0x3fd, 0x3f7, + 0x3, 0x2, 0x2, 0x2, 0x3fd, 0x3f9, 0x3, 0x2, 0x2, 0x2, 0x3fd, 0x3fc, + 0x3, 0x2, 0x2, 0x2, 0x3fe, 0x401, 0x3, 0x2, 0x2, 0x2, 0x3ff, 0x400, + 0x7, 0x4d, 0x2, 0x2, 0x400, 0x402, 0x7, 0x38, 0x2, 0x2, 0x401, 0x3ff, + 0x3, 0x2, 0x2, 0x2, 0x401, 0x402, 0x3, 0x2, 0x2, 0x2, 0x402, 0x403, + 0x3, 0x2, 0x2, 0x2, 0x403, 0x405, 0x5, 0xc0, 0x61, 0x2, 0x404, 0x406, + 0x5, 0x2c, 0x17, 0x2, 0x405, 0x404, 0x3, 0x2, 0x2, 0x2, 0x405, 0x406, + 0x3, 0x2, 0x2, 0x2, 0x406, 0x409, 0x3, 0x2, 0x2, 0x2, 0x407, 0x408, + 0x7, 0x71, 0x2, 0x2, 0x408, 0x40a, 0x7, 0x28, 0x2, 0x2, 0x409, 0x407, + 0x3, 0x2, 0x2, 0x2, 0x409, 0x40a, 0x3, 0x2, 0x2, 0x2, 0x40a, 0x40c, + 0x3, 0x2, 0x2, 0x2, 0x40b, 0x3ec, 0x3, 0x2, 0x2, 0x2, 0x40b, 0x3f6, + 0x3, 0x2, 0x2, 0x2, 0x40c, 0x55, 0x3, 0x2, 0x2, 0x2, 0x40d, 0x40e, 0x7, + 0x38, 0x2, 0x2, 0x40e, 0x40f, 0x7, 0x22, 0x2, 0x2, 0x40f, 0x41b, 0x5, + 0xc6, 0x64, 0x2, 0x410, 0x417, 0x7, 0x38, 0x2, 0x2, 0x411, 0x418, 0x7, + 0x2f, 0x2, 0x2, 0x412, 0x414, 0x7, 0x9c, 0x2, 0x2, 0x413, 0x412, 0x3, + 0x2, 0x2, 0x2, 0x413, 0x414, 0x3, 0x2, 0x2, 0x2, 0x414, 0x415, 0x3, + 0x2, 0x2, 0x2, 0x415, 0x418, 0x7, 0x9a, 0x2, 0x2, 0x416, 0x418, 0x7, + 0xb0, 0x2, 0x2, 0x417, 0x411, 0x3, 0x2, 0x2, 0x2, 0x417, 0x413, 0x3, + 0x2, 0x2, 0x2, 0x417, 0x416, 0x3, 0x2, 0x2, 0x2, 0x417, 0x418, 0x3, + 0x2, 0x2, 0x2, 0x418, 0x419, 0x3, 0x2, 0x2, 0x2, 0x419, 0x41b, 0x5, + 0xc0, 0x61, 0x2, 0x41a, 0x40d, 0x3, 0x2, 0x2, 0x2, 0x41a, 0x410, 0x3, + 0x2, 0x2, 0x2, 0x41b, 0x57, 0x3, 0x2, 0x2, 0x2, 0x41c, 0x41d, 0x7, 0x39, + 0x2, 0x2, 0x41d, 0x41e, 0x7, 0xf, 0x2, 0x2, 0x41e, 0x423, 0x5, 0x4, + 0x3, 0x2, 0x41f, 0x420, 0x7, 0x39, 0x2, 0x2, 0x420, 0x421, 0x7, 0x98, + 0x2, 0x2, 0x421, 0x423, 0x5, 0x4, 0x3, 0x2, 0x422, 0x41c, 0x3, 0x2, + 0x2, 0x2, 0x422, 0x41f, 0x3, 0x2, 0x2, 0x2, 0x423, 0x59, 0x3, 0x2, 0x2, + 0x2, 0x424, 0x425, 0x7, 0x54, 0x2, 0x2, 0x425, 0x427, 0x7, 0x56, 0x2, + 0x2, 0x426, 0x428, 0x7, 0x9a, 0x2, 0x2, 0x427, 0x426, 0x3, 0x2, 0x2, + 0x2, 0x427, 0x428, 0x3, 0x2, 0x2, 0x2, 0x428, 0x42c, 0x3, 0x2, 0x2, + 0x2, 0x429, 0x42d, 0x5, 0xc0, 0x61, 0x2, 0x42a, 0x42b, 0x7, 0x45, 0x2, + 0x2, 0x42b, 0x42d, 0x5, 0xbe, 0x60, 0x2, 0x42c, 0x429, 0x3, 0x2, 0x2, + 0x2, 0x42c, 0x42a, 0x3, 0x2, 0x2, 0x2, 0x42d, 0x42f, 0x3, 0x2, 0x2, + 0x2, 0x42e, 0x430, 0x5, 0x5c, 0x2f, 0x2, 0x42f, 0x42e, 0x3, 0x2, 0x2, + 0x2, 0x42f, 0x430, 0x3, 0x2, 0x2, 0x2, 0x430, 0x431, 0x3, 0x2, 0x2, + 0x2, 0x431, 0x432, 0x5, 0x5e, 0x30, 0x2, 0x432, 0x5b, 0x3, 0x2, 0x2, + 0x2, 0x433, 0x434, 0x7, 0xd0, 0x2, 0x2, 0x434, 0x439, 0x5, 0xba, 0x5e, + 0x2, 0x435, 0x436, 0x7, 0xc5, 0x2, 0x2, 0x436, 0x438, 0x5, 0xba, 0x5e, + 0x2, 0x437, 0x435, 0x3, 0x2, 0x2, 0x2, 0x438, 0x43b, 0x3, 0x2, 0x2, + 0x2, 0x439, 0x437, 0x3, 0x2, 0x2, 0x2, 0x439, 0x43a, 0x3, 0x2, 0x2, + 0x2, 0x43a, 0x43c, 0x3, 0x2, 0x2, 0x2, 0x43b, 0x439, 0x3, 0x2, 0x2, + 0x2, 0x43c, 0x43d, 0x7, 0xda, 0x2, 0x2, 0x43d, 0x5d, 0x3, 0x2, 0x2, + 0x2, 0x43e, 0x43f, 0x7, 0x41, 0x2, 0x2, 0x43f, 0x448, 0x5, 0xd6, 0x6c, + 0x2, 0x440, 0x448, 0x7, 0xaf, 0x2, 0x2, 0x441, 0x443, 0x5, 0x68, 0x35, + 0x2, 0x442, 0x444, 0x7, 0xdb, 0x2, 0x2, 0x443, 0x442, 0x3, 0x2, 0x2, + 0x2, 0x443, 0x444, 0x3, 0x2, 0x2, 0x2, 0x444, 0x445, 0x3, 0x2, 0x2, + 0x2, 0x445, 0x446, 0x7, 0x2, 0x2, 0x3, 0x446, 0x448, 0x3, 0x2, 0x2, + 0x2, 0x447, 0x43e, 0x3, 0x2, 0x2, 0x2, 0x447, 0x440, 0x3, 0x2, 0x2, + 0x2, 0x447, 0x441, 0x3, 0x2, 0x2, 0x2, 0x448, 0x5f, 0x3, 0x2, 0x2, 0x2, + 0x449, 0x44a, 0x7, 0x5b, 0x2, 0x2, 0x44a, 0x44c, 0x7, 0x6f, 0x2, 0x2, + 0x44b, 0x44d, 0x5, 0x2c, 0x17, 0x2, 0x44c, 0x44b, 0x3, 0x2, 0x2, 0x2, + 0x44c, 0x44d, 0x3, 0x2, 0x2, 0x2, 0x44d, 0x44e, 0x3, 0x2, 0x2, 0x2, + 0x44e, 0x450, 0x5, 0x78, 0x3d, 0x2, 0x44f, 0x451, 0x9, 0x7, 0x2, 0x2, + 0x450, 0x44f, 0x3, 0x2, 0x2, 0x2, 0x450, 0x451, 0x3, 0x2, 0x2, 0x2, + 0x451, 0x61, 0x3, 0x2, 0x2, 0x2, 0x452, 0x453, 0x7, 0x77, 0x2, 0x2, + 0x453, 0x454, 0x7, 0x9a, 0x2, 0x2, 0x454, 0x456, 0x5, 0xc0, 0x61, 0x2, + 0x455, 0x457, 0x5, 0x2c, 0x17, 0x2, 0x456, 0x455, 0x3, 0x2, 0x2, 0x2, + 0x456, 0x457, 0x3, 0x2, 0x2, 0x2, 0x457, 0x459, 0x3, 0x2, 0x2, 0x2, + 0x458, 0x45a, 0x5, 0x10, 0x9, 0x2, 0x459, 0x458, 0x3, 0x2, 0x2, 0x2, + 0x459, 0x45a, 0x3, 0x2, 0x2, 0x2, 0x45a, 0x45c, 0x3, 0x2, 0x2, 0x2, + 0x45b, 0x45d, 0x7, 0x3d, 0x2, 0x2, 0x45c, 0x45b, 0x3, 0x2, 0x2, 0x2, + 0x45c, 0x45d, 0x3, 0x2, 0x2, 0x2, 0x45d, 0x45f, 0x3, 0x2, 0x2, 0x2, + 0x45e, 0x460, 0x7, 0x26, 0x2, 0x2, 0x45f, 0x45e, 0x3, 0x2, 0x2, 0x2, + 0x45f, 0x460, 0x3, 0x2, 0x2, 0x2, 0x460, 0x63, 0x3, 0x2, 0x2, 0x2, 0x461, + 0x462, 0x7, 0x85, 0x2, 0x2, 0x462, 0x463, 0x7, 0x9a, 0x2, 0x2, 0x463, + 0x464, 0x5, 0xc0, 0x61, 0x2, 0x464, 0x465, 0x7, 0xa2, 0x2, 0x2, 0x465, + 0x46d, 0x5, 0xc0, 0x61, 0x2, 0x466, 0x467, 0x7, 0xc5, 0x2, 0x2, 0x467, + 0x468, 0x5, 0xc0, 0x61, 0x2, 0x468, 0x469, 0x7, 0xa2, 0x2, 0x2, 0x469, + 0x46a, 0x5, 0xc0, 0x61, 0x2, 0x46a, 0x46c, 0x3, 0x2, 0x2, 0x2, 0x46b, + 0x466, 0x3, 0x2, 0x2, 0x2, 0x46c, 0x46f, 0x3, 0x2, 0x2, 0x2, 0x46d, + 0x46b, 0x3, 0x2, 0x2, 0x2, 0x46d, 0x46e, 0x3, 0x2, 0x2, 0x2, 0x46e, + 0x471, 0x3, 0x2, 0x2, 0x2, 0x46f, 0x46d, 0x3, 0x2, 0x2, 0x2, 0x470, + 0x472, 0x5, 0x2c, 0x17, 0x2, 0x471, 0x470, 0x3, 0x2, 0x2, 0x2, 0x471, + 0x472, 0x3, 0x2, 0x2, 0x2, 0x472, 0x65, 0x3, 0x2, 0x2, 0x2, 0x473, 0x475, + 0x7, 0xd0, 0x2, 0x2, 0x474, 0x476, 0x5, 0x6e, 0x38, 0x2, 0x475, 0x474, + 0x3, 0x2, 0x2, 0x2, 0x475, 0x476, 0x3, 0x2, 0x2, 0x2, 0x476, 0x477, + 0x3, 0x2, 0x2, 0x2, 0x477, 0x478, 0x7, 0x8d, 0x2, 0x2, 0x478, 0x47a, + 0x5, 0xac, 0x57, 0x2, 0x479, 0x47b, 0x5, 0x7a, 0x3e, 0x2, 0x47a, 0x479, + 0x3, 0x2, 0x2, 0x2, 0x47a, 0x47b, 0x3, 0x2, 0x2, 0x2, 0x47b, 0x47d, + 0x3, 0x2, 0x2, 0x2, 0x47c, 0x47e, 0x5, 0x80, 0x41, 0x2, 0x47d, 0x47c, + 0x3, 0x2, 0x2, 0x2, 0x47d, 0x47e, 0x3, 0x2, 0x2, 0x2, 0x47e, 0x47f, + 0x3, 0x2, 0x2, 0x2, 0x47f, 0x480, 0x7, 0xda, 0x2, 0x2, 0x480, 0x67, + 0x3, 0x2, 0x2, 0x2, 0x481, 0x487, 0x5, 0x6a, 0x36, 0x2, 0x482, 0x483, + 0x7, 0xaa, 0x2, 0x2, 0x483, 0x484, 0x7, 0x6, 0x2, 0x2, 0x484, 0x486, + 0x5, 0x6a, 0x36, 0x2, 0x485, 0x482, 0x3, 0x2, 0x2, 0x2, 0x486, 0x489, + 0x3, 0x2, 0x2, 0x2, 0x487, 0x485, 0x3, 0x2, 0x2, 0x2, 0x487, 0x488, + 0x3, 0x2, 0x2, 0x2, 0x488, 0x69, 0x3, 0x2, 0x2, 0x2, 0x489, 0x487, 0x3, + 0x2, 0x2, 0x2, 0x48a, 0x490, 0x5, 0x6c, 0x37, 0x2, 0x48b, 0x48c, 0x7, + 0xd0, 0x2, 0x2, 0x48c, 0x48d, 0x5, 0x68, 0x35, 0x2, 0x48d, 0x48e, 0x7, + 0xda, 0x2, 0x2, 0x48e, 0x490, 0x3, 0x2, 0x2, 0x2, 0x48f, 0x48a, 0x3, + 0x2, 0x2, 0x2, 0x48f, 0x48b, 0x3, 0x2, 0x2, 0x2, 0x490, 0x6b, 0x3, 0x2, + 0x2, 0x2, 0x491, 0x493, 0x5, 0x6e, 0x38, 0x2, 0x492, 0x491, 0x3, 0x2, + 0x2, 0x2, 0x492, 0x493, 0x3, 0x2, 0x2, 0x2, 0x493, 0x494, 0x3, 0x2, + 0x2, 0x2, 0x494, 0x496, 0x7, 0x8d, 0x2, 0x2, 0x495, 0x497, 0x7, 0x31, + 0x2, 0x2, 0x496, 0x495, 0x3, 0x2, 0x2, 0x2, 0x496, 0x497, 0x3, 0x2, + 0x2, 0x2, 0x497, 0x499, 0x3, 0x2, 0x2, 0x2, 0x498, 0x49a, 0x5, 0x70, + 0x39, 0x2, 0x499, 0x498, 0x3, 0x2, 0x2, 0x2, 0x499, 0x49a, 0x3, 0x2, + 0x2, 0x2, 0x49a, 0x49b, 0x3, 0x2, 0x2, 0x2, 0x49b, 0x49d, 0x5, 0xac, + 0x57, 0x2, 0x49c, 0x49e, 0x5, 0x72, 0x3a, 0x2, 0x49d, 0x49c, 0x3, 0x2, + 0x2, 0x2, 0x49d, 0x49e, 0x3, 0x2, 0x2, 0x2, 0x49e, 0x4a0, 0x3, 0x2, + 0x2, 0x2, 0x49f, 0x4a1, 0x5, 0x74, 0x3b, 0x2, 0x4a0, 0x49f, 0x3, 0x2, + 0x2, 0x2, 0x4a0, 0x4a1, 0x3, 0x2, 0x2, 0x2, 0x4a1, 0x4a3, 0x3, 0x2, + 0x2, 0x2, 0x4a2, 0x4a4, 0x5, 0x76, 0x3c, 0x2, 0x4a3, 0x4a2, 0x3, 0x2, + 0x2, 0x2, 0x4a3, 0x4a4, 0x3, 0x2, 0x2, 0x2, 0x4a4, 0x4a6, 0x3, 0x2, + 0x2, 0x2, 0x4a5, 0x4a7, 0x5, 0x78, 0x3d, 0x2, 0x4a6, 0x4a5, 0x3, 0x2, + 0x2, 0x2, 0x4a6, 0x4a7, 0x3, 0x2, 0x2, 0x2, 0x4a7, 0x4a9, 0x3, 0x2, + 0x2, 0x2, 0x4a8, 0x4aa, 0x5, 0x7a, 0x3e, 0x2, 0x4a9, 0x4a8, 0x3, 0x2, + 0x2, 0x2, 0x4a9, 0x4aa, 0x3, 0x2, 0x2, 0x2, 0x4aa, 0x4ad, 0x3, 0x2, + 0x2, 0x2, 0x4ab, 0x4ac, 0x7, 0xb6, 0x2, 0x2, 0x4ac, 0x4ae, 0x9, 0x8, + 0x2, 0x2, 0x4ad, 0x4ab, 0x3, 0x2, 0x2, 0x2, 0x4ad, 0x4ae, 0x3, 0x2, + 0x2, 0x2, 0x4ae, 0x4b1, 0x3, 0x2, 0x2, 0x2, 0x4af, 0x4b0, 0x7, 0xb6, + 0x2, 0x2, 0x4b0, 0x4b2, 0x7, 0xa4, 0x2, 0x2, 0x4b1, 0x4af, 0x3, 0x2, + 0x2, 0x2, 0x4b1, 0x4b2, 0x3, 0x2, 0x2, 0x2, 0x4b2, 0x4b4, 0x3, 0x2, + 0x2, 0x2, 0x4b3, 0x4b5, 0x5, 0x7c, 0x3f, 0x2, 0x4b4, 0x4b3, 0x3, 0x2, + 0x2, 0x2, 0x4b4, 0x4b5, 0x3, 0x2, 0x2, 0x2, 0x4b5, 0x4b7, 0x3, 0x2, + 0x2, 0x2, 0x4b6, 0x4b8, 0x5, 0x7e, 0x40, 0x2, 0x4b7, 0x4b6, 0x3, 0x2, + 0x2, 0x2, 0x4b7, 0x4b8, 0x3, 0x2, 0x2, 0x2, 0x4b8, 0x4ba, 0x3, 0x2, + 0x2, 0x2, 0x4b9, 0x4bb, 0x5, 0x82, 0x42, 0x2, 0x4ba, 0x4b9, 0x3, 0x2, + 0x2, 0x2, 0x4ba, 0x4bb, 0x3, 0x2, 0x2, 0x2, 0x4bb, 0x4bd, 0x3, 0x2, + 0x2, 0x2, 0x4bc, 0x4be, 0x5, 0x84, 0x43, 0x2, 0x4bd, 0x4bc, 0x3, 0x2, + 0x2, 0x2, 0x4bd, 0x4be, 0x3, 0x2, 0x2, 0x2, 0x4be, 0x4c0, 0x3, 0x2, + 0x2, 0x2, 0x4bf, 0x4c1, 0x5, 0x86, 0x44, 0x2, 0x4c0, 0x4bf, 0x3, 0x2, + 0x2, 0x2, 0x4c0, 0x4c1, 0x3, 0x2, 0x2, 0x2, 0x4c1, 0x6d, 0x3, 0x2, 0x2, + 0x2, 0x4c2, 0x4c3, 0x7, 0xb6, 0x2, 0x2, 0x4c3, 0x4c4, 0x5, 0xac, 0x57, + 0x2, 0x4c4, 0x6f, 0x3, 0x2, 0x2, 0x2, 0x4c5, 0x4c6, 0x7, 0xa3, 0x2, + 0x2, 0x4c6, 0x4c9, 0x7, 0xbd, 0x2, 0x2, 0x4c7, 0x4c8, 0x7, 0xb6, 0x2, + 0x2, 0x4c8, 0x4ca, 0x7, 0x9f, 0x2, 0x2, 0x4c9, 0x4c7, 0x3, 0x2, 0x2, + 0x2, 0x4c9, 0x4ca, 0x3, 0x2, 0x2, 0x2, 0x4ca, 0x71, 0x3, 0x2, 0x2, 0x2, + 0x4cb, 0x4cc, 0x7, 0x43, 0x2, 0x2, 0x4cc, 0x4cd, 0x5, 0x88, 0x45, 0x2, + 0x4cd, 0x73, 0x3, 0x2, 0x2, 0x2, 0x4ce, 0x4d0, 0x9, 0x9, 0x2, 0x2, 0x4cf, + 0x4ce, 0x3, 0x2, 0x2, 0x2, 0x4cf, 0x4d0, 0x3, 0x2, 0x2, 0x2, 0x4d0, + 0x4d1, 0x3, 0x2, 0x2, 0x2, 0x4d1, 0x4d2, 0x7, 0xb, 0x2, 0x2, 0x4d2, + 0x4d3, 0x7, 0x59, 0x2, 0x2, 0x4d3, 0x4d4, 0x5, 0xac, 0x57, 0x2, 0x4d4, + 0x75, 0x3, 0x2, 0x2, 0x2, 0x4d5, 0x4d6, 0x7, 0x7e, 0x2, 0x2, 0x4d6, + 0x4d7, 0x5, 0xb0, 0x59, 0x2, 0x4d7, 0x77, 0x3, 0x2, 0x2, 0x2, 0x4d8, + 0x4d9, 0x7, 0xb5, 0x2, 0x2, 0x4d9, 0x4da, 0x5, 0xb0, 0x59, 0x2, 0x4da, + 0x79, 0x3, 0x2, 0x2, 0x2, 0x4db, 0x4dc, 0x7, 0x48, 0x2, 0x2, 0x4dc, + 0x4e3, 0x7, 0x14, 0x2, 0x2, 0x4dd, 0x4de, 0x9, 0x8, 0x2, 0x2, 0x4de, + 0x4df, 0x7, 0xd0, 0x2, 0x2, 0x4df, 0x4e0, 0x5, 0xac, 0x57, 0x2, 0x4e0, + 0x4e1, 0x7, 0xda, 0x2, 0x2, 0x4e1, 0x4e4, 0x3, 0x2, 0x2, 0x2, 0x4e2, + 0x4e4, 0x5, 0xac, 0x57, 0x2, 0x4e3, 0x4dd, 0x3, 0x2, 0x2, 0x2, 0x4e3, + 0x4e2, 0x3, 0x2, 0x2, 0x2, 0x4e4, 0x7b, 0x3, 0x2, 0x2, 0x2, 0x4e5, 0x4e6, + 0x7, 0x49, 0x2, 0x2, 0x4e6, 0x4e7, 0x5, 0xb0, 0x59, 0x2, 0x4e7, 0x7d, + 0x3, 0x2, 0x2, 0x2, 0x4e8, 0x4e9, 0x7, 0x79, 0x2, 0x2, 0x4e9, 0x4ea, + 0x7, 0x14, 0x2, 0x2, 0x4ea, 0x4eb, 0x5, 0x94, 0x4b, 0x2, 0x4eb, 0x7f, + 0x3, 0x2, 0x2, 0x2, 0x4ec, 0x4ed, 0x7, 0x79, 0x2, 0x2, 0x4ed, 0x4ee, + 0x7, 0x14, 0x2, 0x2, 0x4ee, 0x4ef, 0x5, 0xac, 0x57, 0x2, 0x4ef, 0x81, + 0x3, 0x2, 0x2, 0x2, 0x4f0, 0x4f1, 0x7, 0x62, 0x2, 0x2, 0x4f1, 0x4f2, + 0x5, 0x92, 0x4a, 0x2, 0x4f2, 0x4f3, 0x7, 0x14, 0x2, 0x2, 0x4f3, 0x4f4, + 0x5, 0xac, 0x57, 0x2, 0x4f4, 0x83, 0x3, 0x2, 0x2, 0x2, 0x4f5, 0x4f6, + 0x7, 0x62, 0x2, 0x2, 0x4f6, 0x4f9, 0x5, 0x92, 0x4a, 0x2, 0x4f7, 0x4f8, + 0x7, 0xb6, 0x2, 0x2, 0x4f8, 0x4fa, 0x7, 0x9f, 0x2, 0x2, 0x4f9, 0x4f7, + 0x3, 0x2, 0x2, 0x2, 0x4f9, 0x4fa, 0x3, 0x2, 0x2, 0x2, 0x4fa, 0x85, 0x3, + 0x2, 0x2, 0x2, 0x4fb, 0x4fc, 0x7, 0x91, 0x2, 0x2, 0x4fc, 0x4fd, 0x5, + 0x9a, 0x4e, 0x2, 0x4fd, 0x87, 0x3, 0x2, 0x2, 0x2, 0x4fe, 0x4ff, 0x8, + 0x45, 0x1, 0x2, 0x4ff, 0x501, 0x5, 0xbc, 0x5f, 0x2, 0x500, 0x502, 0x7, + 0x3d, 0x2, 0x2, 0x501, 0x500, 0x3, 0x2, 0x2, 0x2, 0x501, 0x502, 0x3, + 0x2, 0x2, 0x2, 0x502, 0x504, 0x3, 0x2, 0x2, 0x2, 0x503, 0x505, 0x5, + 0x90, 0x49, 0x2, 0x504, 0x503, 0x3, 0x2, 0x2, 0x2, 0x504, 0x505, 0x3, + 0x2, 0x2, 0x2, 0x505, 0x50b, 0x3, 0x2, 0x2, 0x2, 0x506, 0x507, 0x7, + 0xd0, 0x2, 0x2, 0x507, 0x508, 0x5, 0x88, 0x45, 0x2, 0x508, 0x509, 0x7, + 0xda, 0x2, 0x2, 0x509, 0x50b, 0x3, 0x2, 0x2, 0x2, 0x50a, 0x4fe, 0x3, + 0x2, 0x2, 0x2, 0x50a, 0x506, 0x3, 0x2, 0x2, 0x2, 0x50b, 0x51d, 0x3, + 0x2, 0x2, 0x2, 0x50c, 0x50d, 0xc, 0x5, 0x2, 0x2, 0x50d, 0x50e, 0x5, + 0x8c, 0x47, 0x2, 0x50e, 0x50f, 0x5, 0x88, 0x45, 0x6, 0x50f, 0x51c, 0x3, + 0x2, 0x2, 0x2, 0x510, 0x512, 0xc, 0x6, 0x2, 0x2, 0x511, 0x513, 0x9, + 0xa, 0x2, 0x2, 0x512, 0x511, 0x3, 0x2, 0x2, 0x2, 0x512, 0x513, 0x3, + 0x2, 0x2, 0x2, 0x513, 0x515, 0x3, 0x2, 0x2, 0x2, 0x514, 0x516, 0x5, + 0x8a, 0x46, 0x2, 0x515, 0x514, 0x3, 0x2, 0x2, 0x2, 0x515, 0x516, 0x3, + 0x2, 0x2, 0x2, 0x516, 0x517, 0x3, 0x2, 0x2, 0x2, 0x517, 0x518, 0x7, + 0x59, 0x2, 0x2, 0x518, 0x519, 0x5, 0x88, 0x45, 0x2, 0x519, 0x51a, 0x5, + 0x8e, 0x48, 0x2, 0x51a, 0x51c, 0x3, 0x2, 0x2, 0x2, 0x51b, 0x50c, 0x3, + 0x2, 0x2, 0x2, 0x51b, 0x510, 0x3, 0x2, 0x2, 0x2, 0x51c, 0x51f, 0x3, + 0x2, 0x2, 0x2, 0x51d, 0x51b, 0x3, 0x2, 0x2, 0x2, 0x51d, 0x51e, 0x3, + 0x2, 0x2, 0x2, 0x51e, 0x89, 0x3, 0x2, 0x2, 0x2, 0x51f, 0x51d, 0x3, 0x2, + 0x2, 0x2, 0x520, 0x522, 0x9, 0xb, 0x2, 0x2, 0x521, 0x520, 0x3, 0x2, + 0x2, 0x2, 0x521, 0x522, 0x3, 0x2, 0x2, 0x2, 0x522, 0x523, 0x3, 0x2, + 0x2, 0x2, 0x523, 0x52a, 0x7, 0x53, 0x2, 0x2, 0x524, 0x526, 0x7, 0x53, + 0x2, 0x2, 0x525, 0x527, 0x9, 0xb, 0x2, 0x2, 0x526, 0x525, 0x3, 0x2, + 0x2, 0x2, 0x526, 0x527, 0x3, 0x2, 0x2, 0x2, 0x527, 0x52a, 0x3, 0x2, + 0x2, 0x2, 0x528, 0x52a, 0x9, 0xb, 0x2, 0x2, 0x529, 0x521, 0x3, 0x2, + 0x2, 0x2, 0x529, 0x524, 0x3, 0x2, 0x2, 0x2, 0x529, 0x528, 0x3, 0x2, + 0x2, 0x2, 0x52a, 0x54c, 0x3, 0x2, 0x2, 0x2, 0x52b, 0x52d, 0x9, 0xc, + 0x2, 0x2, 0x52c, 0x52b, 0x3, 0x2, 0x2, 0x2, 0x52c, 0x52d, 0x3, 0x2, + 0x2, 0x2, 0x52d, 0x52e, 0x3, 0x2, 0x2, 0x2, 0x52e, 0x530, 0x9, 0xd, + 0x2, 0x2, 0x52f, 0x531, 0x7, 0x7a, 0x2, 0x2, 0x530, 0x52f, 0x3, 0x2, + 0x2, 0x2, 0x530, 0x531, 0x3, 0x2, 0x2, 0x2, 0x531, 0x53a, 0x3, 0x2, + 0x2, 0x2, 0x532, 0x534, 0x9, 0xd, 0x2, 0x2, 0x533, 0x535, 0x7, 0x7a, + 0x2, 0x2, 0x534, 0x533, 0x3, 0x2, 0x2, 0x2, 0x534, 0x535, 0x3, 0x2, + 0x2, 0x2, 0x535, 0x537, 0x3, 0x2, 0x2, 0x2, 0x536, 0x538, 0x9, 0xc, + 0x2, 0x2, 0x537, 0x536, 0x3, 0x2, 0x2, 0x2, 0x537, 0x538, 0x3, 0x2, + 0x2, 0x2, 0x538, 0x53a, 0x3, 0x2, 0x2, 0x2, 0x539, 0x52c, 0x3, 0x2, + 0x2, 0x2, 0x539, 0x532, 0x3, 0x2, 0x2, 0x2, 0x53a, 0x54c, 0x3, 0x2, + 0x2, 0x2, 0x53b, 0x53d, 0x9, 0xe, 0x2, 0x2, 0x53c, 0x53b, 0x3, 0x2, + 0x2, 0x2, 0x53c, 0x53d, 0x3, 0x2, 0x2, 0x2, 0x53d, 0x53e, 0x3, 0x2, + 0x2, 0x2, 0x53e, 0x540, 0x7, 0x44, 0x2, 0x2, 0x53f, 0x541, 0x7, 0x7a, + 0x2, 0x2, 0x540, 0x53f, 0x3, 0x2, 0x2, 0x2, 0x540, 0x541, 0x3, 0x2, + 0x2, 0x2, 0x541, 0x54a, 0x3, 0x2, 0x2, 0x2, 0x542, 0x544, 0x7, 0x44, + 0x2, 0x2, 0x543, 0x545, 0x7, 0x7a, 0x2, 0x2, 0x544, 0x543, 0x3, 0x2, + 0x2, 0x2, 0x544, 0x545, 0x3, 0x2, 0x2, 0x2, 0x545, 0x547, 0x3, 0x2, + 0x2, 0x2, 0x546, 0x548, 0x9, 0xe, 0x2, 0x2, 0x547, 0x546, 0x3, 0x2, + 0x2, 0x2, 0x547, 0x548, 0x3, 0x2, 0x2, 0x2, 0x548, 0x54a, 0x3, 0x2, + 0x2, 0x2, 0x549, 0x53c, 0x3, 0x2, 0x2, 0x2, 0x549, 0x542, 0x3, 0x2, + 0x2, 0x2, 0x54a, 0x54c, 0x3, 0x2, 0x2, 0x2, 0x54b, 0x529, 0x3, 0x2, + 0x2, 0x2, 0x54b, 0x539, 0x3, 0x2, 0x2, 0x2, 0x54b, 0x549, 0x3, 0x2, + 0x2, 0x2, 0x54c, 0x8b, 0x3, 0x2, 0x2, 0x2, 0x54d, 0x54f, 0x9, 0xa, 0x2, + 0x2, 0x54e, 0x54d, 0x3, 0x2, 0x2, 0x2, 0x54e, 0x54f, 0x3, 0x2, 0x2, + 0x2, 0x54f, 0x550, 0x3, 0x2, 0x2, 0x2, 0x550, 0x551, 0x7, 0x20, 0x2, + 0x2, 0x551, 0x554, 0x7, 0x59, 0x2, 0x2, 0x552, 0x554, 0x7, 0xc5, 0x2, + 0x2, 0x553, 0x54e, 0x3, 0x2, 0x2, 0x2, 0x553, 0x552, 0x3, 0x2, 0x2, + 0x2, 0x554, 0x8d, 0x3, 0x2, 0x2, 0x2, 0x555, 0x556, 0x7, 0x76, 0x2, + 0x2, 0x556, 0x55f, 0x5, 0xac, 0x57, 0x2, 0x557, 0x558, 0x7, 0xad, 0x2, + 0x2, 0x558, 0x559, 0x7, 0xd0, 0x2, 0x2, 0x559, 0x55a, 0x5, 0xac, 0x57, + 0x2, 0x55a, 0x55b, 0x7, 0xda, 0x2, 0x2, 0x55b, 0x55f, 0x3, 0x2, 0x2, + 0x2, 0x55c, 0x55d, 0x7, 0xad, 0x2, 0x2, 0x55d, 0x55f, 0x5, 0xac, 0x57, + 0x2, 0x55e, 0x555, 0x3, 0x2, 0x2, 0x2, 0x55e, 0x557, 0x3, 0x2, 0x2, + 0x2, 0x55e, 0x55c, 0x3, 0x2, 0x2, 0x2, 0x55f, 0x8f, 0x3, 0x2, 0x2, 0x2, + 0x560, 0x561, 0x7, 0x8b, 0x2, 0x2, 0x561, 0x564, 0x5, 0x98, 0x4d, 0x2, + 0x562, 0x563, 0x7, 0x75, 0x2, 0x2, 0x563, 0x565, 0x5, 0x98, 0x4d, 0x2, + 0x564, 0x562, 0x3, 0x2, 0x2, 0x2, 0x564, 0x565, 0x3, 0x2, 0x2, 0x2, + 0x565, 0x91, 0x3, 0x2, 0x2, 0x2, 0x566, 0x569, 0x5, 0xb0, 0x59, 0x2, + 0x567, 0x568, 0x9, 0xf, 0x2, 0x2, 0x568, 0x56a, 0x5, 0xb0, 0x59, 0x2, + 0x569, 0x567, 0x3, 0x2, 0x2, 0x2, 0x569, 0x56a, 0x3, 0x2, 0x2, 0x2, + 0x56a, 0x93, 0x3, 0x2, 0x2, 0x2, 0x56b, 0x570, 0x5, 0x96, 0x4c, 0x2, + 0x56c, 0x56d, 0x7, 0xc5, 0x2, 0x2, 0x56d, 0x56f, 0x5, 0x96, 0x4c, 0x2, + 0x56e, 0x56c, 0x3, 0x2, 0x2, 0x2, 0x56f, 0x572, 0x3, 0x2, 0x2, 0x2, + 0x570, 0x56e, 0x3, 0x2, 0x2, 0x2, 0x570, 0x571, 0x3, 0x2, 0x2, 0x2, + 0x571, 0x95, 0x3, 0x2, 0x2, 0x2, 0x572, 0x570, 0x3, 0x2, 0x2, 0x2, 0x573, + 0x575, 0x5, 0xb0, 0x59, 0x2, 0x574, 0x576, 0x9, 0x10, 0x2, 0x2, 0x575, + 0x574, 0x3, 0x2, 0x2, 0x2, 0x575, 0x576, 0x3, 0x2, 0x2, 0x2, 0x576, + 0x579, 0x3, 0x2, 0x2, 0x2, 0x577, 0x578, 0x7, 0x74, 0x2, 0x2, 0x578, + 0x57a, 0x9, 0x11, 0x2, 0x2, 0x579, 0x577, 0x3, 0x2, 0x2, 0x2, 0x579, + 0x57a, 0x3, 0x2, 0x2, 0x2, 0x57a, 0x57d, 0x3, 0x2, 0x2, 0x2, 0x57b, + 0x57c, 0x7, 0x1b, 0x2, 0x2, 0x57c, 0x57e, 0x7, 0xbf, 0x2, 0x2, 0x57d, + 0x57b, 0x3, 0x2, 0x2, 0x2, 0x57d, 0x57e, 0x3, 0x2, 0x2, 0x2, 0x57e, + 0x97, 0x3, 0x2, 0x2, 0x2, 0x57f, 0x582, 0x5, 0xca, 0x66, 0x2, 0x580, + 0x581, 0x7, 0xdc, 0x2, 0x2, 0x581, 0x583, 0x5, 0xca, 0x66, 0x2, 0x582, + 0x580, 0x3, 0x2, 0x2, 0x2, 0x582, 0x583, 0x3, 0x2, 0x2, 0x2, 0x583, + 0x99, 0x3, 0x2, 0x2, 0x2, 0x584, 0x589, 0x5, 0x9c, 0x4f, 0x2, 0x585, + 0x586, 0x7, 0xc5, 0x2, 0x2, 0x586, 0x588, 0x5, 0x9c, 0x4f, 0x2, 0x587, + 0x585, 0x3, 0x2, 0x2, 0x2, 0x588, 0x58b, 0x3, 0x2, 0x2, 0x2, 0x589, + 0x587, 0x3, 0x2, 0x2, 0x2, 0x589, 0x58a, 0x3, 0x2, 0x2, 0x2, 0x58a, + 0x9b, 0x3, 0x2, 0x2, 0x2, 0x58b, 0x589, 0x3, 0x2, 0x2, 0x2, 0x58c, 0x58d, + 0x5, 0xd6, 0x6c, 0x2, 0x58d, 0x58e, 0x7, 0xca, 0x2, 0x2, 0x58e, 0x58f, + 0x5, 0xcc, 0x67, 0x2, 0x58f, 0x9d, 0x3, 0x2, 0x2, 0x2, 0x590, 0x591, + 0x7, 0x90, 0x2, 0x2, 0x591, 0x592, 0x5, 0x9a, 0x4e, 0x2, 0x592, 0x9f, + 0x3, 0x2, 0x2, 0x2, 0x593, 0x594, 0x7, 0x92, 0x2, 0x2, 0x594, 0x595, + 0x7, 0x1f, 0x2, 0x2, 0x595, 0x596, 0x7, 0x22, 0x2, 0x2, 0x596, 0x5be, + 0x5, 0xc6, 0x64, 0x2, 0x597, 0x598, 0x7, 0x92, 0x2, 0x2, 0x598, 0x599, + 0x7, 0x1f, 0x2, 0x2, 0x599, 0x59a, 0x7, 0x2f, 0x2, 0x2, 0x59a, 0x5be, + 0x5, 0xc0, 0x61, 0x2, 0x59b, 0x59c, 0x7, 0x92, 0x2, 0x2, 0x59c, 0x59e, + 0x7, 0x1f, 0x2, 0x2, 0x59d, 0x59f, 0x7, 0x9c, 0x2, 0x2, 0x59e, 0x59d, + 0x3, 0x2, 0x2, 0x2, 0x59e, 0x59f, 0x3, 0x2, 0x2, 0x2, 0x59f, 0x5a1, + 0x3, 0x2, 0x2, 0x2, 0x5a0, 0x5a2, 0x7, 0x9a, 0x2, 0x2, 0x5a1, 0x5a0, + 0x3, 0x2, 0x2, 0x2, 0x5a1, 0x5a2, 0x3, 0x2, 0x2, 0x2, 0x5a2, 0x5a3, + 0x3, 0x2, 0x2, 0x2, 0x5a3, 0x5be, 0x5, 0xc0, 0x61, 0x2, 0x5a4, 0x5a5, + 0x7, 0x92, 0x2, 0x2, 0x5a5, 0x5be, 0x7, 0x23, 0x2, 0x2, 0x5a6, 0x5a7, + 0x7, 0x92, 0x2, 0x2, 0x5a7, 0x5aa, 0x7, 0x2e, 0x2, 0x2, 0x5a8, 0x5a9, + 0x7, 0x43, 0x2, 0x2, 0x5a9, 0x5ab, 0x5, 0xc6, 0x64, 0x2, 0x5aa, 0x5a8, + 0x3, 0x2, 0x2, 0x2, 0x5aa, 0x5ab, 0x3, 0x2, 0x2, 0x2, 0x5ab, 0x5be, + 0x3, 0x2, 0x2, 0x2, 0x5ac, 0x5ae, 0x7, 0x92, 0x2, 0x2, 0x5ad, 0x5af, + 0x7, 0x9c, 0x2, 0x2, 0x5ae, 0x5ad, 0x3, 0x2, 0x2, 0x2, 0x5ae, 0x5af, + 0x3, 0x2, 0x2, 0x2, 0x5af, 0x5b0, 0x3, 0x2, 0x2, 0x2, 0x5b0, 0x5b3, + 0x7, 0x9b, 0x2, 0x2, 0x5b1, 0x5b2, 0x9, 0x12, 0x2, 0x2, 0x5b2, 0x5b4, + 0x5, 0xc6, 0x64, 0x2, 0x5b3, 0x5b1, 0x3, 0x2, 0x2, 0x2, 0x5b3, 0x5b4, + 0x3, 0x2, 0x2, 0x2, 0x5b4, 0x5b8, 0x3, 0x2, 0x2, 0x2, 0x5b5, 0x5b6, + 0x7, 0x61, 0x2, 0x2, 0x5b6, 0x5b9, 0x7, 0xbf, 0x2, 0x2, 0x5b7, 0x5b9, + 0x5, 0x78, 0x3d, 0x2, 0x5b8, 0x5b5, 0x3, 0x2, 0x2, 0x2, 0x5b8, 0x5b7, + 0x3, 0x2, 0x2, 0x2, 0x5b8, 0x5b9, 0x3, 0x2, 0x2, 0x2, 0x5b9, 0x5bb, + 0x3, 0x2, 0x2, 0x2, 0x5ba, 0x5bc, 0x5, 0x84, 0x43, 0x2, 0x5bb, 0x5ba, + 0x3, 0x2, 0x2, 0x2, 0x5bb, 0x5bc, 0x3, 0x2, 0x2, 0x2, 0x5bc, 0x5be, + 0x3, 0x2, 0x2, 0x2, 0x5bd, 0x593, 0x3, 0x2, 0x2, 0x2, 0x5bd, 0x597, + 0x3, 0x2, 0x2, 0x2, 0x5bd, 0x59b, 0x3, 0x2, 0x2, 0x2, 0x5bd, 0x5a4, + 0x3, 0x2, 0x2, 0x2, 0x5bd, 0x5a6, 0x3, 0x2, 0x2, 0x2, 0x5bd, 0x5ac, + 0x3, 0x2, 0x2, 0x2, 0x5be, 0xa1, 0x3, 0x2, 0x2, 0x2, 0x5bf, 0x5c0, 0x7, + 0x99, 0x2, 0x2, 0x5c0, 0x5c1, 0x7, 0x3f, 0x2, 0x2, 0x5c1, 0x5c2, 0x7, + 0x32, 0x2, 0x2, 0x5c2, 0x5e2, 0x5, 0xc0, 0x61, 0x2, 0x5c3, 0x5c4, 0x7, + 0x99, 0x2, 0x2, 0x5c4, 0x5c5, 0x7, 0x3f, 0x2, 0x2, 0x5c5, 0x5e2, 0x7, + 0x65, 0x2, 0x2, 0x5c6, 0x5c7, 0x7, 0x99, 0x2, 0x2, 0x5c7, 0x5c8, 0x7, + 0x83, 0x2, 0x2, 0x5c8, 0x5e2, 0x7, 0x2e, 0x2, 0x2, 0x5c9, 0x5ca, 0x7, + 0x99, 0x2, 0x2, 0x5ca, 0x5cb, 0x7, 0x83, 0x2, 0x2, 0x5cb, 0x5cc, 0x7, + 0x2f, 0x2, 0x2, 0x5cc, 0x5e2, 0x5, 0xc0, 0x61, 0x2, 0x5cd, 0x5ce, 0x7, + 0x99, 0x2, 0x2, 0x5ce, 0x5d6, 0x9, 0x13, 0x2, 0x2, 0x5cf, 0x5d0, 0x7, + 0x32, 0x2, 0x2, 0x5d0, 0x5d7, 0x7, 0x8f, 0x2, 0x2, 0x5d1, 0x5d7, 0x7, + 0x3c, 0x2, 0x2, 0x5d2, 0x5d4, 0x7, 0xa8, 0x2, 0x2, 0x5d3, 0x5d2, 0x3, + 0x2, 0x2, 0x2, 0x5d3, 0x5d4, 0x3, 0x2, 0x2, 0x2, 0x5d4, 0x5d5, 0x3, + 0x2, 0x2, 0x2, 0x5d5, 0x5d7, 0x7, 0x69, 0x2, 0x2, 0x5d6, 0x5cf, 0x3, + 0x2, 0x2, 0x2, 0x5d6, 0x5d1, 0x3, 0x2, 0x2, 0x2, 0x5d6, 0x5d3, 0x3, + 0x2, 0x2, 0x2, 0x5d7, 0x5d8, 0x3, 0x2, 0x2, 0x2, 0x5d8, 0x5e2, 0x5, + 0xc0, 0x61, 0x2, 0x5d9, 0x5da, 0x7, 0x99, 0x2, 0x2, 0x5da, 0x5db, 0x9, + 0x13, 0x2, 0x2, 0x5db, 0x5dc, 0x7, 0x88, 0x2, 0x2, 0x5dc, 0x5e2, 0x7, + 0x8f, 0x2, 0x2, 0x5dd, 0x5de, 0x7, 0x99, 0x2, 0x2, 0x5de, 0x5df, 0x7, + 0x97, 0x2, 0x2, 0x5df, 0x5e0, 0x7, 0x87, 0x2, 0x2, 0x5e0, 0x5e2, 0x5, + 0xc0, 0x61, 0x2, 0x5e1, 0x5bf, 0x3, 0x2, 0x2, 0x2, 0x5e1, 0x5c3, 0x3, + 0x2, 0x2, 0x2, 0x5e1, 0x5c6, 0x3, 0x2, 0x2, 0x2, 0x5e1, 0x5c9, 0x3, + 0x2, 0x2, 0x2, 0x5e1, 0x5cd, 0x3, 0x2, 0x2, 0x2, 0x5e1, 0x5d9, 0x3, + 0x2, 0x2, 0x2, 0x5e1, 0x5dd, 0x3, 0x2, 0x2, 0x2, 0x5e2, 0xa3, 0x3, 0x2, + 0x2, 0x2, 0x5e3, 0x5e5, 0x7, 0xa7, 0x2, 0x2, 0x5e4, 0x5e6, 0x7, 0x9c, + 0x2, 0x2, 0x5e5, 0x5e4, 0x3, 0x2, 0x2, 0x2, 0x5e5, 0x5e6, 0x3, 0x2, + 0x2, 0x2, 0x5e6, 0x5e8, 0x3, 0x2, 0x2, 0x2, 0x5e7, 0x5e9, 0x7, 0x9a, + 0x2, 0x2, 0x5e8, 0x5e7, 0x3, 0x2, 0x2, 0x2, 0x5e8, 0x5e9, 0x3, 0x2, + 0x2, 0x2, 0x5e9, 0x5ec, 0x3, 0x2, 0x2, 0x2, 0x5ea, 0x5eb, 0x7, 0x4d, + 0x2, 0x2, 0x5eb, 0x5ed, 0x7, 0x38, 0x2, 0x2, 0x5ec, 0x5ea, 0x3, 0x2, + 0x2, 0x2, 0x5ec, 0x5ed, 0x3, 0x2, 0x2, 0x2, 0x5ed, 0x5ee, 0x3, 0x2, + 0x2, 0x2, 0x5ee, 0x5f0, 0x5, 0xc0, 0x61, 0x2, 0x5ef, 0x5f1, 0x5, 0x2c, + 0x17, 0x2, 0x5f0, 0x5ef, 0x3, 0x2, 0x2, 0x2, 0x5f0, 0x5f1, 0x3, 0x2, + 0x2, 0x2, 0x5f1, 0xa5, 0x3, 0x2, 0x2, 0x2, 0x5f2, 0x5f3, 0x7, 0xac, + 0x2, 0x2, 0x5f3, 0x5f4, 0x5, 0xc6, 0x64, 0x2, 0x5f4, 0xa7, 0x3, 0x2, + 0x2, 0x2, 0x5f5, 0x5f6, 0x7, 0xb2, 0x2, 0x2, 0x5f6, 0x5f8, 0x5, 0xc0, + 0x61, 0x2, 0x5f7, 0x5f9, 0x7, 0x37, 0x2, 0x2, 0x5f8, 0x5f7, 0x3, 0x2, + 0x2, 0x2, 0x5f8, 0x5f9, 0x3, 0x2, 0x2, 0x2, 0x5f9, 0x5fc, 0x3, 0x2, + 0x2, 0x2, 0x5fa, 0x5fb, 0x7, 0x62, 0x2, 0x2, 0x5fb, 0x5fd, 0x7, 0xbd, + 0x2, 0x2, 0x5fc, 0x5fa, 0x3, 0x2, 0x2, 0x2, 0x5fc, 0x5fd, 0x3, 0x2, + 0x2, 0x2, 0x5fd, 0xa9, 0x3, 0x2, 0x2, 0x2, 0x5fe, 0x62e, 0x5, 0xd6, + 0x6c, 0x2, 0x5ff, 0x600, 0x5, 0xd6, 0x6c, 0x2, 0x600, 0x601, 0x7, 0xd0, + 0x2, 0x2, 0x601, 0x602, 0x5, 0xd6, 0x6c, 0x2, 0x602, 0x609, 0x5, 0xaa, + 0x56, 0x2, 0x603, 0x604, 0x7, 0xc5, 0x2, 0x2, 0x604, 0x605, 0x5, 0xd6, + 0x6c, 0x2, 0x605, 0x606, 0x5, 0xaa, 0x56, 0x2, 0x606, 0x608, 0x3, 0x2, + 0x2, 0x2, 0x607, 0x603, 0x3, 0x2, 0x2, 0x2, 0x608, 0x60b, 0x3, 0x2, + 0x2, 0x2, 0x609, 0x607, 0x3, 0x2, 0x2, 0x2, 0x609, 0x60a, 0x3, 0x2, + 0x2, 0x2, 0x60a, 0x60c, 0x3, 0x2, 0x2, 0x2, 0x60b, 0x609, 0x3, 0x2, + 0x2, 0x2, 0x60c, 0x60d, 0x7, 0xda, 0x2, 0x2, 0x60d, 0x62e, 0x3, 0x2, + 0x2, 0x2, 0x60e, 0x60f, 0x5, 0xd6, 0x6c, 0x2, 0x60f, 0x610, 0x7, 0xd0, + 0x2, 0x2, 0x610, 0x615, 0x5, 0xda, 0x6e, 0x2, 0x611, 0x612, 0x7, 0xc5, + 0x2, 0x2, 0x612, 0x614, 0x5, 0xda, 0x6e, 0x2, 0x613, 0x611, 0x3, 0x2, + 0x2, 0x2, 0x614, 0x617, 0x3, 0x2, 0x2, 0x2, 0x615, 0x613, 0x3, 0x2, + 0x2, 0x2, 0x615, 0x616, 0x3, 0x2, 0x2, 0x2, 0x616, 0x618, 0x3, 0x2, + 0x2, 0x2, 0x617, 0x615, 0x3, 0x2, 0x2, 0x2, 0x618, 0x619, 0x7, 0xda, + 0x2, 0x2, 0x619, 0x62e, 0x3, 0x2, 0x2, 0x2, 0x61a, 0x61b, 0x5, 0xd6, + 0x6c, 0x2, 0x61b, 0x61c, 0x7, 0xd0, 0x2, 0x2, 0x61c, 0x621, 0x5, 0xaa, + 0x56, 0x2, 0x61d, 0x61e, 0x7, 0xc5, 0x2, 0x2, 0x61e, 0x620, 0x5, 0xaa, + 0x56, 0x2, 0x61f, 0x61d, 0x3, 0x2, 0x2, 0x2, 0x620, 0x623, 0x3, 0x2, + 0x2, 0x2, 0x621, 0x61f, 0x3, 0x2, 0x2, 0x2, 0x621, 0x622, 0x3, 0x2, + 0x2, 0x2, 0x622, 0x624, 0x3, 0x2, 0x2, 0x2, 0x623, 0x621, 0x3, 0x2, + 0x2, 0x2, 0x624, 0x625, 0x7, 0xda, 0x2, 0x2, 0x625, 0x62e, 0x3, 0x2, + 0x2, 0x2, 0x626, 0x627, 0x5, 0xd6, 0x6c, 0x2, 0x627, 0x629, 0x7, 0xd0, + 0x2, 0x2, 0x628, 0x62a, 0x5, 0xac, 0x57, 0x2, 0x629, 0x628, 0x3, 0x2, + 0x2, 0x2, 0x629, 0x62a, 0x3, 0x2, 0x2, 0x2, 0x62a, 0x62b, 0x3, 0x2, + 0x2, 0x2, 0x62b, 0x62c, 0x7, 0xda, 0x2, 0x2, 0x62c, 0x62e, 0x3, 0x2, + 0x2, 0x2, 0x62d, 0x5fe, 0x3, 0x2, 0x2, 0x2, 0x62d, 0x5ff, 0x3, 0x2, + 0x2, 0x2, 0x62d, 0x60e, 0x3, 0x2, 0x2, 0x2, 0x62d, 0x61a, 0x3, 0x2, + 0x2, 0x2, 0x62d, 0x626, 0x3, 0x2, 0x2, 0x2, 0x62e, 0xab, 0x3, 0x2, 0x2, + 0x2, 0x62f, 0x634, 0x5, 0xae, 0x58, 0x2, 0x630, 0x631, 0x7, 0xc5, 0x2, + 0x2, 0x631, 0x633, 0x5, 0xae, 0x58, 0x2, 0x632, 0x630, 0x3, 0x2, 0x2, + 0x2, 0x633, 0x636, 0x3, 0x2, 0x2, 0x2, 0x634, 0x632, 0x3, 0x2, 0x2, + 0x2, 0x634, 0x635, 0x3, 0x2, 0x2, 0x2, 0x635, 0xad, 0x3, 0x2, 0x2, 0x2, + 0x636, 0x634, 0x3, 0x2, 0x2, 0x2, 0x637, 0x638, 0x5, 0xc0, 0x61, 0x2, + 0x638, 0x639, 0x7, 0xc8, 0x2, 0x2, 0x639, 0x63b, 0x3, 0x2, 0x2, 0x2, + 0x63a, 0x637, 0x3, 0x2, 0x2, 0x2, 0x63a, 0x63b, 0x3, 0x2, 0x2, 0x2, + 0x63b, 0x63c, 0x3, 0x2, 0x2, 0x2, 0x63c, 0x643, 0x7, 0xc1, 0x2, 0x2, + 0x63d, 0x63e, 0x7, 0xd0, 0x2, 0x2, 0x63e, 0x63f, 0x5, 0x68, 0x35, 0x2, + 0x63f, 0x640, 0x7, 0xda, 0x2, 0x2, 0x640, 0x643, 0x3, 0x2, 0x2, 0x2, + 0x641, 0x643, 0x5, 0xb0, 0x59, 0x2, 0x642, 0x63a, 0x3, 0x2, 0x2, 0x2, + 0x642, 0x63d, 0x3, 0x2, 0x2, 0x2, 0x642, 0x641, 0x3, 0x2, 0x2, 0x2, + 0x643, 0xaf, 0x3, 0x2, 0x2, 0x2, 0x644, 0x645, 0x8, 0x59, 0x1, 0x2, + 0x645, 0x647, 0x7, 0x15, 0x2, 0x2, 0x646, 0x648, 0x5, 0xb0, 0x59, 0x2, + 0x647, 0x646, 0x3, 0x2, 0x2, 0x2, 0x647, 0x648, 0x3, 0x2, 0x2, 0x2, + 0x648, 0x64e, 0x3, 0x2, 0x2, 0x2, 0x649, 0x64a, 0x7, 0xb4, 0x2, 0x2, + 0x64a, 0x64b, 0x5, 0xb0, 0x59, 0x2, 0x64b, 0x64c, 0x7, 0x9e, 0x2, 0x2, + 0x64c, 0x64d, 0x5, 0xb0, 0x59, 0x2, 0x64d, 0x64f, 0x3, 0x2, 0x2, 0x2, + 0x64e, 0x649, 0x3, 0x2, 0x2, 0x2, 0x64f, 0x650, 0x3, 0x2, 0x2, 0x2, + 0x650, 0x64e, 0x3, 0x2, 0x2, 0x2, 0x650, 0x651, 0x3, 0x2, 0x2, 0x2, + 0x651, 0x654, 0x3, 0x2, 0x2, 0x2, 0x652, 0x653, 0x7, 0x34, 0x2, 0x2, + 0x653, 0x655, 0x5, 0xb0, 0x59, 0x2, 0x654, 0x652, 0x3, 0x2, 0x2, 0x2, + 0x654, 0x655, 0x3, 0x2, 0x2, 0x2, 0x655, 0x656, 0x3, 0x2, 0x2, 0x2, + 0x656, 0x657, 0x7, 0x35, 0x2, 0x2, 0x657, 0x6b0, 0x3, 0x2, 0x2, 0x2, + 0x658, 0x659, 0x7, 0x16, 0x2, 0x2, 0x659, 0x65a, 0x7, 0xd0, 0x2, 0x2, + 0x65a, 0x65b, 0x5, 0xb0, 0x59, 0x2, 0x65b, 0x65c, 0x7, 0xc, 0x2, 0x2, + 0x65c, 0x65d, 0x5, 0xaa, 0x56, 0x2, 0x65d, 0x65e, 0x7, 0xda, 0x2, 0x2, + 0x65e, 0x6b0, 0x3, 0x2, 0x2, 0x2, 0x65f, 0x660, 0x7, 0x24, 0x2, 0x2, + 0x660, 0x6b0, 0x7, 0xbf, 0x2, 0x2, 0x661, 0x662, 0x7, 0x3b, 0x2, 0x2, + 0x662, 0x663, 0x7, 0xd0, 0x2, 0x2, 0x663, 0x664, 0x5, 0xce, 0x68, 0x2, + 0x664, 0x665, 0x7, 0x43, 0x2, 0x2, 0x665, 0x666, 0x5, 0xb0, 0x59, 0x2, + 0x666, 0x667, 0x7, 0xda, 0x2, 0x2, 0x667, 0x6b0, 0x3, 0x2, 0x2, 0x2, + 0x668, 0x669, 0x7, 0x55, 0x2, 0x2, 0x669, 0x66a, 0x5, 0xb0, 0x59, 0x2, + 0x66a, 0x66b, 0x5, 0xce, 0x68, 0x2, 0x66b, 0x6b0, 0x3, 0x2, 0x2, 0x2, + 0x66c, 0x66d, 0x7, 0x96, 0x2, 0x2, 0x66d, 0x66e, 0x7, 0xd0, 0x2, 0x2, + 0x66e, 0x66f, 0x5, 0xb0, 0x59, 0x2, 0x66f, 0x670, 0x7, 0x43, 0x2, 0x2, + 0x670, 0x673, 0x5, 0xb0, 0x59, 0x2, 0x671, 0x672, 0x7, 0x40, 0x2, 0x2, + 0x672, 0x674, 0x5, 0xb0, 0x59, 0x2, 0x673, 0x671, 0x3, 0x2, 0x2, 0x2, + 0x673, 0x674, 0x3, 0x2, 0x2, 0x2, 0x674, 0x675, 0x3, 0x2, 0x2, 0x2, + 0x675, 0x676, 0x7, 0xda, 0x2, 0x2, 0x676, 0x6b0, 0x3, 0x2, 0x2, 0x2, + 0x677, 0x678, 0x7, 0xa1, 0x2, 0x2, 0x678, 0x6b0, 0x7, 0xbf, 0x2, 0x2, + 0x679, 0x67a, 0x7, 0xa6, 0x2, 0x2, 0x67a, 0x67b, 0x7, 0xd0, 0x2, 0x2, + 0x67b, 0x67c, 0x9, 0x14, 0x2, 0x2, 0x67c, 0x67d, 0x7, 0xbf, 0x2, 0x2, + 0x67d, 0x67e, 0x7, 0x43, 0x2, 0x2, 0x67e, 0x67f, 0x5, 0xb0, 0x59, 0x2, + 0x67f, 0x680, 0x7, 0xda, 0x2, 0x2, 0x680, 0x6b0, 0x3, 0x2, 0x2, 0x2, + 0x681, 0x687, 0x5, 0xd6, 0x6c, 0x2, 0x682, 0x684, 0x7, 0xd0, 0x2, 0x2, + 0x683, 0x685, 0x5, 0xac, 0x57, 0x2, 0x684, 0x683, 0x3, 0x2, 0x2, 0x2, + 0x684, 0x685, 0x3, 0x2, 0x2, 0x2, 0x685, 0x686, 0x3, 0x2, 0x2, 0x2, + 0x686, 0x688, 0x7, 0xda, 0x2, 0x2, 0x687, 0x682, 0x3, 0x2, 0x2, 0x2, + 0x687, 0x688, 0x3, 0x2, 0x2, 0x2, 0x688, 0x689, 0x3, 0x2, 0x2, 0x2, + 0x689, 0x68b, 0x7, 0xd0, 0x2, 0x2, 0x68a, 0x68c, 0x7, 0x31, 0x2, 0x2, + 0x68b, 0x68a, 0x3, 0x2, 0x2, 0x2, 0x68b, 0x68c, 0x3, 0x2, 0x2, 0x2, + 0x68c, 0x68e, 0x3, 0x2, 0x2, 0x2, 0x68d, 0x68f, 0x5, 0xb2, 0x5a, 0x2, + 0x68e, 0x68d, 0x3, 0x2, 0x2, 0x2, 0x68e, 0x68f, 0x3, 0x2, 0x2, 0x2, + 0x68f, 0x690, 0x3, 0x2, 0x2, 0x2, 0x690, 0x691, 0x7, 0xda, 0x2, 0x2, + 0x691, 0x6b0, 0x3, 0x2, 0x2, 0x2, 0x692, 0x6b0, 0x5, 0xcc, 0x67, 0x2, + 0x693, 0x694, 0x7, 0xc7, 0x2, 0x2, 0x694, 0x6b0, 0x5, 0xb0, 0x59, 0x13, + 0x695, 0x696, 0x7, 0x72, 0x2, 0x2, 0x696, 0x6b0, 0x5, 0xb0, 0x59, 0xe, + 0x697, 0x698, 0x5, 0xc0, 0x61, 0x2, 0x698, 0x699, 0x7, 0xc8, 0x2, 0x2, + 0x699, 0x69b, 0x3, 0x2, 0x2, 0x2, 0x69a, 0x697, 0x3, 0x2, 0x2, 0x2, + 0x69a, 0x69b, 0x3, 0x2, 0x2, 0x2, 0x69b, 0x69c, 0x3, 0x2, 0x2, 0x2, + 0x69c, 0x6b0, 0x7, 0xc1, 0x2, 0x2, 0x69d, 0x69e, 0x7, 0xd0, 0x2, 0x2, + 0x69e, 0x69f, 0x5, 0x68, 0x35, 0x2, 0x69f, 0x6a0, 0x7, 0xda, 0x2, 0x2, + 0x6a0, 0x6b0, 0x3, 0x2, 0x2, 0x2, 0x6a1, 0x6a2, 0x7, 0xd0, 0x2, 0x2, + 0x6a2, 0x6a3, 0x5, 0xb0, 0x59, 0x2, 0x6a3, 0x6a4, 0x7, 0xda, 0x2, 0x2, + 0x6a4, 0x6b0, 0x3, 0x2, 0x2, 0x2, 0x6a5, 0x6a6, 0x7, 0xd0, 0x2, 0x2, + 0x6a6, 0x6a7, 0x5, 0xac, 0x57, 0x2, 0x6a7, 0x6a8, 0x7, 0xda, 0x2, 0x2, + 0x6a8, 0x6b0, 0x3, 0x2, 0x2, 0x2, 0x6a9, 0x6ab, 0x7, 0xce, 0x2, 0x2, + 0x6aa, 0x6ac, 0x5, 0xac, 0x57, 0x2, 0x6ab, 0x6aa, 0x3, 0x2, 0x2, 0x2, + 0x6ab, 0x6ac, 0x3, 0x2, 0x2, 0x2, 0x6ac, 0x6ad, 0x3, 0x2, 0x2, 0x2, + 0x6ad, 0x6b0, 0x7, 0xd9, 0x2, 0x2, 0x6ae, 0x6b0, 0x5, 0xb8, 0x5d, 0x2, + 0x6af, 0x644, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x658, 0x3, 0x2, 0x2, 0x2, + 0x6af, 0x65f, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x661, 0x3, 0x2, 0x2, 0x2, + 0x6af, 0x668, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x66c, 0x3, 0x2, 0x2, 0x2, + 0x6af, 0x677, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x679, 0x3, 0x2, 0x2, 0x2, + 0x6af, 0x681, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x692, 0x3, 0x2, 0x2, 0x2, + 0x6af, 0x693, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x695, 0x3, 0x2, 0x2, 0x2, + 0x6af, 0x69a, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x69d, 0x3, 0x2, 0x2, 0x2, + 0x6af, 0x6a1, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x6a5, 0x3, 0x2, 0x2, 0x2, + 0x6af, 0x6a9, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x6ae, 0x3, 0x2, 0x2, 0x2, + 0x6b0, 0x6f8, 0x3, 0x2, 0x2, 0x2, 0x6b1, 0x6b2, 0xc, 0x12, 0x2, 0x2, + 0x6b2, 0x6b3, 0x9, 0x15, 0x2, 0x2, 0x6b3, 0x6f7, 0x5, 0xb0, 0x59, 0x13, + 0x6b4, 0x6b5, 0xc, 0x11, 0x2, 0x2, 0x6b5, 0x6b6, 0x9, 0x16, 0x2, 0x2, + 0x6b6, 0x6f7, 0x5, 0xb0, 0x59, 0x12, 0x6b7, 0x6ca, 0xc, 0x10, 0x2, 0x2, + 0x6b8, 0x6cb, 0x7, 0xc9, 0x2, 0x2, 0x6b9, 0x6cb, 0x7, 0xca, 0x2, 0x2, + 0x6ba, 0x6cb, 0x7, 0xd2, 0x2, 0x2, 0x6bb, 0x6cb, 0x7, 0xcf, 0x2, 0x2, + 0x6bc, 0x6cb, 0x7, 0xcb, 0x2, 0x2, 0x6bd, 0x6cb, 0x7, 0xd1, 0x2, 0x2, + 0x6be, 0x6cb, 0x7, 0xcc, 0x2, 0x2, 0x6bf, 0x6c1, 0x7, 0x46, 0x2, 0x2, + 0x6c0, 0x6bf, 0x3, 0x2, 0x2, 0x2, 0x6c0, 0x6c1, 0x3, 0x2, 0x2, 0x2, + 0x6c1, 0x6c3, 0x3, 0x2, 0x2, 0x2, 0x6c2, 0x6c4, 0x7, 0x72, 0x2, 0x2, + 0x6c3, 0x6c2, 0x3, 0x2, 0x2, 0x2, 0x6c3, 0x6c4, 0x3, 0x2, 0x2, 0x2, + 0x6c4, 0x6c5, 0x3, 0x2, 0x2, 0x2, 0x6c5, 0x6cb, 0x7, 0x4f, 0x2, 0x2, + 0x6c6, 0x6c8, 0x7, 0x72, 0x2, 0x2, 0x6c7, 0x6c6, 0x3, 0x2, 0x2, 0x2, + 0x6c7, 0x6c8, 0x3, 0x2, 0x2, 0x2, 0x6c8, 0x6c9, 0x3, 0x2, 0x2, 0x2, + 0x6c9, 0x6cb, 0x9, 0x17, 0x2, 0x2, 0x6ca, 0x6b8, 0x3, 0x2, 0x2, 0x2, + 0x6ca, 0x6b9, 0x3, 0x2, 0x2, 0x2, 0x6ca, 0x6ba, 0x3, 0x2, 0x2, 0x2, + 0x6ca, 0x6bb, 0x3, 0x2, 0x2, 0x2, 0x6ca, 0x6bc, 0x3, 0x2, 0x2, 0x2, + 0x6ca, 0x6bd, 0x3, 0x2, 0x2, 0x2, 0x6ca, 0x6be, 0x3, 0x2, 0x2, 0x2, + 0x6ca, 0x6c0, 0x3, 0x2, 0x2, 0x2, 0x6ca, 0x6c7, 0x3, 0x2, 0x2, 0x2, + 0x6cb, 0x6cc, 0x3, 0x2, 0x2, 0x2, 0x6cc, 0x6f7, 0x5, 0xb0, 0x59, 0x11, + 0x6cd, 0x6ce, 0xc, 0xd, 0x2, 0x2, 0x6ce, 0x6cf, 0x7, 0x8, 0x2, 0x2, + 0x6cf, 0x6f7, 0x5, 0xb0, 0x59, 0xe, 0x6d0, 0x6d1, 0xc, 0xc, 0x2, 0x2, + 0x6d1, 0x6d2, 0x7, 0x78, 0x2, 0x2, 0x6d2, 0x6f7, 0x5, 0xb0, 0x59, 0xd, + 0x6d3, 0x6d5, 0xc, 0xb, 0x2, 0x2, 0x6d4, 0x6d6, 0x7, 0x72, 0x2, 0x2, + 0x6d5, 0x6d4, 0x3, 0x2, 0x2, 0x2, 0x6d5, 0x6d6, 0x3, 0x2, 0x2, 0x2, + 0x6d6, 0x6d7, 0x3, 0x2, 0x2, 0x2, 0x6d7, 0x6d8, 0x7, 0x12, 0x2, 0x2, + 0x6d8, 0x6d9, 0x5, 0xb0, 0x59, 0x2, 0x6d9, 0x6da, 0x7, 0x8, 0x2, 0x2, + 0x6da, 0x6db, 0x5, 0xb0, 0x59, 0xc, 0x6db, 0x6f7, 0x3, 0x2, 0x2, 0x2, + 0x6dc, 0x6dd, 0xc, 0xa, 0x2, 0x2, 0x6dd, 0x6de, 0x7, 0xd5, 0x2, 0x2, + 0x6de, 0x6df, 0x5, 0xb0, 0x59, 0x2, 0x6df, 0x6e0, 0x7, 0xc4, 0x2, 0x2, + 0x6e0, 0x6e1, 0x5, 0xb0, 0x59, 0xa, 0x6e1, 0x6f7, 0x3, 0x2, 0x2, 0x2, + 0x6e2, 0x6e3, 0xc, 0x15, 0x2, 0x2, 0x6e3, 0x6e4, 0x7, 0xce, 0x2, 0x2, + 0x6e4, 0x6e5, 0x5, 0xb0, 0x59, 0x2, 0x6e5, 0x6e6, 0x7, 0xd9, 0x2, 0x2, + 0x6e6, 0x6f7, 0x3, 0x2, 0x2, 0x2, 0x6e7, 0x6e8, 0xc, 0x14, 0x2, 0x2, + 0x6e8, 0x6e9, 0x7, 0xc8, 0x2, 0x2, 0x6e9, 0x6f7, 0x7, 0xbd, 0x2, 0x2, + 0x6ea, 0x6eb, 0xc, 0xf, 0x2, 0x2, 0x6eb, 0x6ed, 0x7, 0x57, 0x2, 0x2, + 0x6ec, 0x6ee, 0x7, 0x72, 0x2, 0x2, 0x6ed, 0x6ec, 0x3, 0x2, 0x2, 0x2, + 0x6ed, 0x6ee, 0x3, 0x2, 0x2, 0x2, 0x6ee, 0x6ef, 0x3, 0x2, 0x2, 0x2, + 0x6ef, 0x6f7, 0x7, 0x73, 0x2, 0x2, 0x6f0, 0x6f4, 0xc, 0x9, 0x2, 0x2, + 0x6f1, 0x6f5, 0x5, 0xd4, 0x6b, 0x2, 0x6f2, 0x6f3, 0x7, 0xc, 0x2, 0x2, + 0x6f3, 0x6f5, 0x5, 0xd6, 0x6c, 0x2, 0x6f4, 0x6f1, 0x3, 0x2, 0x2, 0x2, + 0x6f4, 0x6f2, 0x3, 0x2, 0x2, 0x2, 0x6f5, 0x6f7, 0x3, 0x2, 0x2, 0x2, + 0x6f6, 0x6b1, 0x3, 0x2, 0x2, 0x2, 0x6f6, 0x6b4, 0x3, 0x2, 0x2, 0x2, + 0x6f6, 0x6b7, 0x3, 0x2, 0x2, 0x2, 0x6f6, 0x6cd, 0x3, 0x2, 0x2, 0x2, + 0x6f6, 0x6d0, 0x3, 0x2, 0x2, 0x2, 0x6f6, 0x6d3, 0x3, 0x2, 0x2, 0x2, + 0x6f6, 0x6dc, 0x3, 0x2, 0x2, 0x2, 0x6f6, 0x6e2, 0x3, 0x2, 0x2, 0x2, + 0x6f6, 0x6e7, 0x3, 0x2, 0x2, 0x2, 0x6f6, 0x6ea, 0x3, 0x2, 0x2, 0x2, + 0x6f6, 0x6f0, 0x3, 0x2, 0x2, 0x2, 0x6f7, 0x6fa, 0x3, 0x2, 0x2, 0x2, + 0x6f8, 0x6f6, 0x3, 0x2, 0x2, 0x2, 0x6f8, 0x6f9, 0x3, 0x2, 0x2, 0x2, + 0x6f9, 0xb1, 0x3, 0x2, 0x2, 0x2, 0x6fa, 0x6f8, 0x3, 0x2, 0x2, 0x2, 0x6fb, + 0x700, 0x5, 0xb4, 0x5b, 0x2, 0x6fc, 0x6fd, 0x7, 0xc5, 0x2, 0x2, 0x6fd, + 0x6ff, 0x5, 0xb4, 0x5b, 0x2, 0x6fe, 0x6fc, 0x3, 0x2, 0x2, 0x2, 0x6ff, + 0x702, 0x3, 0x2, 0x2, 0x2, 0x700, 0x6fe, 0x3, 0x2, 0x2, 0x2, 0x700, + 0x701, 0x3, 0x2, 0x2, 0x2, 0x701, 0xb3, 0x3, 0x2, 0x2, 0x2, 0x702, 0x700, + 0x3, 0x2, 0x2, 0x2, 0x703, 0x706, 0x5, 0xb6, 0x5c, 0x2, 0x704, 0x706, + 0x5, 0xb0, 0x59, 0x2, 0x705, 0x703, 0x3, 0x2, 0x2, 0x2, 0x705, 0x704, + 0x3, 0x2, 0x2, 0x2, 0x706, 0xb5, 0x3, 0x2, 0x2, 0x2, 0x707, 0x708, 0x7, + 0xd0, 0x2, 0x2, 0x708, 0x70d, 0x5, 0xd6, 0x6c, 0x2, 0x709, 0x70a, 0x7, + 0xc5, 0x2, 0x2, 0x70a, 0x70c, 0x5, 0xd6, 0x6c, 0x2, 0x70b, 0x709, 0x3, + 0x2, 0x2, 0x2, 0x70c, 0x70f, 0x3, 0x2, 0x2, 0x2, 0x70d, 0x70b, 0x3, + 0x2, 0x2, 0x2, 0x70d, 0x70e, 0x3, 0x2, 0x2, 0x2, 0x70e, 0x710, 0x3, + 0x2, 0x2, 0x2, 0x70f, 0x70d, 0x3, 0x2, 0x2, 0x2, 0x710, 0x711, 0x7, + 0xda, 0x2, 0x2, 0x711, 0x71b, 0x3, 0x2, 0x2, 0x2, 0x712, 0x717, 0x5, + 0xd6, 0x6c, 0x2, 0x713, 0x714, 0x7, 0xc5, 0x2, 0x2, 0x714, 0x716, 0x5, + 0xd6, 0x6c, 0x2, 0x715, 0x713, 0x3, 0x2, 0x2, 0x2, 0x716, 0x719, 0x3, + 0x2, 0x2, 0x2, 0x717, 0x715, 0x3, 0x2, 0x2, 0x2, 0x717, 0x718, 0x3, + 0x2, 0x2, 0x2, 0x718, 0x71b, 0x3, 0x2, 0x2, 0x2, 0x719, 0x717, 0x3, + 0x2, 0x2, 0x2, 0x71a, 0x707, 0x3, 0x2, 0x2, 0x2, 0x71a, 0x712, 0x3, + 0x2, 0x2, 0x2, 0x71b, 0x71c, 0x3, 0x2, 0x2, 0x2, 0x71c, 0x71d, 0x7, + 0xc0, 0x2, 0x2, 0x71d, 0x71e, 0x5, 0xb0, 0x59, 0x2, 0x71e, 0xb7, 0x3, + 0x2, 0x2, 0x2, 0x71f, 0x720, 0x5, 0xc0, 0x61, 0x2, 0x720, 0x721, 0x7, + 0xc8, 0x2, 0x2, 0x721, 0x723, 0x3, 0x2, 0x2, 0x2, 0x722, 0x71f, 0x3, + 0x2, 0x2, 0x2, 0x722, 0x723, 0x3, 0x2, 0x2, 0x2, 0x723, 0x724, 0x3, + 0x2, 0x2, 0x2, 0x724, 0x725, 0x5, 0xba, 0x5e, 0x2, 0x725, 0xb9, 0x3, + 0x2, 0x2, 0x2, 0x726, 0x729, 0x5, 0xd6, 0x6c, 0x2, 0x727, 0x728, 0x7, + 0xc8, 0x2, 0x2, 0x728, 0x72a, 0x5, 0xd6, 0x6c, 0x2, 0x729, 0x727, 0x3, + 0x2, 0x2, 0x2, 0x729, 0x72a, 0x3, 0x2, 0x2, 0x2, 0x72a, 0xbb, 0x3, 0x2, + 0x2, 0x2, 0x72b, 0x72c, 0x8, 0x5f, 0x1, 0x2, 0x72c, 0x733, 0x5, 0xc0, + 0x61, 0x2, 0x72d, 0x733, 0x5, 0xbe, 0x60, 0x2, 0x72e, 0x72f, 0x7, 0xd0, + 0x2, 0x2, 0x72f, 0x730, 0x5, 0x68, 0x35, 0x2, 0x730, 0x731, 0x7, 0xda, + 0x2, 0x2, 0x731, 0x733, 0x3, 0x2, 0x2, 0x2, 0x732, 0x72b, 0x3, 0x2, + 0x2, 0x2, 0x732, 0x72d, 0x3, 0x2, 0x2, 0x2, 0x732, 0x72e, 0x3, 0x2, + 0x2, 0x2, 0x733, 0x73c, 0x3, 0x2, 0x2, 0x2, 0x734, 0x738, 0xc, 0x3, + 0x2, 0x2, 0x735, 0x739, 0x5, 0xd4, 0x6b, 0x2, 0x736, 0x737, 0x7, 0xc, + 0x2, 0x2, 0x737, 0x739, 0x5, 0xd6, 0x6c, 0x2, 0x738, 0x735, 0x3, 0x2, + 0x2, 0x2, 0x738, 0x736, 0x3, 0x2, 0x2, 0x2, 0x739, 0x73b, 0x3, 0x2, + 0x2, 0x2, 0x73a, 0x734, 0x3, 0x2, 0x2, 0x2, 0x73b, 0x73e, 0x3, 0x2, + 0x2, 0x2, 0x73c, 0x73a, 0x3, 0x2, 0x2, 0x2, 0x73c, 0x73d, 0x3, 0x2, + 0x2, 0x2, 0x73d, 0xbd, 0x3, 0x2, 0x2, 0x2, 0x73e, 0x73c, 0x3, 0x2, 0x2, + 0x2, 0x73f, 0x740, 0x5, 0xd6, 0x6c, 0x2, 0x740, 0x742, 0x7, 0xd0, 0x2, + 0x2, 0x741, 0x743, 0x5, 0xc2, 0x62, 0x2, 0x742, 0x741, 0x3, 0x2, 0x2, + 0x2, 0x742, 0x743, 0x3, 0x2, 0x2, 0x2, 0x743, 0x744, 0x3, 0x2, 0x2, + 0x2, 0x744, 0x745, 0x7, 0xda, 0x2, 0x2, 0x745, 0xbf, 0x3, 0x2, 0x2, + 0x2, 0x746, 0x747, 0x5, 0xc6, 0x64, 0x2, 0x747, 0x748, 0x7, 0xc8, 0x2, + 0x2, 0x748, 0x74a, 0x3, 0x2, 0x2, 0x2, 0x749, 0x746, 0x3, 0x2, 0x2, + 0x2, 0x749, 0x74a, 0x3, 0x2, 0x2, 0x2, 0x74a, 0x74b, 0x3, 0x2, 0x2, + 0x2, 0x74b, 0x74c, 0x5, 0xd6, 0x6c, 0x2, 0x74c, 0xc1, 0x3, 0x2, 0x2, + 0x2, 0x74d, 0x752, 0x5, 0xc4, 0x63, 0x2, 0x74e, 0x74f, 0x7, 0xc5, 0x2, + 0x2, 0x74f, 0x751, 0x5, 0xc4, 0x63, 0x2, 0x750, 0x74e, 0x3, 0x2, 0x2, + 0x2, 0x751, 0x754, 0x3, 0x2, 0x2, 0x2, 0x752, 0x750, 0x3, 0x2, 0x2, + 0x2, 0x752, 0x753, 0x3, 0x2, 0x2, 0x2, 0x753, 0xc3, 0x3, 0x2, 0x2, 0x2, + 0x754, 0x752, 0x3, 0x2, 0x2, 0x2, 0x755, 0x759, 0x5, 0xba, 0x5e, 0x2, + 0x756, 0x759, 0x5, 0xbe, 0x60, 0x2, 0x757, 0x759, 0x5, 0xcc, 0x67, 0x2, + 0x758, 0x755, 0x3, 0x2, 0x2, 0x2, 0x758, 0x756, 0x3, 0x2, 0x2, 0x2, + 0x758, 0x757, 0x3, 0x2, 0x2, 0x2, 0x759, 0xc5, 0x3, 0x2, 0x2, 0x2, 0x75a, + 0x75b, 0x5, 0xd6, 0x6c, 0x2, 0x75b, 0xc7, 0x3, 0x2, 0x2, 0x2, 0x75c, + 0x765, 0x7, 0xbb, 0x2, 0x2, 0x75d, 0x75e, 0x7, 0xc8, 0x2, 0x2, 0x75e, + 0x765, 0x9, 0x18, 0x2, 0x2, 0x75f, 0x760, 0x7, 0xbd, 0x2, 0x2, 0x760, + 0x762, 0x7, 0xc8, 0x2, 0x2, 0x761, 0x763, 0x9, 0x18, 0x2, 0x2, 0x762, + 0x761, 0x3, 0x2, 0x2, 0x2, 0x762, 0x763, 0x3, 0x2, 0x2, 0x2, 0x763, + 0x765, 0x3, 0x2, 0x2, 0x2, 0x764, 0x75c, 0x3, 0x2, 0x2, 0x2, 0x764, + 0x75d, 0x3, 0x2, 0x2, 0x2, 0x764, 0x75f, 0x3, 0x2, 0x2, 0x2, 0x765, + 0xc9, 0x3, 0x2, 0x2, 0x2, 0x766, 0x768, 0x9, 0x19, 0x2, 0x2, 0x767, + 0x766, 0x3, 0x2, 0x2, 0x2, 0x767, 0x768, 0x3, 0x2, 0x2, 0x2, 0x768, + 0x76f, 0x3, 0x2, 0x2, 0x2, 0x769, 0x770, 0x5, 0xc8, 0x65, 0x2, 0x76a, + 0x770, 0x7, 0xbc, 0x2, 0x2, 0x76b, 0x770, 0x7, 0xbd, 0x2, 0x2, 0x76c, + 0x770, 0x7, 0xbe, 0x2, 0x2, 0x76d, 0x770, 0x7, 0x51, 0x2, 0x2, 0x76e, + 0x770, 0x7, 0x70, 0x2, 0x2, 0x76f, 0x769, 0x3, 0x2, 0x2, 0x2, 0x76f, + 0x76a, 0x3, 0x2, 0x2, 0x2, 0x76f, 0x76b, 0x3, 0x2, 0x2, 0x2, 0x76f, + 0x76c, 0x3, 0x2, 0x2, 0x2, 0x76f, 0x76d, 0x3, 0x2, 0x2, 0x2, 0x76f, + 0x76e, 0x3, 0x2, 0x2, 0x2, 0x770, 0xcb, 0x3, 0x2, 0x2, 0x2, 0x771, 0x775, + 0x5, 0xca, 0x66, 0x2, 0x772, 0x775, 0x7, 0xbf, 0x2, 0x2, 0x773, 0x775, + 0x7, 0x73, 0x2, 0x2, 0x774, 0x771, 0x3, 0x2, 0x2, 0x2, 0x774, 0x772, + 0x3, 0x2, 0x2, 0x2, 0x774, 0x773, 0x3, 0x2, 0x2, 0x2, 0x775, 0xcd, 0x3, + 0x2, 0x2, 0x2, 0x776, 0x777, 0x9, 0x1a, 0x2, 0x2, 0x777, 0xcf, 0x3, + 0x2, 0x2, 0x2, 0x778, 0x779, 0x9, 0x1b, 0x2, 0x2, 0x779, 0xd1, 0x3, + 0x2, 0x2, 0x2, 0x77a, 0x77b, 0x9, 0x1c, 0x2, 0x2, 0x77b, 0xd3, 0x3, + 0x2, 0x2, 0x2, 0x77c, 0x77f, 0x7, 0xba, 0x2, 0x2, 0x77d, 0x77f, 0x5, + 0xd2, 0x6a, 0x2, 0x77e, 0x77c, 0x3, 0x2, 0x2, 0x2, 0x77e, 0x77d, 0x3, + 0x2, 0x2, 0x2, 0x77f, 0xd5, 0x3, 0x2, 0x2, 0x2, 0x780, 0x784, 0x7, 0xba, + 0x2, 0x2, 0x781, 0x784, 0x5, 0xce, 0x68, 0x2, 0x782, 0x784, 0x5, 0xd0, + 0x69, 0x2, 0x783, 0x780, 0x3, 0x2, 0x2, 0x2, 0x783, 0x781, 0x3, 0x2, + 0x2, 0x2, 0x783, 0x782, 0x3, 0x2, 0x2, 0x2, 0x784, 0xd7, 0x3, 0x2, 0x2, + 0x2, 0x785, 0x788, 0x5, 0xd6, 0x6c, 0x2, 0x786, 0x788, 0x7, 0x73, 0x2, + 0x2, 0x787, 0x785, 0x3, 0x2, 0x2, 0x2, 0x787, 0x786, 0x3, 0x2, 0x2, + 0x2, 0x788, 0xd9, 0x3, 0x2, 0x2, 0x2, 0x789, 0x78a, 0x7, 0xbf, 0x2, + 0x2, 0x78a, 0x78b, 0x7, 0xca, 0x2, 0x2, 0x78b, 0x78c, 0x5, 0xca, 0x66, + 0x2, 0x78c, 0xdb, 0x3, 0x2, 0x2, 0x2, 0x107, 0xe0, 0xe4, 0xe7, 0xea, 0xfe, 0x104, 0x10b, 0x113, 0x118, 0x11f, 0x124, 0x12b, 0x130, 0x136, 0x13c, 0x141, 0x147, 0x14c, 0x152, 0x157, 0x15d, 0x16b, 0x172, 0x179, 0x180, 0x186, 0x18b, 0x191, 0x196, 0x19c, 0x1a5, 0x1af, 0x1b9, 0x1cd, - 0x1d5, 0x1e4, 0x1eb, 0x1f9, 0x1ff, 0x205, 0x20c, 0x210, 0x213, 0x21a, - 0x21e, 0x221, 0x22c, 0x230, 0x233, 0x238, 0x23a, 0x23d, 0x240, 0x24a, - 0x24e, 0x251, 0x254, 0x259, 0x25b, 0x261, 0x267, 0x26b, 0x26e, 0x271, - 0x274, 0x277, 0x27c, 0x282, 0x286, 0x289, 0x28c, 0x290, 0x298, 0x2b2, - 0x2b4, 0x2b8, 0x2ce, 0x2d0, 0x2db, 0x2de, 0x2e7, 0x2f8, 0x303, 0x315, - 0x322, 0x333, 0x33c, 0x357, 0x359, 0x36e, 0x373, 0x378, 0x37b, 0x387, - 0x38c, 0x390, 0x393, 0x397, 0x39b, 0x3a0, 0x3a3, 0x3a7, 0x3a9, 0x3bf, - 0x3c7, 0x3ca, 0x3d4, 0x3d8, 0x3e0, 0x3e4, 0x3e9, 0x3ed, 0x3f1, 0x3f5, - 0x3f9, 0x3fb, 0x403, 0x407, 0x40a, 0x412, 0x417, 0x41c, 0x41f, 0x429, - 0x433, 0x437, 0x43c, 0x440, 0x446, 0x449, 0x44c, 0x44f, 0x45d, 0x461, - 0x465, 0x46a, 0x46d, 0x477, 0x47f, 0x482, 0x486, 0x489, 0x48d, 0x490, - 0x493, 0x496, 0x499, 0x49d, 0x4a1, 0x4a4, 0x4a7, 0x4aa, 0x4ad, 0x4b0, - 0x4b9, 0x4bf, 0x4d3, 0x4e9, 0x4f1, 0x4f4, 0x4fa, 0x502, 0x505, 0x50b, - 0x50d, 0x511, 0x516, 0x519, 0x51c, 0x520, 0x524, 0x527, 0x529, 0x52c, - 0x530, 0x534, 0x537, 0x539, 0x53b, 0x53e, 0x543, 0x54e, 0x554, 0x559, - 0x560, 0x565, 0x569, 0x56d, 0x572, 0x579, 0x58e, 0x591, 0x59a, 0x59e, - 0x5a3, 0x5a8, 0x5ab, 0x5ad, 0x5c3, 0x5c6, 0x5d1, 0x5d5, 0x5d8, 0x5dc, - 0x5e0, 0x5e8, 0x5ec, 0x5f9, 0x605, 0x611, 0x619, 0x61d, 0x624, 0x62a, - 0x632, 0x637, 0x640, 0x644, 0x663, 0x674, 0x677, 0x67b, 0x67e, 0x68a, - 0x69b, 0x69f, 0x6b0, 0x6b3, 0x6b7, 0x6ba, 0x6c5, 0x6dd, 0x6e4, 0x6e6, - 0x6e8, 0x6f0, 0x6f5, 0x6fd, 0x707, 0x70a, 0x712, 0x719, 0x722, 0x728, - 0x72c, 0x732, 0x739, 0x742, 0x748, 0x752, 0x754, 0x757, 0x75f, 0x764, - 0x76e, 0x773, 0x777, + 0x1d5, 0x1e4, 0x1eb, 0x1f9, 0x1ff, 0x205, 0x20c, 0x210, 0x213, 0x219, + 0x21c, 0x222, 0x226, 0x229, 0x234, 0x238, 0x23b, 0x240, 0x242, 0x245, + 0x248, 0x252, 0x256, 0x259, 0x25c, 0x261, 0x263, 0x26b, 0x26e, 0x271, + 0x277, 0x27b, 0x27e, 0x281, 0x284, 0x287, 0x28c, 0x292, 0x296, 0x299, + 0x29c, 0x2a0, 0x2a8, 0x2c2, 0x2c4, 0x2c8, 0x2de, 0x2e0, 0x2eb, 0x2ee, + 0x2f7, 0x308, 0x313, 0x325, 0x332, 0x343, 0x34c, 0x367, 0x369, 0x37e, + 0x383, 0x388, 0x38b, 0x397, 0x39c, 0x3a0, 0x3a3, 0x3a7, 0x3ab, 0x3b0, + 0x3b3, 0x3b7, 0x3b9, 0x3cf, 0x3d7, 0x3da, 0x3e4, 0x3e8, 0x3f0, 0x3f4, + 0x3f9, 0x3fd, 0x401, 0x405, 0x409, 0x40b, 0x413, 0x417, 0x41a, 0x422, + 0x427, 0x42c, 0x42f, 0x439, 0x443, 0x447, 0x44c, 0x450, 0x456, 0x459, + 0x45c, 0x45f, 0x46d, 0x471, 0x475, 0x47a, 0x47d, 0x487, 0x48f, 0x492, + 0x496, 0x499, 0x49d, 0x4a0, 0x4a3, 0x4a6, 0x4a9, 0x4ad, 0x4b1, 0x4b4, + 0x4b7, 0x4ba, 0x4bd, 0x4c0, 0x4c9, 0x4cf, 0x4e3, 0x4f9, 0x501, 0x504, + 0x50a, 0x512, 0x515, 0x51b, 0x51d, 0x521, 0x526, 0x529, 0x52c, 0x530, + 0x534, 0x537, 0x539, 0x53c, 0x540, 0x544, 0x547, 0x549, 0x54b, 0x54e, + 0x553, 0x55e, 0x564, 0x569, 0x570, 0x575, 0x579, 0x57d, 0x582, 0x589, + 0x59e, 0x5a1, 0x5aa, 0x5ae, 0x5b3, 0x5b8, 0x5bb, 0x5bd, 0x5d3, 0x5d6, + 0x5e1, 0x5e5, 0x5e8, 0x5ec, 0x5f0, 0x5f8, 0x5fc, 0x609, 0x615, 0x621, + 0x629, 0x62d, 0x634, 0x63a, 0x642, 0x647, 0x650, 0x654, 0x673, 0x684, + 0x687, 0x68b, 0x68e, 0x69a, 0x6ab, 0x6af, 0x6c0, 0x6c3, 0x6c7, 0x6ca, + 0x6d5, 0x6ed, 0x6f4, 0x6f6, 0x6f8, 0x700, 0x705, 0x70d, 0x717, 0x71a, + 0x722, 0x729, 0x732, 0x738, 0x73c, 0x742, 0x749, 0x752, 0x758, 0x762, + 0x764, 0x767, 0x76f, 0x774, 0x77e, 0x783, 0x787, }; atn::ATNDeserializer deserializer; diff --git a/src/Parsers/New/ClickHouseParser.g4 b/src/Parsers/New/ClickHouseParser.g4 index 28e5b1217ab..eb1908ed073 100644 --- a/src/Parsers/New/ClickHouseParser.g4 +++ b/src/Parsers/New/ClickHouseParser.g4 @@ -91,10 +91,10 @@ checkStmt: CHECK TABLE tableIdentifier partitionClause?; createStmt : (ATTACH | CREATE) DATABASE (IF NOT EXISTS)? databaseIdentifier clusterClause? engineExpr? # CreateDatabaseStmt - | (ATTACH | CREATE) DICTIONARY (IF NOT EXISTS)? tableIdentifier uuidClause? clusterClause? dictionarySchemaClause dictionaryEngineClause # CreateDictionaryStmt + | (ATTACH | CREATE (OR REPLACE)? | REPLACE) DICTIONARY (IF NOT EXISTS)? tableIdentifier uuidClause? clusterClause? dictionarySchemaClause dictionaryEngineClause # CreateDictionaryStmt | (ATTACH | CREATE) LIVE VIEW (IF NOT EXISTS)? tableIdentifier uuidClause? clusterClause? (WITH TIMEOUT DECIMAL_LITERAL?)? destinationClause? tableSchemaClause? subqueryClause # CreateLiveViewStmt | (ATTACH | CREATE) MATERIALIZED VIEW (IF NOT EXISTS)? tableIdentifier uuidClause? clusterClause? tableSchemaClause? (destinationClause | engineClause POPULATE?) subqueryClause # CreateMaterializedViewStmt - | (ATTACH | CREATE) TEMPORARY? TABLE (IF NOT EXISTS)? tableIdentifier uuidClause? clusterClause? tableSchemaClause? engineClause? subqueryClause? # CreateTableStmt + | (ATTACH | CREATE (OR REPLACE)? | REPLACE) TEMPORARY? TABLE (IF NOT EXISTS)? tableIdentifier uuidClause? clusterClause? tableSchemaClause? engineClause? subqueryClause? # CreateTableStmt | (ATTACH | CREATE) (OR REPLACE)? VIEW (IF NOT EXISTS)? tableIdentifier uuidClause? clusterClause? tableSchemaClause? subqueryClause # CreateViewStmt ; diff --git a/src/Parsers/New/ClickHouseParser.h b/src/Parsers/New/ClickHouseParser.h index c860932ba1c..db9083b608b 100644 --- a/src/Parsers/New/ClickHouseParser.h +++ b/src/Parsers/New/ClickHouseParser.h @@ -793,11 +793,13 @@ public: DictionaryEngineClauseContext *dictionaryEngineClause(); antlr4::tree::TerminalNode *ATTACH(); antlr4::tree::TerminalNode *CREATE(); + antlr4::tree::TerminalNode *REPLACE(); antlr4::tree::TerminalNode *IF(); antlr4::tree::TerminalNode *NOT(); antlr4::tree::TerminalNode *EXISTS(); UuidClauseContext *uuidClause(); ClusterClauseContext *clusterClause(); + antlr4::tree::TerminalNode *OR(); virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; @@ -870,6 +872,7 @@ public: TableIdentifierContext *tableIdentifier(); antlr4::tree::TerminalNode *ATTACH(); antlr4::tree::TerminalNode *CREATE(); + antlr4::tree::TerminalNode *REPLACE(); antlr4::tree::TerminalNode *TEMPORARY(); antlr4::tree::TerminalNode *IF(); antlr4::tree::TerminalNode *NOT(); @@ -879,6 +882,7 @@ public: TableSchemaClauseContext *tableSchemaClause(); EngineClauseContext *engineClause(); SubqueryClauseContext *subqueryClause(); + antlr4::tree::TerminalNode *OR(); virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; diff --git a/tests/queries/0_stateless/01157_replace_table.sql b/tests/queries/0_stateless/01157_replace_table.sql index 337290bb4d2..a29b381a522 100644 --- a/tests/queries/0_stateless/01157_replace_table.sql +++ b/tests/queries/0_stateless/01157_replace_table.sql @@ -8,14 +8,17 @@ create table t (n UInt64, s String default 's' || toString(n)) engine=Memory; create table dist (n int) engine=Distributed(test_shard_localhost, currentDatabase(), t); create table buf (n int) engine=Buffer(currentDatabase(), dist, 1, 10, 100, 10, 100, 1000, 1000); +system stop distributed sends dist; insert into buf values (1); replace table buf (n int) engine=Distributed(test_shard_localhost, currentDatabase(), dist); replace table dist (n int) engine=Buffer(currentDatabase(), t, 1, 10, 100, 10, 100, 1000, 1000); +system stop distributed sends buf; insert into buf values (2); replace table buf (n int) engine=Buffer(currentDatabase(), dist, 1, 10, 100, 10, 100, 1000, 1000); replace table dist (n int) engine=Distributed(test_shard_localhost, currentDatabase(), t); +system stop distributed sends dist; insert into buf values (3); replace table buf (n int) engine=Null; replace table dist (n int) engine=Null; diff --git a/tests/queries/0_stateless/01185_create_or_replace_table.reference b/tests/queries/0_stateless/01185_create_or_replace_table.reference index 84df5f0f5b5..be187d9dcd4 100644 --- a/tests/queries/0_stateless/01185_create_or_replace_table.reference +++ b/tests/queries/0_stateless/01185_create_or_replace_table.reference @@ -1,8 +1,8 @@ t1 -CREATE TABLE test_01185.t1\n(\n `n` UInt64,\n `s` String\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t1\n(\n `n` UInt64,\n `s` String\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192 t1 -CREATE TABLE test_01185.t1\n(\n `n` UInt64,\n `s` Nullable(String)\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t1\n(\n `n` UInt64,\n `s` Nullable(String)\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192 2 \N t1 -CREATE TABLE test_01185.t1\n(\n `n` UInt64\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t1\n(\n `n` UInt64\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192 3 diff --git a/tests/queries/0_stateless/01185_create_or_replace_table.sql b/tests/queries/0_stateless/01185_create_or_replace_table.sql index fe408cc7ac6..45900329b2c 100644 --- a/tests/queries/0_stateless/01185_create_or_replace_table.sql +++ b/tests/queries/0_stateless/01185_create_or_replace_table.sql @@ -1,23 +1,22 @@ -drop database if exists test_01185; -create database test_01185 engine=Atomic; +drop table if exists t1; -replace table test_01185.t1 (n UInt64, s String) engine=MergeTree order by n; -- { serverError 60 } -show tables from test_01185; -create or replace table test_01185.t1 (n UInt64, s String) engine=MergeTree order by n; -show tables from test_01185; -show create table test_01185.t1; +replace table t1 (n UInt64, s String) engine=MergeTree order by n; -- { serverError 60 } +show tables; +create or replace table t1 (n UInt64, s String) engine=MergeTree order by n; +show tables; +show create table t1; -insert into test_01185.t1 values (1, 'test'); -create or replace table test_01185.t1 (n UInt64, s Nullable(String)) engine=MergeTree order by n; -insert into test_01185.t1 values (2, null); -show tables from test_01185; -show create table test_01185.t1; -select * from test_01185.t1; +insert into t1 values (1, 'test'); +create or replace table t1 (n UInt64, s Nullable(String)) engine=MergeTree order by n; +insert into t1 values (2, null); +show tables; +show create table t1; +select * from t1; -replace table test_01185.t1 (n UInt64) engine=MergeTree order by n; -insert into test_01185.t1 values (3); -show tables from test_01185; -show create table test_01185.t1; -select * from test_01185.t1; +replace table t1 (n UInt64) engine=MergeTree order by n; +insert into t1 values (3); +show tables; +show create table t1; +select * from t1; -drop database test_01185; +drop table t1; diff --git a/tests/queries/0_stateless/01915_create_or_replace_dictionary.sql b/tests/queries/0_stateless/01915_create_or_replace_dictionary.sql index c9df6114ec9..1520dd41973 100644 --- a/tests/queries/0_stateless/01915_create_or_replace_dictionary.sql +++ b/tests/queries/0_stateless/01915_create_or_replace_dictionary.sql @@ -1,51 +1,51 @@ -DROP DATABASE IF EXISTS 01915_db; -CREATE DATABASE 01915_db ENGINE=Atomic; +DROP DATABASE IF EXISTS test_01915_db; +CREATE DATABASE test_01915_db ENGINE=Atomic; -DROP TABLE IF EXISTS 01915_db.test_source_table_1; -CREATE TABLE 01915_db.test_source_table_1 +DROP TABLE IF EXISTS test_01915_db.test_source_table_1; +CREATE TABLE test_01915_db.test_source_table_1 ( id UInt64, value String ) ENGINE=TinyLog; -INSERT INTO 01915_db.test_source_table_1 VALUES (0, 'Value0'); +INSERT INTO test_01915_db.test_source_table_1 VALUES (0, 'Value0'); -DROP DICTIONARY IF EXISTS 01915_db.test_dictionary; -CREATE OR REPLACE DICTIONARY 01915_db.test_dictionary +DROP DICTIONARY IF EXISTS test_01915_db.test_dictionary; +CREATE OR REPLACE DICTIONARY test_01915_db.test_dictionary ( id UInt64, value String ) PRIMARY KEY id LAYOUT(DIRECT()) -SOURCE(CLICKHOUSE(DB '01915_db' TABLE 'test_source_table_1')); +SOURCE(CLICKHOUSE(DB 'test_01915_db' TABLE 'test_source_table_1')); -SELECT * FROM 01915_db.test_dictionary; +SELECT * FROM test_01915_db.test_dictionary; -DROP TABLE IF EXISTS 01915_db.test_source_table_2; -CREATE TABLE 01915_db.test_source_table_2 +DROP TABLE IF EXISTS test_01915_db.test_source_table_2; +CREATE TABLE test_01915_db.test_source_table_2 ( id UInt64, value_1 String ) ENGINE=TinyLog; -INSERT INTO 01915_db.test_source_table_2 VALUES (0, 'Value1'); +INSERT INTO test_01915_db.test_source_table_2 VALUES (0, 'Value1'); -CREATE OR REPLACE DICTIONARY 01915_db.test_dictionary +CREATE OR REPLACE DICTIONARY test_01915_db.test_dictionary ( id UInt64, value_1 String ) PRIMARY KEY id LAYOUT(HASHED()) -SOURCE(CLICKHOUSE(DB '01915_db' TABLE 'test_source_table_2')) +SOURCE(CLICKHOUSE(DB 'test_01915_db' TABLE 'test_source_table_2')) LIFETIME(0); -SELECT * FROM 01915_db.test_dictionary; +SELECT * FROM test_01915_db.test_dictionary; -DROP DICTIONARY 01915_db.test_dictionary; +DROP DICTIONARY test_01915_db.test_dictionary; -DROP TABLE 01915_db.test_source_table_1; -DROP TABLE 01915_db.test_source_table_2; +DROP TABLE test_01915_db.test_source_table_1; +DROP TABLE test_01915_db.test_source_table_2; -DROP DATABASE 01915_db; +DROP DATABASE test_01915_db; diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index c9d517b0285..a56b42a4f75 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -110,6 +110,7 @@ "00738_lock_for_inner_table", "01153_attach_mv_uuid", "01157_replace_table", + "01185_create_or_replace_table", /// Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database. "rocksdb", "01914_exchange_dictionaries" /// Requires Atomic database @@ -519,7 +520,6 @@ "01924_argmax_bitmap_state", "01913_replace_dictionary", "01914_exchange_dictionaries", - "01915_create_or_replace_dictionary", "01913_names_of_tuple_literal", "01925_merge_prewhere_table" ], From c0bbe67cc9de0ad4477236d90e690130578d0a00 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 6 Jul 2021 13:26:03 +0300 Subject: [PATCH 007/147] fix --- src/Interpreters/DDLTask.cpp | 2 +- src/Interpreters/DDLTask.h | 6 +++++- src/Interpreters/InterpreterCreateQuery.cpp | 8 +++++++- 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index bf8380f5af6..1dc56d2e28a 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -362,7 +362,7 @@ ContextMutablePtr DatabaseReplicatedTask::makeQueryContext(ContextPtr from_conte query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; query_context->setCurrentDatabase(database->getDatabaseName()); - auto txn = std::make_shared(zookeeper, database->zookeeper_path, is_initial_query); + auto txn = std::make_shared(zookeeper, database->zookeeper_path, is_initial_query, entry_path); query_context->initZooKeeperMetadataTransaction(txn); if (is_initial_query) diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 00bb740f9c5..ac278e048b6 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -169,13 +169,15 @@ class ZooKeeperMetadataTransaction ZooKeeperPtr current_zookeeper; String zookeeper_path; bool is_initial_query; + String task_path; Coordination::Requests ops; public: - ZooKeeperMetadataTransaction(const ZooKeeperPtr & current_zookeeper_, const String & zookeeper_path_, bool is_initial_query_) + ZooKeeperMetadataTransaction(const ZooKeeperPtr & current_zookeeper_, const String & zookeeper_path_, bool is_initial_query_, const String & task_path_) : current_zookeeper(current_zookeeper_) , zookeeper_path(zookeeper_path_) , is_initial_query(is_initial_query_) + , task_path(task_path_) { } @@ -185,6 +187,8 @@ public: String getDatabaseZooKeeperPath() const { return zookeeper_path; } + String getTaskZooKeeperPath() const { return task_path; } + ZooKeeperPtr getZooKeeper() const { return current_zookeeper; } void addOp(Coordination::RequestPtr && op) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a4db21ec22f..7183306c9bb 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1098,7 +1098,8 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, /// Execute drop as separate query, because [CREATE OR] REPLACE query can be considered as /// successfully executed after RENAME/EXCHANGE query. drop_context->resetZooKeeperMetadataTransaction(); - auto drop_txn = std::make_shared(txn->getZooKeeper(), txn->getDatabaseZooKeeperPath(), txn->isInitialQuery()); + auto drop_txn = std::make_shared(txn->getZooKeeper(), txn->getDatabaseZooKeeperPath(), + txn->isInitialQuery(), txn->getTaskZooKeeperPath()); drop_context->initZooKeeperMetadataTransaction(drop_txn); } return drop_context; @@ -1117,6 +1118,11 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, UInt64 name_hash = sipHash64(create.database + create.table); UInt16 random_suffix = thread_local_rng(); + if (auto txn = current_context->getZooKeeperMetadataTransaction()) + { + /// Avoid different table name on database replicas + random_suffix = sipHash64(txn->getTaskZooKeeperPath()); + } create.table = fmt::format("_tmp_replace_{}_{}", getHexUIntLowercase(name_hash), getHexUIntLowercase(random_suffix)); From d6fe824340cb4ac7956f72899a0ef498be6c352a Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 21 Jul 2021 09:20:17 +0200 Subject: [PATCH 008/147] Add Map type support in mapPopulateSeries --- src/Functions/array/mapPopulateSeries.cpp | 351 ++++++++++++------ ...01925_map_populate_series_on_map.reference | 37 ++ .../01925_map_populate_series_on_map.sql | 35 ++ 3 files changed, 319 insertions(+), 104 deletions(-) create mode 100644 tests/queries/0_stateless/01925_map_populate_series_on_map.reference create mode 100644 tests/queries/0_stateless/01925_map_populate_series_on_map.sql diff --git a/src/Functions/array/mapPopulateSeries.cpp b/src/Functions/array/mapPopulateSeries.cpp index eb2f6192346..f8bcbf8d451 100644 --- a/src/Functions/array/mapPopulateSeries.cpp +++ b/src/Functions/array/mapPopulateSeries.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -7,6 +8,7 @@ #include #include #include "Core/ColumnWithTypeAndName.h" +#include "DataTypes/DataTypeMap.h" #include "DataTypes/IDataType.h" namespace DB @@ -32,31 +34,17 @@ private: bool isVariadic() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + void checkTypes(const DataTypePtr & key_type, const DataTypePtr max_key_type) const { - if (arguments.size() < 2) - throw Exception{getName() + " accepts at least two arrays for key and value", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - if (arguments.size() > 3) - throw Exception{"too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - const DataTypeArray * key_array_type = checkAndGetDataType(arguments[0].get()); - const DataTypeArray * val_array_type = checkAndGetDataType(arguments[1].get()); - - if (!key_array_type || !val_array_type) - throw Exception{getName() + " accepts two arrays for key and value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - DataTypePtr keys_type = key_array_type->getNestedType(); - WhichDataType which_key(keys_type); - if (!(which_key.isNativeInt() || which_key.isNativeUInt())) + WhichDataType which_key(key_type); + if (!(which_key.isInt() || which_key.isUInt())) { throw Exception( - "Keys for " + getName() + " should be of native integer type (signed or unsigned)", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + "Keys for " + getName() + " should be of integer type (signed or unsigned)", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } - if (arguments.size() == 3) + if (max_key_type) { - DataTypePtr max_key_type = arguments[2]; WhichDataType which_max_key(max_key_type); if (which_max_key.isNullable()) @@ -64,53 +52,186 @@ private: "Max key argument in arguments of function " + getName() + " can not be Nullable", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (keys_type->getTypeId() != max_key_type->getTypeId()) + if (key_type->getTypeId() != max_key_type->getTypeId()) throw Exception("Max key type in " + getName() + " should be same as keys type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } + } + + DataTypePtr getReturnTypeForTuple(const DataTypes & arguments) const + { + if (arguments.size() < 2) + throw Exception(getName() + " accepts at least two arrays for key and value", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (arguments.size() > 3) + throw Exception("too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const DataTypeArray * key_array_type = checkAndGetDataType(arguments[0].get()); + const DataTypeArray * val_array_type = checkAndGetDataType(arguments[1].get()); + + if (!key_array_type || !val_array_type) + throw Exception(getName() + " accepts two arrays for key and value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto & key_type = key_array_type->getNestedType(); + + if (arguments.size() == 3) + this->checkTypes(key_type, arguments[2]); + else + this->checkTypes(key_type, nullptr); return std::make_shared(DataTypes{arguments[0], arguments[1]}); } - template - ColumnPtr execute2(ColumnPtr key_column, ColumnPtr val_column, ColumnPtr max_key_column, const DataTypeTuple & res_type) const + DataTypePtr getReturnTypeForMap(const DataTypes & arguments) const { - MutableColumnPtr res_tuple = res_type.createColumn(); + const auto * map = assert_cast(arguments[0].get()); + if (arguments.size() == 1) + this->checkTypes(map->getKeyType(), nullptr); + else if (arguments.size() == 2) + this->checkTypes(map->getKeyType(), arguments[1]); + else + throw Exception("too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - auto * to_tuple = assert_cast(res_tuple.get()); - auto & to_keys_arr = assert_cast(to_tuple->getColumn(0)); - auto & to_keys_data = to_keys_arr.getData(); - auto & to_keys_offsets = to_keys_arr.getOffsets(); + return std::make_shared(map->getKeyType(), map->getValueType()); + } - auto & to_vals_arr = assert_cast(to_tuple->getColumn(1)); - auto & to_values_data = to_vals_arr.getData(); + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception(getName() + " accepts at least one map", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - bool max_key_is_const = false, key_is_const = false, val_is_const = false; + if (arguments[0]->getTypeId() == TypeIndex::Array) + return getReturnTypeForTuple(arguments); + else if (arguments[0]->getTypeId() == TypeIndex::Map) + return getReturnTypeForMap(arguments); + else + throw Exception(getName() + " only accepts maps", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } - const auto * keys_array = checkAndGetColumn(key_column.get()); - if (!keys_array) + // Struct holds input and output columns references, + // Both arrays and maps have similar columns to work with but extracted differently + template + struct ColumnsInOut + { + // inputs + const PaddedPODArray & in_keys_data; + const PaddedPODArray & in_vals_data; + const IColumn::Offsets & in_key_offsets; + const IColumn::Offsets & in_val_offsets; + size_t row_count; + bool key_is_const; + bool val_is_const; + + // outputs + PaddedPODArray & out_keys_data; + PaddedPODArray & out_vals_data; + + IColumn::Offsets & out_keys_offsets; + // with map argument this field will not be used + IColumn::Offsets * out_vals_offsets; + }; + + template + ColumnsInOut getInOutDataFromArrays(MutableColumnPtr & res_column, ColumnPtr * arg_columns) const + { + auto * out_tuple = assert_cast(res_column.get()); + auto & out_keys_array = assert_cast(out_tuple->getColumn(0)); + auto & out_vals_array = assert_cast(out_tuple->getColumn(1)); + + const auto * key_column = arg_columns[0].get(); + const auto * in_keys_array = checkAndGetColumn(key_column); + + bool key_is_const = false, val_is_const = false; + + if (!in_keys_array) { - const ColumnConst * const_array = checkAndGetColumnConst(key_column.get()); + const ColumnConst * const_array = checkAndGetColumnConst(key_column); if (!const_array) throw Exception("Expected array column, found " + key_column->getName(), ErrorCodes::ILLEGAL_COLUMN); - keys_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); + in_keys_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); key_is_const = true; } - const auto * values_array = checkAndGetColumn(val_column.get()); - if (!values_array) + const auto * val_column = arg_columns[1].get(); + const auto * in_values_array = checkAndGetColumn(val_column); + if (!in_values_array) { - const ColumnConst * const_array = checkAndGetColumnConst(val_column.get()); + const ColumnConst * const_array = checkAndGetColumnConst(val_column); if (!const_array) throw Exception("Expected array column, found " + val_column->getName(), ErrorCodes::ILLEGAL_COLUMN); - values_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); + in_values_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); val_is_const = true; } - if (!keys_array || !values_array) + if (!in_keys_array || !in_values_array) /* something went wrong */ - throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + + const auto & in_keys_data = assert_cast &>(in_keys_array->getData()).getData(); + const auto & in_values_data = assert_cast &>(in_values_array->getData()).getData(); + const auto & in_keys_offsets = in_keys_array->getOffsets(); + const auto & in_vals_offsets = in_values_array->getOffsets(); + + auto & out_keys_data = assert_cast &>(out_keys_array.getData()).getData(); + auto & out_vals_data = assert_cast &>(out_vals_array.getData()).getData(); + auto & out_keys_offsets = out_keys_array.getOffsets(); + + size_t row_count = key_is_const ? in_values_array->size() : in_keys_array->size(); + IColumn::Offsets * out_vals_offsets = &out_vals_array.getOffsets(); + + return { + in_keys_data, + in_values_data, + in_keys_offsets, + in_vals_offsets, + row_count, + key_is_const, + val_is_const, + out_keys_data, + out_vals_data, + out_keys_offsets, + out_vals_offsets}; + } + + template + ColumnsInOut getInOutDataFromMap(MutableColumnPtr & res_column, ColumnPtr * arg_columns) const + { + const auto * in_map = assert_cast(arg_columns[0].get()); + const auto & in_nested_array = in_map->getNestedColumn(); + const auto & in_nested_tuple = in_map->getNestedData(); + const auto & in_keys_data = assert_cast &>(in_nested_tuple.getColumn(0)).getData(); + const auto & in_vals_data = assert_cast &>(in_nested_tuple.getColumn(1)).getData(); + const auto & in_keys_offsets = in_nested_array.getOffsets(); + + auto * out_map = assert_cast(res_column.get()); + auto & out_nested_array = out_map->getNestedColumn(); + auto & out_nested_tuple = out_map->getNestedData(); + auto & out_keys_data = assert_cast &>(out_nested_tuple.getColumn(0)).getData(); + auto & out_vals_data = assert_cast &>(out_nested_tuple.getColumn(1)).getData(); + auto & out_keys_offsets = out_nested_array.getOffsets(); + + return { + in_keys_data, + in_vals_data, + in_keys_offsets, + in_keys_offsets, + in_nested_array.size(), + false, + false, + out_keys_data, + out_vals_data, + out_keys_offsets, + nullptr}; + } + + template + ColumnPtr execute2(ColumnPtr * arg_columns, ColumnPtr max_key_column, const DataTypePtr & res_type) const + { + MutableColumnPtr res_column = res_type->createColumn(); + bool max_key_is_const = false; + auto inout = res_column->getDataType() == TypeIndex::Tuple ? getInOutDataFromArrays(res_column, arg_columns) + : getInOutDataFromMap(res_column, arg_columns); KeyType max_key_const{0}; @@ -121,49 +242,40 @@ private: max_key_is_const = true; } - auto & keys_data = assert_cast &>(keys_array->getData()).getData(); - auto & values_data = assert_cast &>(values_array->getData()).getData(); - - // Original offsets - const IColumn::Offsets & key_offsets = keys_array->getOffsets(); - const IColumn::Offsets & val_offsets = values_array->getOffsets(); - IColumn::Offset offset{0}; - size_t row_count = key_is_const ? values_array->size() : keys_array->size(); - std::map res_map; //Iterate through two arrays and fill result values. - for (size_t row = 0; row < row_count; ++row) + for (size_t row = 0; row < inout.row_count; ++row) { - size_t key_offset = 0, val_offset = 0, array_size = key_offsets[0], val_array_size = val_offsets[0]; + size_t key_offset = 0, val_offset = 0, items_count = inout.in_key_offsets[0], val_array_size = inout.in_val_offsets[0]; res_map.clear(); - if (!key_is_const) + if (!inout.key_is_const) { - key_offset = row > 0 ? key_offsets[row - 1] : 0; - array_size = key_offsets[row] - key_offset; + key_offset = row > 0 ? inout.in_key_offsets[row - 1] : 0; + items_count = inout.in_key_offsets[row] - key_offset; } - if (!val_is_const) + if (!inout.val_is_const) { - val_offset = row > 0 ? val_offsets[row - 1] : 0; - val_array_size = val_offsets[row] - val_offset; + val_offset = row > 0 ? inout.in_val_offsets[row - 1] : 0; + val_array_size = inout.in_val_offsets[row] - val_offset; } - if (array_size != val_array_size) + if (items_count != val_array_size) throw Exception("Key and value array should have same amount of elements", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (array_size == 0) + if (items_count == 0) { - to_keys_offsets.push_back(offset); + inout.out_keys_offsets.push_back(offset); continue; } - for (size_t i = 0; i < array_size; ++i) + for (size_t i = 0; i < items_count; ++i) { - res_map.insert({keys_data[key_offset + i], values_data[val_offset + i]}); + res_map.insert({inout.in_keys_data[key_offset + i], inout.in_vals_data[val_offset + i]}); } auto min_key = res_map.begin()->first; @@ -184,7 +296,7 @@ private: /* no need to add anything, max key is less that first key */ if (max_key < min_key) { - to_keys_offsets.push_back(offset); + inout.out_keys_offsets.push_back(offset); continue; } } @@ -197,16 +309,16 @@ private: KeyType key; for (key = min_key;; ++key) { - to_keys_data.insert(key); + inout.out_keys_data.push_back(key); auto it = res_map.find(key); if (it != res_map.end()) { - to_values_data.insert(it->second); + inout.out_vals_data.push_back(it->second); } else { - to_values_data.insertDefault(); + inout.out_vals_data.push_back(0); } ++offset; @@ -214,80 +326,112 @@ private: break; } - to_keys_offsets.push_back(offset); + inout.out_keys_offsets.push_back(offset); } - to_vals_arr.getOffsets().insert(to_keys_offsets.begin(), to_keys_offsets.end()); - return res_tuple; + if (inout.out_vals_offsets) + inout.out_vals_offsets->insert(inout.out_keys_offsets.begin(), inout.out_keys_offsets.end()); + + return res_column; } template - ColumnPtr execute1(ColumnPtr key_column, ColumnPtr val_column, ColumnPtr max_key_column, const DataTypeTuple & res_type) const + ColumnPtr execute1(ColumnPtr * arg_columns, ColumnPtr max_key_column, const DataTypePtr & res_type, const DataTypePtr & val_type) const { - const auto & val_type = (assert_cast(res_type.getElements()[1].get()))->getNestedType(); switch (val_type->getTypeId()) { case TypeIndex::Int8: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::Int16: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::Int32: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::Int64: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); + case TypeIndex::Int128: + return execute2(arg_columns, max_key_column, res_type); + case TypeIndex::Int256: + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::UInt8: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::UInt16: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::UInt32: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::UInt64: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); + case TypeIndex::UInt128: + return execute2(arg_columns, max_key_column, res_type); + case TypeIndex::UInt256: + return execute2(arg_columns, max_key_column, res_type); default: - throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override { - auto col1 = arguments[0]; - auto col2 = arguments[1]; - - const auto * k = assert_cast(col1.type.get()); - const auto * v = assert_cast(col2.type.get()); - - /* determine output type */ - const DataTypeTuple & res_type = DataTypeTuple( - DataTypes{std::make_shared(k->getNestedType()), std::make_shared(v->getNestedType())}); - + DataTypePtr res_type, key_type, val_type; ColumnPtr max_key_column = nullptr; + ColumnPtr arg_columns[] = {arguments[0].column, nullptr}; - if (arguments.size() == 3) + if (arguments[0].type->getTypeId() == TypeIndex::Array) { - /* max key provided */ - max_key_column = arguments[2].column; + key_type = assert_cast(arguments[0].type.get())->getNestedType(); + val_type = assert_cast(arguments[1].type.get())->getNestedType(); + res_type = getReturnTypeImpl(DataTypes{arguments[0].type, arguments[1].type}); + + arg_columns[1] = arguments[1].column; + if (arguments.size() == 3) + { + /* max key provided */ + max_key_column = arguments[2].column; + } + } + else + { + assert(arguments[0].type->getTypeId() == TypeIndex::Map); + + const auto * map_type = assert_cast(arguments[0].type.get()); + res_type = getReturnTypeImpl(DataTypes{arguments[0].type}); + key_type = map_type->getKeyType(); + val_type = map_type->getValueType(); + + if (arguments.size() == 2) + { + /* max key provided */ + max_key_column = arguments[1].column; + } } - switch (k->getNestedType()->getTypeId()) + switch (key_type->getTypeId()) { case TypeIndex::Int8: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::Int16: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::Int32: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::Int64: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); + case TypeIndex::Int128: + return execute1(arg_columns, max_key_column, res_type, val_type); + case TypeIndex::Int256: + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::UInt8: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::UInt16: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::UInt32: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::UInt64: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); + case TypeIndex::UInt128: + return execute1(arg_columns, max_key_column, res_type, val_type); + case TypeIndex::UInt256: + return execute1(arg_columns, max_key_column, res_type, val_type); default: - throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } } }; @@ -296,5 +440,4 @@ void registerFunctionMapPopulateSeries(FunctionFactory & factory) { factory.registerFunction(); } - } diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference new file mode 100644 index 00000000000..74ed35f0638 --- /dev/null +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference @@ -0,0 +1,37 @@ +{1:1} +{1:1,2:2} +{1:1,2:0,3:2} +{1:1,2:0,3:0,4:2} +{1:1,2:0,3:0,4:0,5:2} +{1:1,2:0,3:0} +{1:1,2:2,3:0} +{1:1,2:0,3:2} +{1:1,2:0,3:0} +{1:1,2:0,3:0} +{1:1,2:0,3:0,4:0,5:0,6:0,7:0,8:0,9:0,10:0} +{1:1,2:2,3:0,4:0,5:0,6:0,7:0,8:0,9:0,10:0} +{1:1,2:0,3:2,4:0,5:0,6:0,7:0,8:0,9:0,10:0} +{1:1,2:0,3:0,4:2,5:0,6:0,7:0,8:0,9:0,10:0} +{1:1,2:0,3:0,4:0,5:2,6:0,7:0,8:0,9:0,10:0} +{1:1,2:0} +{1:1,2:2,3:0} +{1:1,2:0,3:2,4:0} +{1:1,2:0,3:0,4:2,5:0} +{1:1,2:0,3:0,4:0,5:2,6:0} +{1:1,2:1} Map(UInt8,UInt8) +{1:1,2:1} Map(UInt16,UInt16) +{1:1,2:1} Map(UInt32,UInt32) +{1:1,2:1} Map(UInt64,UInt64) +{1:1,2:1} Map(UInt128,UInt128) +{1:1,2:1} Map(UInt256,UInt256) +{1:1,2:1} Map(Int16,Int16) +{1:1,2:1} Map(Int16,Int16) +{1:1,2:1} Map(Int32,Int32) +{1:1,2:1} Map(Int64,Int64) +{1:1,2:1} Map(Int128,Int128) +{1:1,2:1} Map(Int256,Int256) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int32,Int32) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int64,Int64) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0} Map(Int64,Int64) diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.sql b/tests/queries/0_stateless/01925_map_populate_series_on_map.sql new file mode 100644 index 00000000000..f45543a8dba --- /dev/null +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.sql @@ -0,0 +1,35 @@ +drop table if exists map_test; +set allow_experimental_map_type = 1; +create table map_test engine=TinyLog() as (select (number + 1) as n, map(1, 1, number,2) as m from numbers(1, 5)); + +select mapPopulateSeries(m) from map_test; +select mapPopulateSeries(m, toUInt64(3)) from map_test; +select mapPopulateSeries(m, toUInt64(10)) from map_test; +select mapPopulateSeries(m, 1000) from map_test; -- { serverError 43 } +select mapPopulateSeries(m, n) from map_test; + +drop table map_test; + +select mapPopulateSeries(map(toUInt8(1), toUInt8(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toUInt16(1), toUInt16(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toUInt32(1), toUInt32(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toUInt64(1), toUInt64(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toUInt128(1), toUInt128(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toUInt256(1), toUInt256(1), 2, 1)) as res, toTypeName(res); + +select mapPopulateSeries(map(toInt8(1), toInt8(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt16(1), toInt16(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt32(1), toInt32(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt64(1), toInt64(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt128(1), toInt128(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt256(1), toInt256(1), 2, 1)) as res, toTypeName(res); + +select mapPopulateSeries(map(toInt8(-10), toInt8(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt16(-10), toInt16(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt32(-10), toInt32(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1), toInt64(-5)) as res, toTypeName(res); + +select mapPopulateSeries(); -- { serverError 42 } +select mapPopulateSeries('asdf'); -- { serverError 43 } +select mapPopulateSeries(map('1', 1, '2', 1)) as res, toTypeName(res); -- { serverError 43 } From 0f480e6b06b2d9ddfeb8a900fb0f6fbf8782757c Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 21 Jul 2021 10:08:29 +0200 Subject: [PATCH 009/147] Fix docs for map functions --- .../functions/tuple-map-functions.md | 77 +++++++++++++++---- 1 file changed, 61 insertions(+), 16 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index dcfa18e04bf..908d3e1bbd3 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -103,81 +103,126 @@ Result: Query with `Map` type: ``` sql +SELECT mapAdd(map(1,1), map(1,1)); +``` + +Result: + +``` text +┌─mapAdd(map(1, 1), map(1, 1))─┐ +│ {1:2} │ +└──────────────────────────────┘ ``` ## mapSubtract {#function-mapsubtract} Collect all the keys and subtract corresponding values. -**Syntax** +**Syntax** ``` sql mapSubtract(Tuple(Array, Array), Tuple(Array, Array) [, ...]) ``` -**Arguments** +**Arguments** -Arguments are [tuples](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array), where items in the first array represent keys, and the second array contains values for the each key. All key arrays should have same type, and all value arrays should contain items which are promote to the one type ([Int64](../../sql-reference/data-types/int-uint.md#int-ranges), [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges) or [Float64](../../sql-reference/data-types/float.md#float32-float64)). The common promoted type is used as a type for the result array. +Arguments are [maps](../../sql-reference/data-types/map.md) or [tuples](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array), where items in the first array represent keys, and the second array contains values for the each key. All key arrays should have same type, and all value arrays should contain items which are promote to the one type ([Int64](../../sql-reference/data-types/int-uint.md#int-ranges), [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges) or [Float64](../../sql-reference/data-types/float.md#float32-float64)). The common promoted type is used as a type for the result array. **Returned value** -- Returns one [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2), where the first array contains the sorted keys and the second array contains values. +- Depending on the arguments returns one [map](../../sql-reference/data-types/map.md) or [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2), where the first array contains the sorted keys and the second array contains values. **Example** -Query: +Query with a tuple map: -```sql +``` sql SELECT mapSubtract(([toUInt8(1), 2], [toInt32(1), 1]), ([toUInt8(1), 2], [toInt32(2), 1])) as res, toTypeName(res) as type; ``` Result: -```text +``` text ┌─res────────────┬─type──────────────────────────────┐ │ ([1,2],[-1,0]) │ Tuple(Array(UInt8), Array(Int64)) │ └────────────────┴───────────────────────────────────┘ ``` +Query with `Map` type: + +``` sql +SELECT mapSubtract(map(1,1), map(1,1)); +``` + +Result: + +``` text +┌─mapSubtract(map(1, 1), map(1, 1))─┐ +│ {1:0} │ +└───────────────────────────────────┘ +``` + ## mapPopulateSeries {#function-mappopulateseries} Fills missing keys in the maps (key and value array pair), where keys are integers. Also, it supports specifying the max key, which is used to extend the keys array. +Arguments are [maps](../../sql-reference/data-types/map.md) or two [arrays](../../sql-reference/data-types/array.md#data-type-array), where the first array represent keys, and the second array contains values for the each key. -**Syntax** +For array arguments the number of elements in `keys` and `values` must be the same for each row. + +**Syntax** ``` sql mapPopulateSeries(keys, values[, max]) +mapPopulateSeries(map[, max]) ``` -Generates a map, where keys are a series of numbers, from minimum to maximum keys (or `max` argument if it specified) taken from `keys` array with a step size of one, and corresponding values taken from `values` array. If the value is not specified for the key, then it uses the default value in the resulting map. For repeated keys, only the first value (in order of appearing) gets associated with the key. - -The number of elements in `keys` and `values` must be the same for each row. +Generates a map (a tuple with two arrays or a value of `Map` type, depending on the arguments), where keys are a series of numbers, from minimum to maximum keys (or `max` argument if it specified) taken from the map with a step size of one, and corresponding values. If the value is not specified for the key, then it uses the default value in the resulting map. For repeated keys, only the first value (in order of appearing) gets associated with the key. **Arguments** +Mapped arrays: + - `keys` — Array of keys. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#uint-ranges)). - `values` — Array of values. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#uint-ranges)). +or + +- `map` — Map with integer keys. [Map](../../sql-reference/data-types/map.md). + **Returned value** -- Returns a [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array): keys in sorted order, and values the corresponding keys. +- Depending on the arguments returns a [map](../../sql-reference/data-types/map.md) or a [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array): keys in sorted order, and values the corresponding keys. **Example** -Query: +Query with mapped arrays: -```sql +``` sql select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type; ``` Result: -```text +``` text ┌─res──────────────────────────┬─type──────────────────────────────┐ │ ([1,2,3,4,5],[11,22,0,44,0]) │ Tuple(Array(UInt8), Array(UInt8)) │ └──────────────────────────────┴───────────────────────────────────┘ ``` +Query with `Map` type: + +``` sql +SELECT mapPopulateSeries(map(1, 10, 5, 20), 6); +``` + +Result: + +``` text +┌─mapPopulateSeries(map(1, 10, 5, 20), 6)─┐ +│ {1:10,2:0,3:0,4:0,5:20,6:0} │ +└─────────────────────────────────────────┘ +``` + ## mapContains {#mapcontains} Determines whether the `map` contains the `key` parameter. @@ -188,7 +233,7 @@ Determines whether the `map` contains the `key` parameter. mapContains(map, key) ``` -**Parameters** +**Parameters** - `map` — Map. [Map](../../sql-reference/data-types/map.md). - `key` — Key. Type matches the type of keys of `map` parameter. From 2df3e14559fa096d0b395dd8b29936fc1cc3a38e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jul 2021 02:56:17 +0300 Subject: [PATCH 010/147] Attempt to add memory tracking to RoaringBitmaps --- base/common/memory.h | 57 -------- contrib/croaring-cmake/CMakeLists.txt | 2 + .../AggregateFunctionGroupBitmapData.h | 9 +- src/CMakeLists.txt | 2 + src/Common/CurrentMemoryTracker.cpp | 9 ++ src/Common/CurrentMemoryTracker.h | 1 + src/Common/memory.h | 131 ++++++++++++++++++ src/Common/new_delete.cpp | 85 +----------- 8 files changed, 152 insertions(+), 144 deletions(-) delete mode 100644 base/common/memory.h create mode 100644 src/Common/memory.h diff --git a/base/common/memory.h b/base/common/memory.h deleted file mode 100644 index e82c019ceab..00000000000 --- a/base/common/memory.h +++ /dev/null @@ -1,57 +0,0 @@ -#pragma once - -#include -#include "defines.h" - -#if USE_JEMALLOC -# include -#endif - -#if !USE_JEMALLOC || JEMALLOC_VERSION_MAJOR < 4 -# include -#endif - - -namespace Memory -{ - -inline ALWAYS_INLINE void * newImpl(std::size_t size) -{ - auto * ptr = malloc(size); - if (likely(ptr != nullptr)) - return ptr; - - /// @note no std::get_new_handler logic implemented - throw std::bad_alloc{}; -} - -inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept -{ - return malloc(size); -} - -inline ALWAYS_INLINE void deleteImpl(void * ptr) noexcept -{ - free(ptr); -} - -#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 4 - -inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size) noexcept -{ - if (unlikely(ptr == nullptr)) - return; - - sdallocx(ptr, size, 0); -} - -#else - -inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unused]]) noexcept -{ - free(ptr); -} - -#endif - -} diff --git a/contrib/croaring-cmake/CMakeLists.txt b/contrib/croaring-cmake/CMakeLists.txt index f4a5d8a01dc..522540e5e40 100644 --- a/contrib/croaring-cmake/CMakeLists.txt +++ b/contrib/croaring-cmake/CMakeLists.txt @@ -24,3 +24,5 @@ add_library(roaring ${SRCS}) target_include_directories(roaring PRIVATE "${LIBRARY_DIR}/include/roaring") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/include") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/cpp") + +target_compile_definitions(roaring PRIVATE -Dmalloc=clickhouse_malloc -Dfree=clickhouse_free -Dposix_memalign=clickhouse_posix_memalign) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 067daf6dc3a..d5484c4db79 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -7,6 +7,7 @@ #include #include #include +#include // Include this header last, because it is an auto-generated dump of questionable // garbage that breaks the build (e.g. it changes _POSIX_C_SOURCE). @@ -44,7 +45,7 @@ private: void toLarge() { - rb = std::make_unique(); + rb = std::make_shared(); for (const auto & x : small) rb->add(static_cast(x.getValue())); small.clear(); @@ -67,12 +68,14 @@ public: { toLarge(); rb->add(static_cast(value)); + CurrentMemoryTracker::check(); } } } else { rb->add(static_cast(value)); + CurrentMemoryTracker::check(); } } @@ -114,7 +117,7 @@ public: readVarUInt(size, in); std::unique_ptr buf(new char[size]); in.readStrict(buf.get(), size); - rb = std::make_unique(RoaringBitmap::read(buf.get())); + rb = std::make_shared(RoaringBitmap::read(buf.get())); } } @@ -141,7 +144,7 @@ public: */ std::shared_ptr getNewRoaringBitmapFromSmall() const { - std::shared_ptr ret = std::make_unique(); + std::shared_ptr ret = std::make_shared(); for (const auto & x : small) ret->add(static_cast(x.getValue())); return ret; diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 31286c740d4..9c1f884edc1 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -158,6 +158,8 @@ else() target_link_libraries (clickhouse_new_delete PRIVATE clickhouse_common_io jemalloc) endif() +target_link_libraries (clickhouse_common_io PRIVATE jemalloc) + add_subdirectory(Common/ZooKeeper) add_subdirectory(Common/Config) diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index d38a5a9c70c..1156c6e56bc 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -2,6 +2,8 @@ #include #include +#include + namespace { @@ -36,6 +38,7 @@ namespace if (current_thread) { current_thread->untracked_memory += size; + if (current_thread->untracked_memory > current_thread->untracked_memory_limit) { /// Zero untracked before track. If tracker throws out-of-limit we would be able to alloc up to untracked_memory_limit bytes @@ -54,6 +57,12 @@ namespace } } +void check() +{ + if (auto * memory_tracker = getMemoryTracker()) + memory_tracker->allocImpl(0, true); +} + void alloc(Int64 size) { bool throw_if_memory_exceeded = true; diff --git a/src/Common/CurrentMemoryTracker.h b/src/Common/CurrentMemoryTracker.h index 5090b7c3687..2f9ace4291f 100644 --- a/src/Common/CurrentMemoryTracker.h +++ b/src/Common/CurrentMemoryTracker.h @@ -9,4 +9,5 @@ namespace CurrentMemoryTracker void allocNoThrow(Int64 size); void realloc(Int64 old_size, Int64 new_size); void free(Int64 size); + void check(); } diff --git a/src/Common/memory.h b/src/Common/memory.h new file mode 100644 index 00000000000..0aa99cdf70d --- /dev/null +++ b/src/Common/memory.h @@ -0,0 +1,131 @@ +#pragma once + +#include +#include + +#include + +#if USE_JEMALLOC +# include +#endif + +#if !USE_JEMALLOC || JEMALLOC_VERSION_MAJOR < 4 +# include +#endif + + +namespace Memory +{ + +inline ALWAYS_INLINE void * newImpl(std::size_t size) +{ + auto * ptr = malloc(size); + if (likely(ptr != nullptr)) + return ptr; + + /// @note no std::get_new_handler logic implemented + throw std::bad_alloc{}; +} + +inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept +{ + return malloc(size); +} + +inline ALWAYS_INLINE void deleteImpl(void * ptr) noexcept +{ + free(ptr); +} + +#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 4 + +inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size) noexcept +{ + if (unlikely(ptr == nullptr)) + return; + + sdallocx(ptr, size, 0); +} + +#else + +inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unused]]) noexcept +{ + free(ptr); +} + +#endif + +#if defined(OS_LINUX) +# include +#elif defined(OS_DARWIN) +# include +#endif + +#if defined(OS_DARWIN) && defined(BUNDLED_STATIC_JEMALLOC) +extern "C" +{ +extern void zone_register(); +} + +struct InitializeJemallocZoneAllocatorForOSX +{ + InitializeJemallocZoneAllocatorForOSX() + { + /// In case of OSX jemalloc register itself as a default zone allocator. + /// + /// But when you link statically then zone_register() will not be called, + /// and even will be optimized out: + /// + /// It is ok to call it twice (i.e. in case of shared libraries) + /// Since zone_register() is a no-op if the default zone is already replaced with something. + /// + /// https://github.com/jemalloc/jemalloc/issues/708 + zone_register(); + } +} initializeJemallocZoneAllocatorForOSX; +#endif + +inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size) +{ + size_t actual_size = size; + +#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5 + /// The nallocx() function allocates no memory, but it performs the same size computation as the mallocx() function + /// @note je_mallocx() != je_malloc(). It's expected they don't differ much in allocation logic. + if (likely(size != 0)) + actual_size = nallocx(size, 0); +#endif + + return actual_size; +} + +inline ALWAYS_INLINE void trackMemory(std::size_t size) +{ + std::size_t actual_size = getActualAllocationSize(size); + CurrentMemoryTracker::allocNoThrow(actual_size); +} + +inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0) noexcept +{ + try + { +#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5 + /// @note It's also possible to use je_malloc_usable_size() here. + if (likely(ptr != nullptr)) + CurrentMemoryTracker::free(sallocx(ptr, 0)); +#else + if (size) + CurrentMemoryTracker::free(size); +# if defined(_GNU_SOURCE) + /// It's innaccurate resource free for sanitizers. malloc_usable_size() result is greater or equal to allocated size. + else + CurrentMemoryTracker::free(malloc_usable_size(ptr)); +# endif +#endif + } + catch (...) + {} +} + +} diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp index 56173fb108a..fa32d56b350 100644 --- a/src/Common/new_delete.cpp +++ b/src/Common/new_delete.cpp @@ -1,117 +1,34 @@ -#include -#include - -#include +#include #include -#if defined(OS_LINUX) -# include -#elif defined(OS_DARWIN) -# include -#endif - -#if defined(OS_DARWIN) && defined(BUNDLED_STATIC_JEMALLOC) -extern "C" -{ -extern void zone_register(); -} - -struct InitializeJemallocZoneAllocatorForOSX -{ - InitializeJemallocZoneAllocatorForOSX() - { - /// In case of OSX jemalloc register itself as a default zone allocator. - /// - /// But when you link statically then zone_register() will not be called, - /// and even will be optimized out: - /// - /// It is ok to call it twice (i.e. in case of shared libraries) - /// Since zone_register() is a no-op if the default zone is already replaced with something. - /// - /// https://github.com/jemalloc/jemalloc/issues/708 - zone_register(); - } -} initializeJemallocZoneAllocatorForOSX; -#endif /// Replace default new/delete with memory tracking versions. /// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new /// https://en.cppreference.com/w/cpp/memory/new/operator_delete -namespace Memory -{ - -inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size) -{ - size_t actual_size = size; - -#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5 - /// The nallocx() function allocates no memory, but it performs the same size computation as the mallocx() function - /// @note je_mallocx() != je_malloc(). It's expected they don't differ much in allocation logic. - if (likely(size != 0)) - actual_size = nallocx(size, 0); -#endif - - return actual_size; -} - -inline ALWAYS_INLINE void trackMemory(std::size_t size) -{ - std::size_t actual_size = getActualAllocationSize(size); - CurrentMemoryTracker::allocNoThrow(actual_size); -} - -inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0) noexcept -{ - try - { -#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5 - /// @note It's also possible to use je_malloc_usable_size() here. - if (likely(ptr != nullptr)) - CurrentMemoryTracker::free(sallocx(ptr, 0)); -#else - if (size) - CurrentMemoryTracker::free(size); -# if defined(_GNU_SOURCE) - /// It's innaccurate resource free for sanitizers. malloc_usable_size() result is greater or equal to allocated size. - else - CurrentMemoryTracker::free(malloc_usable_size(ptr)); -# endif -#endif - } - catch (...) - {} -} - -} - /// new void * operator new(std::size_t size) { Memory::trackMemory(size); - return Memory::newImpl(size); } void * operator new[](std::size_t size) { Memory::trackMemory(size); - return Memory::newImpl(size); } void * operator new(std::size_t size, const std::nothrow_t &) noexcept { Memory::trackMemory(size); - return Memory::newNoExept(size); } void * operator new[](std::size_t size, const std::nothrow_t &) noexcept { Memory::trackMemory(size); - return Memory::newNoExept(size); } From b428d282e0fd58a1821c3027b3515a788b2410db Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 26 Jul 2021 11:38:30 +0300 Subject: [PATCH 011/147] Update arcadia_skip_list.txt --- tests/queries/0_stateless/arcadia_skip_list.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 903c72f044a..eab220fa213 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -93,6 +93,8 @@ 01138_join_on_distributed_and_tmp 01153_attach_mv_uuid 01155_rename_move_materialized_view +01157_replace_table +01185_create_or_replace_table 01191_rename_dictionary 01200_mutations_memory_consumption 01211_optimize_skip_unused_shards_type_mismatch From 1feee18455ebd16e73efc01a25035bd39f00adff Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Jul 2021 21:24:32 +0300 Subject: [PATCH 012/147] Remove test_keeper_server usage (for {operation/session}_timeout_ms) --- src/Server/KeeperTCPHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 2d5f41fe666..df40a78749b 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -195,8 +195,8 @@ KeeperTCPHandler::KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSoc , log(&Poco::Logger::get("NuKeeperTCPHandler")) , global_context(Context::createCopy(server.context())) , keeper_dispatcher(global_context->getKeeperStorageDispatcher()) - , operation_timeout(0, global_context->getConfigRef().getUInt("test_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000) - , session_timeout(0, global_context->getConfigRef().getUInt("test_keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) + , operation_timeout(0, global_context->getConfigRef().getUInt("keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000) + , session_timeout(0, global_context->getConfigRef().getUInt("keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) , poll_wrapper(std::make_unique(socket_)) , responses(std::make_unique()) { From a4e9a56e8e2bc786edd78158650ceb36f63155bb Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Fri, 30 Jul 2021 20:57:32 +0200 Subject: [PATCH 013/147] Make changes by review on #26663 PR --- src/Functions/array/mapPopulateSeries.cpp | 80 +++++++++++-------- ...01925_map_populate_series_on_map.reference | 30 +++++++ .../01925_map_populate_series_on_map.sql | 1 + .../tests/array_tuple_map.py | 22 +++-- 4 files changed, 91 insertions(+), 42 deletions(-) diff --git a/src/Functions/array/mapPopulateSeries.cpp b/src/Functions/array/mapPopulateSeries.cpp index f8bcbf8d451..51e436e8022 100644 --- a/src/Functions/array/mapPopulateSeries.cpp +++ b/src/Functions/array/mapPopulateSeries.cpp @@ -40,7 +40,7 @@ private: if (!(which_key.isInt() || which_key.isUInt())) { throw Exception( - "Keys for " + getName() + " should be of integer type (signed or unsigned)", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Keys for {} function should be of integer type (signed or unsigned)", getName()); } if (max_key_type) @@ -49,27 +49,28 @@ private: if (which_max_key.isNullable()) throw Exception( - "Max key argument in arguments of function " + getName() + " can not be Nullable", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Max key argument in arguments of function " + getName() + " can not be Nullable"); if (key_type->getTypeId() != max_key_type->getTypeId()) - throw Exception("Max key type in " + getName() + " should be same as keys type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Max key type in {} should be same as keys type", getName()); } } DataTypePtr getReturnTypeForTuple(const DataTypes & arguments) const { if (arguments.size() < 2) - throw Exception(getName() + " accepts at least two arrays for key and value", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} accepts at least two arrays for key and value", getName()); if (arguments.size() > 3) - throw Exception("too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in {} call", getName()); const DataTypeArray * key_array_type = checkAndGetDataType(arguments[0].get()); const DataTypeArray * val_array_type = checkAndGetDataType(arguments[1].get()); if (!key_array_type || !val_array_type) - throw Exception(getName() + " accepts two arrays for key and value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} accepts two arrays for key and value", getName()); const auto & key_type = key_array_type->getNestedType(); @@ -89,7 +90,7 @@ private: else if (arguments.size() == 2) this->checkTypes(map->getKeyType(), arguments[1]); else - throw Exception("too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in {} call", getName()); return std::make_shared(map->getKeyType(), map->getValueType()); } @@ -97,14 +98,18 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.empty()) - throw Exception(getName() + " accepts at least one map", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, getName() + " accepts at least one map or two arrays"); if (arguments[0]->getTypeId() == TypeIndex::Array) return getReturnTypeForTuple(arguments); else if (arguments[0]->getTypeId() == TypeIndex::Map) return getReturnTypeForMap(arguments); else - throw Exception(getName() + " only accepts maps", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Function {} only accepts one map or arrays, but got {}", + getName(), + arguments[0]->getName()); } // Struct holds input and output columns references, @@ -146,7 +151,8 @@ private: { const ColumnConst * const_array = checkAndGetColumnConst(key_column); if (!const_array) - throw Exception("Expected array column, found " + key_column->getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Expected array column in function {}, found {}", getName(), key_column->getName()); in_keys_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); key_is_const = true; @@ -158,7 +164,8 @@ private: { const ColumnConst * const_array = checkAndGetColumnConst(val_column); if (!const_array) - throw Exception("Expected array column, found " + val_column->getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Expected array column in function {}, found {}", getName(), val_column->getName()); in_values_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); val_is_const = true; @@ -166,7 +173,7 @@ private: if (!in_keys_array || !in_values_array) /* something went wrong */ - throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName()); const auto & in_keys_data = assert_cast &>(in_keys_array->getData()).getData(); const auto & in_values_data = assert_cast &>(in_values_array->getData()).getData(); @@ -230,8 +237,8 @@ private: { MutableColumnPtr res_column = res_type->createColumn(); bool max_key_is_const = false; - auto inout = res_column->getDataType() == TypeIndex::Tuple ? getInOutDataFromArrays(res_column, arg_columns) - : getInOutDataFromMap(res_column, arg_columns); + auto columns = res_column->getDataType() == TypeIndex::Tuple ? getInOutDataFromArrays(res_column, arg_columns) + : getInOutDataFromMap(res_column, arg_columns); KeyType max_key_const{0}; @@ -246,36 +253,39 @@ private: std::map res_map; //Iterate through two arrays and fill result values. - for (size_t row = 0; row < inout.row_count; ++row) + for (size_t row = 0; row < columns.row_count; ++row) { - size_t key_offset = 0, val_offset = 0, items_count = inout.in_key_offsets[0], val_array_size = inout.in_val_offsets[0]; + size_t key_offset = 0, val_offset = 0, items_count = columns.in_key_offsets[0], val_array_size = columns.in_val_offsets[0]; res_map.clear(); - if (!inout.key_is_const) + if (!columns.key_is_const) { - key_offset = row > 0 ? inout.in_key_offsets[row - 1] : 0; - items_count = inout.in_key_offsets[row] - key_offset; + key_offset = row > 0 ? columns.in_key_offsets[row - 1] : 0; + items_count = columns.in_key_offsets[row] - key_offset; } - if (!inout.val_is_const) + if (!columns.val_is_const) { - val_offset = row > 0 ? inout.in_val_offsets[row - 1] : 0; - val_array_size = inout.in_val_offsets[row] - val_offset; + val_offset = row > 0 ? columns.in_val_offsets[row - 1] : 0; + val_array_size = columns.in_val_offsets[row] - val_offset; } if (items_count != val_array_size) - throw Exception("Key and value array should have same amount of elements", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Key and value array should have same amount of elements in function {}", + getName()); if (items_count == 0) { - inout.out_keys_offsets.push_back(offset); + columns.out_keys_offsets.push_back(offset); continue; } for (size_t i = 0; i < items_count; ++i) { - res_map.insert({inout.in_keys_data[key_offset + i], inout.in_vals_data[val_offset + i]}); + res_map.insert({columns.in_keys_data[key_offset + i], columns.in_vals_data[val_offset + i]}); } auto min_key = res_map.begin()->first; @@ -296,7 +306,7 @@ private: /* no need to add anything, max key is less that first key */ if (max_key < min_key) { - inout.out_keys_offsets.push_back(offset); + columns.out_keys_offsets.push_back(offset); continue; } } @@ -309,16 +319,16 @@ private: KeyType key; for (key = min_key;; ++key) { - inout.out_keys_data.push_back(key); + columns.out_keys_data.push_back(key); auto it = res_map.find(key); if (it != res_map.end()) { - inout.out_vals_data.push_back(it->second); + columns.out_vals_data.push_back(it->second); } else { - inout.out_vals_data.push_back(0); + columns.out_vals_data.push_back(0); } ++offset; @@ -326,11 +336,11 @@ private: break; } - inout.out_keys_offsets.push_back(offset); + columns.out_keys_offsets.push_back(offset); } - if (inout.out_vals_offsets) - inout.out_vals_offsets->insert(inout.out_keys_offsets.begin(), inout.out_keys_offsets.end()); + if (columns.out_vals_offsets) + columns.out_vals_offsets->insert(columns.out_keys_offsets.begin(), columns.out_keys_offsets.end()); return res_column; } @@ -365,7 +375,7 @@ private: case TypeIndex::UInt256: return execute2(arg_columns, max_key_column, res_type); default: - throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName()); } } @@ -431,7 +441,7 @@ private: case TypeIndex::UInt256: return execute1(arg_columns, max_key_column, res_type, val_type); default: - throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName()); } } }; diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference index 74ed35f0638..235a227f548 100644 --- a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference @@ -1,37 +1,67 @@ +-- { echo } +drop table if exists map_test; +set allow_experimental_map_type = 1; +create table map_test engine=TinyLog() as (select (number + 1) as n, map(1, 1, number,2) as m from numbers(1, 5)); +select mapPopulateSeries(m) from map_test; {1:1} {1:1,2:2} {1:1,2:0,3:2} {1:1,2:0,3:0,4:2} {1:1,2:0,3:0,4:0,5:2} +select mapPopulateSeries(m, toUInt64(3)) from map_test; {1:1,2:0,3:0} {1:1,2:2,3:0} {1:1,2:0,3:2} {1:1,2:0,3:0} {1:1,2:0,3:0} +select mapPopulateSeries(m, toUInt64(10)) from map_test; {1:1,2:0,3:0,4:0,5:0,6:0,7:0,8:0,9:0,10:0} {1:1,2:2,3:0,4:0,5:0,6:0,7:0,8:0,9:0,10:0} {1:1,2:0,3:2,4:0,5:0,6:0,7:0,8:0,9:0,10:0} {1:1,2:0,3:0,4:2,5:0,6:0,7:0,8:0,9:0,10:0} {1:1,2:0,3:0,4:0,5:2,6:0,7:0,8:0,9:0,10:0} +select mapPopulateSeries(m, 1000) from map_test; -- { serverError 43 } +select mapPopulateSeries(m, n) from map_test; {1:1,2:0} {1:1,2:2,3:0} {1:1,2:0,3:2,4:0} {1:1,2:0,3:0,4:2,5:0} {1:1,2:0,3:0,4:0,5:2,6:0} +drop table map_test; +select mapPopulateSeries(map(toUInt8(1), toUInt8(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt8,UInt8) +select mapPopulateSeries(map(toUInt16(1), toUInt16(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt16,UInt16) +select mapPopulateSeries(map(toUInt32(1), toUInt32(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt32,UInt32) +select mapPopulateSeries(map(toUInt64(1), toUInt64(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt64,UInt64) +select mapPopulateSeries(map(toUInt128(1), toUInt128(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt128,UInt128) +select mapPopulateSeries(map(toUInt256(1), toUInt256(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt256,UInt256) +select mapPopulateSeries(map(toInt8(1), toInt8(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int16,Int16) +select mapPopulateSeries(map(toInt16(1), toInt16(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int16,Int16) +select mapPopulateSeries(map(toInt32(1), toInt32(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int32,Int32) +select mapPopulateSeries(map(toInt64(1), toInt64(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int64,Int64) +select mapPopulateSeries(map(toInt128(1), toInt128(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int128,Int128) +select mapPopulateSeries(map(toInt256(1), toInt256(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int256,Int256) +select mapPopulateSeries(map(toInt8(-10), toInt8(1), 2, 1)) as res, toTypeName(res); {-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +select mapPopulateSeries(map(toInt16(-10), toInt16(1), 2, 1)) as res, toTypeName(res); {-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +select mapPopulateSeries(map(toInt32(-10), toInt32(1), 2, 1)) as res, toTypeName(res); {-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int32,Int32) +select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1)) as res, toTypeName(res); {-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int64,Int64) +select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1), toInt64(-5)) as res, toTypeName(res); {-10:1,-9:0,-8:0,-7:0,-6:0,-5:0} Map(Int64,Int64) +select mapPopulateSeries(); -- { serverError 42 } +select mapPopulateSeries('asdf'); -- { serverError 43 } +select mapPopulateSeries(map('1', 1, '2', 1)) as res, toTypeName(res); -- { serverError 43 } diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.sql b/tests/queries/0_stateless/01925_map_populate_series_on_map.sql index f45543a8dba..ac78280ec1d 100644 --- a/tests/queries/0_stateless/01925_map_populate_series_on_map.sql +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.sql @@ -1,3 +1,4 @@ +-- { echo } drop table if exists map_test; set allow_experimental_map_type = 1; create table map_test engine=TinyLog() as (select (number + 1) as n, map(1, 1, number,2) as m from numbers(1, 5)); diff --git a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py index 938beabfff4..04150d945f6 100644 --- a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py +++ b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py @@ -357,7 +357,7 @@ def map_func(self, data_type, node=None): exitcode, message = 0, None if data_type.startswith("Decimal"): - exitcode, message = 43, "Exception:" + exitcode, message = 43, "Exception:" node.query(sql, exitcode=exitcode, message=message) execute_query(f"""SELECT * FROM {table_name} ORDER BY a ASC""") @@ -393,9 +393,13 @@ def map_func(self, data_type, node=None): execute_query(f"SELECT * FROM {table_name} ORDER BY a ASC") with Scenario(f"mapPopulateSeries with {data_type}"): - node.query(f"SELECT mapPopulateSeries([1,2,3], [{to_data_type(data_type,1)}," - f"{to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)", - exitcode = 44, message='Exception:') + sql = (f"SELECT mapPopulateSeries([1,2,3], [{to_data_type(data_type,1)}," + f"{to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)") + + exitcode, message = 0, None + if data_type.startswith("Decimal"): + exitcode, message = 44, "Exception:" + node.query(sql, exitcode=exitcode, message=message) with Scenario(f"mapPopulateSeries with {data_type} on a table"): table_name = get_table_name() @@ -403,9 +407,13 @@ def map_func(self, data_type, node=None): table(name = table_name, data_type = f'Tuple(Array({data_type}), Array({data_type}))') with When("I insert the output into a table"): - node.query(f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3]," - f"[{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)", - exitcode = 44, message='Exception:') + sql = (f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3],", + f"[{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)") + + exitcode, message = 0, None + if data_type.startswith("Decimal"): + exitcode, message = 44, "Exception:" + node.query(sql, exitcode=exitcode, message=message) execute_query(f"SELECT * FROM {table_name} ORDER BY a ASC") From df0d9be04b4c32e4a21744c8f667e53f44b7e6fa Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Mon, 2 Aug 2021 12:59:13 +0300 Subject: [PATCH 014/147] Update docs/en/sql-reference/functions/tuple-map-functions.md --- docs/en/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 908d3e1bbd3..34935a10d8c 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -197,7 +197,7 @@ or Query with mapped arrays: -``` sql +```sql select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type; ``` From 53c3a4fbc6fb0129b4e225b035e0c79ac8365014 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 2 Aug 2021 14:57:51 +0300 Subject: [PATCH 015/147] Fix excessive logging in NuRaft on server shutdown --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 0ce94900930..b721083987c 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 0ce9490093021c63564cca159571a8b27772ad48 +Subproject commit b721083987c3ef306cf311fd59f0722c20c08d7d From f3e0f648d09e77f01b3ee8561f7ed808e722c3fa Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 2 Aug 2021 15:00:05 +0300 Subject: [PATCH 016/147] Fixup --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index b721083987c..7ecb16844af 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit b721083987c3ef306cf311fd59f0722c20c08d7d +Subproject commit 7ecb16844af6a9c283ad432d85ecc2e7d1544676 From 740c16939658e0752dfff6b15e8d43758b77bee4 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Mon, 2 Aug 2021 16:08:27 +0300 Subject: [PATCH 017/147] Fix test_merge_tree_s3_failover with debug build --- .../test_merge_tree_s3_failover/test.py | 27 +++++++++++++------ 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 4dec1bc713f..d19236f4425 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -68,17 +68,19 @@ def drop_table(cluster): # S3 request will be failed for an appropriate part file write. FILES_PER_PART_BASE = 5 # partition.dat, default_compression_codec.txt, count.txt, columns.txt, checksums.txt FILES_PER_PART_WIDE = FILES_PER_PART_BASE + 1 + 1 + 3 * 2 # Primary index, MinMax, Mark and data file for column(s) +FILES_PER_PART_WIDE_DEBUG = 2 # Additional requests to S3 in debug build FILES_PER_PART_COMPACT = FILES_PER_PART_BASE + 1 + 1 + 2 +FILES_PER_PART_COMPACT_DEBUG = 0 @pytest.mark.parametrize( - "min_bytes_for_wide_part,request_count", + "min_bytes_for_wide_part,request_count,debug_request_count", [ - (0, FILES_PER_PART_WIDE), - (1024 * 1024, FILES_PER_PART_COMPACT) + (0, FILES_PER_PART_WIDE, FILES_PER_PART_WIDE_DEBUG), + (1024 * 1024, FILES_PER_PART_COMPACT, FILES_PER_PART_COMPACT_DEBUG) ] ) -def test_write_failover(cluster, min_bytes_for_wide_part, request_count): +def test_write_failover(cluster, min_bytes_for_wide_part, request_count, debug_request_count): node = cluster.instances["node"] node.query( @@ -95,17 +97,24 @@ def test_write_failover(cluster, min_bytes_for_wide_part, request_count): .format(min_bytes_for_wide_part) ) - for request in range(request_count + 1): + is_debug_mode = False + success_count = 0 + + for request in range(request_count + debug_request_count + 1): # Fail N-th request to S3. fail_request(cluster, request + 1) data = "('2020-03-01',0,'data'),('2020-03-01',1,'data')" - positive = request == request_count + positive = request >= (request_count + debug_request_count if is_debug_mode else request_count) try: node.query("INSERT INTO s3_failover_test VALUES {}".format(data)) - assert positive, "Insert query should be failed, request {}".format(request) + success_count += 1 except QueryRuntimeException as e: + if not is_debug_mode and positive: + is_debug_mode = True + positive = False + assert not positive, "Insert query shouldn't be failed, request {}".format(request) assert str(e).find("Expected Error") != -1, "Unexpected error {}".format(str(e)) @@ -114,7 +123,9 @@ def test_write_failover(cluster, min_bytes_for_wide_part, request_count): fail_request(cluster, 0) assert node.query("CHECK TABLE s3_failover_test") == '1\n' - assert node.query("SELECT * FROM s3_failover_test FORMAT Values") == data + assert success_count > 1 or node.query("SELECT * FROM s3_failover_test FORMAT Values") == data + + assert success_count == (1 if is_debug_mode else debug_request_count + 1), "Insert query should be successful at least once" # Check that second data part move is ended successfully if first attempt was failed. From bb6d030fb8beff3dfc29d04ed998a6fb23631e8c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 17 Jul 2021 08:04:55 +0300 Subject: [PATCH 018/147] Optimize distributed SELECT w/o GROUP BY --- src/Storages/StorageDistributed.cpp | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 5173abeb9f9..b6a04c5cd34 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -338,16 +338,8 @@ std::optional getOptimizedQueryProcessingStage(const // GROUP BY const ASTPtr group_by = select.groupBy(); - if (!group_by) - { - if (!select.distinct) - return {}; - } - else - { - if (!sharding_block_has(group_by->children)) - return {}; - } + if (!query_info.syntax_analyzer_result->aggregates.empty() && (!group_by || !sharding_block_has(group_by->children))) + return {}; // ORDER BY const ASTPtr order_by = select.orderBy(); From 2fb95d9ee0711e2327f3be884fa130e82b8c04b3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 17 Jul 2021 16:31:06 +0300 Subject: [PATCH 019/147] Rework SELECT from Distributed query stages optimization Before this patch it wasn't possible to optimize simple SELECT * FROM dist ORDER BY (w/o GROUP BY and DISTINCT) to more optimal stage (QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit), since that code was under allow_nondeterministic_optimize_skip_unused_shards, rework it and make it possible. Also now distributed_push_down_limit is respected for optimize_distributed_group_by_sharding_key. Next step will be to enable distributed_push_down_limit by default. v2: fix detection of aggregates --- src/Storages/StorageDistributed.cpp | 166 +++++++++--------- src/Storages/StorageDistributed.h | 18 ++ ...istributed_group_by_sharding_key.reference | 4 +- ...mize_distributed_group_by_sharding_key.sql | 6 +- ...1814_distributed_push_down_limit.reference | 12 -- .../01814_distributed_push_down_limit.sh | 29 +-- 6 files changed, 109 insertions(+), 126 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b6a04c5cd34..85a2efb9963 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -284,78 +284,6 @@ void replaceConstantExpressions( visitor.visit(node); } -/// This is the implementation of optimize_distributed_group_by_sharding_key. -/// It returns up to which stage the query can be processed on a shard, which -/// is one of the following: -/// - QueryProcessingStage::Complete -/// - QueryProcessingStage::WithMergeableStateAfterAggregation -/// - QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit -/// - none (in this case regular WithMergeableState should be used) -std::optional getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, bool extremes, const Names & sharding_key_columns) -{ - const auto & select = query_info.query->as(); - - auto sharding_block_has = [&](const auto & exprs) -> bool - { - std::unordered_set expr_columns; - for (auto & expr : exprs) - { - auto id = expr->template as(); - if (!id) - continue; - expr_columns.emplace(id->name()); - } - - for (const auto & column : sharding_key_columns) - { - if (!expr_columns.contains(column)) - return false; - } - - return true; - }; - - // GROUP BY qualifiers - // - TODO: WITH TOTALS can be implemented - // - TODO: WITH ROLLUP can be implemented (I guess) - if (select.group_by_with_totals || select.group_by_with_rollup || select.group_by_with_cube) - return {}; - - // Window functions are not supported. - if (query_info.has_window) - return {}; - - // TODO: extremes support can be implemented - if (extremes) - return {}; - - // DISTINCT - if (select.distinct) - { - if (!sharding_block_has(select.select()->children)) - return {}; - } - - // GROUP BY - const ASTPtr group_by = select.groupBy(); - if (!query_info.syntax_analyzer_result->aggregates.empty() && (!group_by || !sharding_block_has(group_by->children))) - return {}; - - // ORDER BY - const ASTPtr order_by = select.orderBy(); - if (order_by) - return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; - - // LIMIT BY - // LIMIT - // OFFSET - if (select.limitBy() || select.limitLength() || select.limitOffset()) - return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; - - // Only simple SELECT FROM GROUP BY sharding_key can use Complete state. - return QueryProcessingStage::Complete; -} - size_t getClusterQueriedNodes(const Settings & settings, const ClusterPtr & cluster) { size_t num_local_shards = cluster->getLocalShardCount(); @@ -523,9 +451,6 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( } } - if (settings.distributed_push_down_limit) - return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; - /// Nested distributed query cannot return Complete stage, /// since the parent query need to aggregate the results after. if (to_stage == QueryProcessingStage::WithMergeableState) @@ -536,22 +461,89 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( if (getClusterQueriedNodes(settings, cluster) == 1) return QueryProcessingStage::Complete; - if (settings.optimize_skip_unused_shards && - settings.optimize_distributed_group_by_sharding_key && - has_sharding_key && - (settings.allow_nondeterministic_optimize_skip_unused_shards || sharding_key_is_deterministic)) - { - auto stage = getOptimizedQueryProcessingStage(query_info, settings.extremes, sharding_key_expr->getRequiredColumns()); - if (stage) - { - LOG_DEBUG(log, "Force processing stage to {}", QueryProcessingStage::toString(*stage)); - return *stage; - } - } + auto optimized_stage = getOptimizedQueryProcessingStage(query_info, settings); + if (optimized_stage) + return *optimized_stage; return QueryProcessingStage::WithMergeableState; } +std::optional StorageDistributed::getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, const Settings & settings) const +{ + bool optimize_sharding_key_aggregation = + settings.optimize_skip_unused_shards && + settings.optimize_distributed_group_by_sharding_key && + has_sharding_key && + (settings.allow_nondeterministic_optimize_skip_unused_shards || sharding_key_is_deterministic); + + QueryProcessingStage::Enum default_stage = QueryProcessingStage::WithMergeableStateAfterAggregation; + if (settings.distributed_push_down_limit) + default_stage = QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; + + const auto & select = query_info.query->as(); + + auto expr_contains_sharding_key = [&](const auto & exprs) -> bool + { + std::unordered_set expr_columns; + for (auto & expr : exprs) + { + auto id = expr->template as(); + if (!id) + continue; + expr_columns.emplace(id->name()); + } + + for (const auto & column : sharding_key_expr->getRequiredColumns()) + { + if (!expr_columns.contains(column)) + return false; + } + + return true; + }; + + // GROUP BY qualifiers + // - TODO: WITH TOTALS can be implemented + // - TODO: WITH ROLLUP can be implemented (I guess) + if (select.group_by_with_totals || select.group_by_with_rollup || select.group_by_with_cube) + return {}; + // Window functions are not supported. + if (query_info.has_window) + return {}; + // TODO: extremes support can be implemented + if (settings.extremes) + return {}; + + // DISTINCT + if (select.distinct) + { + if (!optimize_sharding_key_aggregation || !expr_contains_sharding_key(select.select()->children)) + return {}; + } + + // GROUP BY + const ASTPtr group_by = select.groupBy(); + if (!query_info.syntax_analyzer_result->aggregates.empty() || group_by) + { + if (!optimize_sharding_key_aggregation || !group_by || !expr_contains_sharding_key(group_by->children)) + return {}; + } + + // ORDER BY + const ASTPtr order_by = select.orderBy(); + if (order_by) + return default_stage; + + // LIMIT BY + // LIMIT + // OFFSET + if (select.limitBy() || select.limitLength() || select.limitOffset()) + return default_stage; + + // Only simple SELECT FROM GROUP BY sharding_key can use Complete state. + return QueryProcessingStage::Complete; +} + Pipe StorageDistributed::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index bf48e814ae2..e09eda00224 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -176,6 +176,24 @@ private: ClusterPtr skipUnusedShards(ClusterPtr cluster, const ASTPtr & query_ptr, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) const; + /// This method returns optimal query processing stage. + /// + /// Here is the list of stages (from the less optimal to more optimal): + /// - WithMergeableState + /// - WithMergeableStateAfterAggregation + /// - WithMergeableStateAfterAggregationAndLimit + /// - Complete + /// + /// Some simple queries w/o GROUP BY/DISTINCT can use more optimal stage. + /// + /// Also in case of optimize_distributed_group_by_sharding_key=1 the queries + /// with GROUP BY/DISTINCT sharding_key can also use more optimal stage. + /// (see also optimize_skip_unused_shards/allow_nondeterministic_optimize_skip_unused_shards) + /// + /// @return QueryProcessingStage or empty std::optoinal + /// (in this case regular WithMergeableState should be used) + std::optional getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, const Settings & settings) const; + size_t getRandomShardIndex(const Cluster::ShardsInfo & shards); const DistributedSettings & getDistributedSettingsRef() const { return distributed_settings; } diff --git a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference index 4442b0b6b61..8d356a6966f 100644 --- a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference @@ -57,7 +57,9 @@ LIMIT 1 0 LIMIT OFFSET 1 1 -OFFSET +OFFSET distributed_push_down_limit=0 +1 1 +OFFSET distributed_push_down_limit=1 1 1 1 0 1 1 diff --git a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.sql b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.sql index 4719119165a..10b47f64cc6 100644 --- a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.sql +++ b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.sql @@ -60,8 +60,10 @@ select 'LIMIT'; select count(), * from dist_01247 group by number limit 1; select 'LIMIT OFFSET'; select count(), * from dist_01247 group by number limit 1 offset 1; -select 'OFFSET'; -select count(), * from dist_01247 group by number offset 1; +select 'OFFSET distributed_push_down_limit=0'; +select count(), * from dist_01247 group by number offset 1 settings distributed_push_down_limit=0; +select 'OFFSET distributed_push_down_limit=1'; +select count(), * from dist_01247 group by number offset 1 settings distributed_push_down_limit=1; -- this will emulate different data on for different shards select 'WHERE LIMIT OFFSET'; select count(), * from dist_01247 where number = _shard_num-1 group by number order by number limit 1 offset 1; diff --git a/tests/queries/0_stateless/01814_distributed_push_down_limit.reference b/tests/queries/0_stateless/01814_distributed_push_down_limit.reference index f879f2cbd21..c542b5b7325 100644 --- a/tests/queries/0_stateless/01814_distributed_push_down_limit.reference +++ b/tests/queries/0_stateless/01814_distributed_push_down_limit.reference @@ -12,18 +12,6 @@ distributed_push_down_limit=1 8 9 40 40 -auto-distributed_push_down_limit -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 -40 40 distributed_push_down_limit=1 with OFFSET 97 96 diff --git a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh index 93321646037..24b27e74ba5 100755 --- a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh +++ b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh @@ -86,9 +86,11 @@ function test_distributed_push_down_limit_0() function test_distributed_push_down_limit_1() { local args=( - "remote('127.{2,3}', $CLICKHOUSE_DATABASE, data_01814)" + "remote('127.{2,3}', $CLICKHOUSE_DATABASE, data_01814, key)" 0 # offset --distributed_push_down_limit 1 + --optimize_skip_unused_shards 1 + --optimize_distributed_group_by_sharding_key 1 ) test_distributed_push_down_limit_with_query_log "${args[@]}" } @@ -97,22 +99,11 @@ function test_distributed_push_down_limit_1_offset() { local settings_and_opts=( --distributed_push_down_limit 1 - ) - - $CLICKHOUSE_CLIENT "${settings_and_opts[@]}" -q "select * from remote('127.{2,3}', $CLICKHOUSE_DATABASE, data_01814) group by key order by key desc limit 5, 10" -} - -function test_auto_distributed_push_down_limit() -{ - local args=( - dist_01814 - 0 # offset --optimize_skip_unused_shards 1 --optimize_distributed_group_by_sharding_key 1 - --prefer_localhost_replica 0 - --distributed_push_down_limit 0 ) - test_distributed_push_down_limit_with_query_log "${args[@]}" + + $CLICKHOUSE_CLIENT "${settings_and_opts[@]}" -q "select * from remote('127.{2,3}', $CLICKHOUSE_DATABASE, data_01814, key) group by key order by key desc limit 5, 10" } function main() @@ -151,16 +142,6 @@ function main() done echo "$out" - echo 'auto-distributed_push_down_limit' - for ((i = 0; i < max_tries; ++i)); do - out=$(test_auto_distributed_push_down_limit) - out_lines=( $out ) - if [[ ${#out_lines[@]} -gt 2 ]] && [[ ${out_lines[-1]} = 40 ]] && [[ ${out_lines[-2]} = 40 ]]; then - break - fi - done - echo "$out" - echo 'distributed_push_down_limit=1 with OFFSET' test_distributed_push_down_limit_1_offset } From c83096424029eadce104aaeb948774aa4cf5ed3e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 17 Jul 2021 17:16:29 +0300 Subject: [PATCH 020/147] Cover distributed_push_down_limit for queries w/o sharding expr --- ...1951_distributed_push_down_limit.reference | 32 +++++++++++++++++++ .../01951_distributed_push_down_limit.sql | 3 ++ 2 files changed, 35 insertions(+) create mode 100644 tests/queries/0_stateless/01951_distributed_push_down_limit.reference create mode 100644 tests/queries/0_stateless/01951_distributed_push_down_limit.sql diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference new file mode 100644 index 00000000000..9e803a171c4 --- /dev/null +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference @@ -0,0 +1,32 @@ +-- { echo } +explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; +Expression (Projection) + Limit (preliminary LIMIT) + MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; +Expression (Projection) + Limit (preliminary LIMIT) + MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + Limit (preliminary LIMIT) + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.sql b/tests/queries/0_stateless/01951_distributed_push_down_limit.sql new file mode 100644 index 00000000000..0d6e2069215 --- /dev/null +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.sql @@ -0,0 +1,3 @@ +-- { echo } +explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; +explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; From ff12f5102a0f549f0716954657585ab4cdf1d904 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 17 Jul 2021 20:21:13 +0300 Subject: [PATCH 021/147] Avoid running LIMIT BY/DISTINCT step on the initiator for optimize_distributed_group_by_sharding_key Before the following queries was running LimitBy/Distinct step on the initator: select distinct sharding_key from dist order by k While this can be omitted. --- src/Interpreters/InterpreterSelectQuery.cpp | 4 +- src/Storages/StorageDistributed.cpp | 12 +- ...rd_distributed_group_by_no_merge.reference | 2 + ...istributed_group_by_sharding_key.reference | 2 + ...istributed_group_by_sharding_key.reference | 115 ++++++++++++++++++ ...mize_distributed_group_by_sharding_key.sql | 13 ++ 6 files changed, 143 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference create mode 100644 tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c126f3bca88..a5b6ff061bc 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1341,10 +1341,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu /** If there was more than one stream, * then DISTINCT needs to be performed once again after merging all streams. */ - if (query.distinct) + if (!from_aggregation_stage && query.distinct) executeDistinct(query_plan, false, expressions.selected_columns, false); - if (expressions.hasLimitBy()) + if (!from_aggregation_stage && expressions.hasLimitBy()) { executeExpression(query_plan, expressions.before_limit_by, "Before LIMIT BY"); executeLimitBy(query_plan); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 85a2efb9963..2bbb92cf0b8 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -529,15 +529,21 @@ std::optional StorageDistributed::getOptimizedQueryP return {}; } + // LIMIT BY + if (const ASTPtr limit_by = select.limitBy()) + { + if (!optimize_sharding_key_aggregation || !expr_contains_sharding_key(limit_by->children)) + return {}; + } + // ORDER BY - const ASTPtr order_by = select.orderBy(); - if (order_by) + if (const ASTPtr order_by = select.orderBy()) return default_stage; // LIMIT BY // LIMIT // OFFSET - if (select.limitBy() || select.limitLength() || select.limitOffset()) + if (select.limitLength() || select.limitOffset()) return default_stage; // Only simple SELECT FROM GROUP BY sharding_key can use Complete state. diff --git a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference index b667c57a14c..b2b0b43e490 100644 --- a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference +++ b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference @@ -25,6 +25,8 @@ ORDER BY LIMIT LIMIT BY 0 1 +0 +1 LIMIT BY LIMIT 0 GROUP BY ORDER BY diff --git a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference index 8d356a6966f..a4a6b87de25 100644 --- a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference @@ -67,6 +67,8 @@ WHERE LIMIT OFFSET 1 1 LIMIT BY 1 1 0 +1 0 +1 1 1 1 GROUP BY (Distributed-over-Distributed) 4 0 diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference new file mode 100644 index 00000000000..10787068f43 --- /dev/null +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference @@ -0,0 +1,115 @@ +-- { echo } +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +Expression (Projection) + Distinct + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized +SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + Expression (Projection) + Distinct + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +Expression (Projection) + LimitBy + Expression (Before LIMIT BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + LimitBy + Expression ((Before LIMIT BY + Before ORDER BY)) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized +SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + Expression (Projection) + LimitBy + Expression ((Before LIMIT BY + Before ORDER BY)) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +Expression (Projection) + Distinct + MergingSorted (Merge sorted streams for ORDER BY, without aggregation) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized +Expression (Projection) + MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + Distinct + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +Expression (Projection) + LimitBy + Expression (Before LIMIT BY) + MergingSorted (Merge sorted streams for ORDER BY, without aggregation) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + LimitBy + Expression (Before LIMIT BY) + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized +Expression (Projection) + MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + LimitBy + Expression (Before LIMIT BY) + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql new file mode 100644 index 00000000000..2ae872f72b0 --- /dev/null +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql @@ -0,0 +1,13 @@ +set optimize_skip_unused_shards=1; +set optimize_distributed_group_by_sharding_key=1; + +-- { echo } +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized + +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized From 97851bde088ab3289064b808da7dd63d541d8ea7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 3 Aug 2021 10:10:08 +0300 Subject: [PATCH 022/147] Fix Distributed over Distributed for WithMergeableStateAfterAggregation* stages In case if one Distributed has multiple shards, and underlying Distributed has only one, there can be the case when the query will be tried to process from Complete to WithMergeableStateAfterAggregation, which is obviously wrong. --- src/Storages/StorageDistributed.cpp | 15 ++++++++++++++- ...t_WithMergeableStateAfterAggregation.reference | 8 ++++++++ ...on_dist_WithMergeableStateAfterAggregation.sql | 6 ++++++ 3 files changed, 28 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.reference create mode 100644 tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.sql diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 2bbb92cf0b8..8f9c4bcc655 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -447,6 +447,8 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( { /// NOTE: distributed_group_by_no_merge=1 does not respect distributed_push_down_limit /// (since in this case queries processed separately and the initiator is just a proxy in this case). + if (to_stage != QueryProcessingStage::Complete) + throw Exception("Queries with distributed_group_by_no_merge=1 should be processed to Complete stage", ErrorCodes::LOGICAL_ERROR); return QueryProcessingStage::Complete; } } @@ -459,11 +461,22 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( /// If there is only one node, the query can be fully processed by the /// shard, initiator will work as a proxy only. if (getClusterQueriedNodes(settings, cluster) == 1) - return QueryProcessingStage::Complete; + { + /// In case the query was processed to + /// WithMergeableStateAfterAggregation/WithMergeableStateAfterAggregationAndLimit + /// (which are greater the Complete stage) + /// we cannot return Complete (will break aliases and similar), + /// relevant for Distributed over Distributed + return std::max(to_stage, QueryProcessingStage::Complete); + } auto optimized_stage = getOptimizedQueryProcessingStage(query_info, settings); if (optimized_stage) + { + if (*optimized_stage == QueryProcessingStage::Complete) + return std::min(to_stage, *optimized_stage); return *optimized_stage; + } return QueryProcessingStage::WithMergeableState; } diff --git a/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.reference b/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.reference new file mode 100644 index 00000000000..6c680840239 --- /dev/null +++ b/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.reference @@ -0,0 +1,8 @@ +-- { echo } +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_push_down_limit=0; +0 +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_push_down_limit=1; +0 +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_group_by_no_merge=1; +0 +0 diff --git a/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.sql b/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.sql new file mode 100644 index 00000000000..0925df1888d --- /dev/null +++ b/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.sql @@ -0,0 +1,6 @@ +drop table if exists dist; +create table dist as system.one engine=Distributed('test_shard_localhost', system, one); +-- { echo } +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_push_down_limit=0; +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_push_down_limit=1; +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_group_by_no_merge=1; From d8c112d0c0c9bbb52db7bbb8b435211b6c754ca2 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Tue, 3 Aug 2021 12:03:10 +0000 Subject: [PATCH 023/147] Small current fixes --- .../database-engines/materialize-mysql.md | 4 +- .../en/engines/database-engines/replicated.md | 5 + .../functions/other-functions.md | 155 ++++++++++++++++++ .../database-engines/materialize-mysql.md | 7 +- .../ru/engines/database-engines/replicated.md | 4 + 5 files changed, 167 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/database-engines/materialize-mysql.md b/docs/en/engines/database-engines/materialize-mysql.md index 198808fa952..b53e09bba89 100644 --- a/docs/en/engines/database-engines/materialize-mysql.md +++ b/docs/en/engines/database-engines/materialize-mysql.md @@ -3,14 +3,12 @@ toc_priority: 29 toc_title: MaterializeMySQL --- -# MaterializeMySQL {#materialize-mysql} +# [experimental] MaterializeMySQL {#materialize-mysql} Creates ClickHouse database with all the tables existing in MySQL, and all the data in those tables. ClickHouse server works as MySQL replica. It reads binlog and performs DDL and DML queries. -This feature is experimental. - ## Creating a Database {#creating-a-database} ``` sql diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index 575aa9d3ee5..18c39d1754d 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -1,3 +1,8 @@ +--- +toc_priority: 40 +toc_title: Replicated +--- + # [experimental] Replicated {#replicated} The engine is based on the [Atomic](../../engines/database-engines/atomic.md) engine. It supports replication of metadata via DDL log being written to ZooKeeper and executed on all of the replicas for a given database. diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 30e2e427158..63892bc712c 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2138,3 +2138,158 @@ Result: - [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) +## currentProfiles {#current-profiles} + +Short description. + +**Syntax** + +``` sql + +``` + +Alias: ``. (Optional) + +More text (Optional). + +**Arguments** (Optional) + +- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). +- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). + +**Parameters** (Optional, only for parametric aggregate functions) + +- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). + +**Returned value(s)** + +- Returned values list. + +Type: [Type name](relative/path/to/type/dscr.md#type). + +**Example** + +The example must show usage and/or a use cases. The following text contains recommended parts of an example. + +Input table (Optional): + +``` text +``` + +Query: + +``` sql +``` + +Result: + +``` text +``` + +**See Also** (Optional) + +- [link](#) + +## enabledProfiles {#enabled-profiles} + +Short description. + +**Syntax** + +``` sql + +``` + +Alias: ``. (Optional) + +More text (Optional). + +**Arguments** (Optional) + +- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). +- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). + +**Parameters** (Optional, only for parametric aggregate functions) + +- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). + +**Returned value(s)** + +- Returned values list. + +Type: [Type name](relative/path/to/type/dscr.md#type). + +**Example** + +The example must show usage and/or a use cases. The following text contains recommended parts of an example. + +Input table (Optional): + +``` text +``` + +Query: + +``` sql +``` + +Result: + +``` text +``` + +**See Also** (Optional) + +- [link](#) + +## defaultProfiles {#default-profiles} + +Short description. + +**Syntax** + +``` sql + +``` + +Alias: ``. (Optional) + +More text (Optional). + +**Arguments** (Optional) + +- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). +- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). + +**Parameters** (Optional, only for parametric aggregate functions) + +- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). + +**Returned value(s)** + +- Returned values list. + +Type: [Type name](relative/path/to/type/dscr.md#type). + +**Example** + +The example must show usage and/or a use cases. The following text contains recommended parts of an example. + +Input table (Optional): + +``` text +``` + +Query: + +``` sql +``` + +Result: + +``` text +``` + +**See Also** (Optional) + +- [link](#) \ No newline at end of file diff --git a/docs/ru/engines/database-engines/materialize-mysql.md b/docs/ru/engines/database-engines/materialize-mysql.md index b62670bc3c9..d2bef29fc16 100644 --- a/docs/ru/engines/database-engines/materialize-mysql.md +++ b/docs/ru/engines/database-engines/materialize-mysql.md @@ -1,16 +1,13 @@ - --- toc_priority: 29 toc_title: MaterializeMySQL --- -# MaterializeMySQL {#materialize-mysql} +# [экспериментальный] MaterializeMySQL {#materialize-mysql} Создает базу данных ClickHouse со всеми таблицами, существующими в MySQL, и всеми данными в этих таблицах. -Сервер ClickHouse работает как реплика MySQL. Он читает файл binlog и выполняет DDL and DML-запросы. - -`MaterializeMySQL` — экспериментальный движок баз данных. +Сервер ClickHouse работает как реплика MySQL. Он читает файл binlog и выполняет DDL и DML-запросы. ## Создание базы данных {#creating-a-database} diff --git a/docs/ru/engines/database-engines/replicated.md b/docs/ru/engines/database-engines/replicated.md index 62be07f617c..063c19bcebd 100644 --- a/docs/ru/engines/database-engines/replicated.md +++ b/docs/ru/engines/database-engines/replicated.md @@ -1,3 +1,7 @@ +--- +toc_priority: 40 +toc_title: Replicated +--- # [экспериментальный] Replicated {#replicated} From ff2572c243aa561c376be5e275ae3f3918390a58 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 3 Aug 2021 16:28:03 +0300 Subject: [PATCH 024/147] fix --- src/Databases/DatabaseReplicated.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index ac6c525af68..26dd8763c40 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -525,7 +525,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep query_context->getClientInfo().is_replicated_database_internal = true; query_context->setCurrentDatabase(database_name); query_context->setCurrentQueryId(""); - auto txn = std::make_shared(current_zookeeper, zookeeper_path, false); + auto txn = std::make_shared(current_zookeeper, zookeeper_path, false, ""); query_context->initZooKeeperMetadataTransaction(txn); return query_context; }; From 699a3d9031c1ba88503b7e275ab6c4f6d2bfce96 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 3 Aug 2021 21:03:24 +0300 Subject: [PATCH 025/147] implement legacy_column_name_of_tuple_literal in less intrusive way --- src/Interpreters/ActionsVisitor.cpp | 26 ++++----- src/Interpreters/ExpressionAnalyzer.cpp | 54 +++++++++---------- src/Interpreters/InterpreterSelectQuery.cpp | 22 ++++---- src/Interpreters/TreeRewriter.cpp | 27 ++++++++++ .../evaluateConstantExpression.cpp | 2 +- src/Parsers/ASTFunction.cpp | 20 +------ src/Parsers/ASTFunction.h | 4 -- src/Parsers/ASTLiteral.cpp | 14 +++-- src/Parsers/ASTLiteral.h | 5 +- src/Parsers/ASTWithAlias.cpp | 8 --- src/Parsers/ASTWithAlias.h | 3 -- src/Parsers/IAST.cpp | 8 --- src/Parsers/IAST.h | 4 -- .../02002_system_table_with_tuple.reference | 1 + .../02002_system_table_with_tuple.sql | 2 + 15 files changed, 92 insertions(+), 108 deletions(-) create mode 100644 tests/queries/0_stateless/02002_system_table_with_tuple.reference create mode 100644 tests/queries/0_stateless/02002_system_table_with_tuple.sql diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 03fa756276e..61e484ff6f1 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -348,7 +348,7 @@ SetPtr makeExplicitSet( const ASTPtr & left_arg = args.children.at(0); const ASTPtr & right_arg = args.children.at(1); - auto column_name = left_arg->getColumnName(context->getSettingsRef()); + auto column_name = left_arg->getColumnName(); const auto & dag_node = actions.findInIndex(column_name); const DataTypePtr & left_arg_type = dag_node.result_type; @@ -641,7 +641,7 @@ std::optional ActionsMatcher::getNameAndTypeFromAST(const ASTPt { // If the argument is a literal, we generated a unique column name for it. // Use it instead of a generic display name. - auto child_column_name = ast->getColumnName(data.getContext()->getSettingsRef()); + auto child_column_name = ast->getColumnName(); const auto * as_literal = ast->as(); if (as_literal) { @@ -698,7 +698,7 @@ ASTs ActionsMatcher::doUntuple(const ASTFunction * function, ActionsMatcher::Dat auto func = makeASTFunction("tupleElement", tuple_ast, literal); auto function_builder = FunctionFactory::instance().get(func->name, data.getContext()); - data.addFunction(function_builder, {tuple_name_type->name, literal->getColumnName(data.getContext()->getSettingsRef())}, func->getColumnName(data.getContext()->getSettingsRef())); + data.addFunction(function_builder, {tuple_name_type->name, literal->getColumnName()}, func->getColumnName()); columns.push_back(std::move(func)); } @@ -762,7 +762,7 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Dat void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data) { - auto column_name = ast->getColumnName(data.getContext()->getSettingsRef()); + auto column_name = ast->getColumnName(); if (data.hasColumn(column_name)) return; @@ -778,7 +778,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & ASTPtr arg = node.arguments->children.at(0); visit(arg, data); if (!data.only_consts) - data.addArrayJoin(arg->getColumnName(data.getContext()->getSettingsRef()), column_name); + data.addArrayJoin(arg->getColumnName(), column_name); return; } @@ -800,7 +800,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// We are in the part of the tree that we are not going to compute. You just need to define types. /// Do not subquery and create sets. We replace "in*" function to "in*IgnoreSet". - auto argument_name = node.arguments->children.at(0)->getColumnName(data.getContext()->getSettingsRef()); + auto argument_name = node.arguments->children.at(0)->getColumnName(); data.addFunction( FunctionFactory::instance().get(node.name + "IgnoreSet", data.getContext()), @@ -929,7 +929,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (!prepared_set->empty()) column.name = data.getUniqueName("__set"); else - column.name = child->getColumnName(data.getContext()->getSettingsRef()); + column.name = child->getColumnName(); if (!data.hasColumn(column.name)) { @@ -1008,7 +1008,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & visit(lambda->arguments->children.at(1), data); auto lambda_dag = data.actions_stack.popLevel(); - String result_name = lambda->arguments->children.at(1)->getColumnName(data.getContext()->getSettingsRef()); + String result_name = lambda->arguments->children.at(1)->getColumnName(); lambda_dag->removeUnusedActions(Names(1, result_name)); auto lambda_actions = std::make_shared( @@ -1023,7 +1023,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end()) captured.push_back(required_arg); - /// We can not name `getColumnName(data.getContext()->getSettingsRef())`, + /// We can not name `getColumnName()`, /// because it does not uniquely define the expression (the types of arguments can be different). String lambda_name = data.getUniqueName("__lambda"); @@ -1053,7 +1053,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (arguments_present) { /// Calculate column name here again, because AST may be changed here (in case of untuple). - data.addFunction(function_builder, argument_names, ast->getColumnName(data.getContext()->getSettingsRef())); + data.addFunction(function_builder, argument_names, ast->getColumnName()); } } @@ -1067,7 +1067,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, // AST here? Anyway, do not modify the column name if it is set already. if (literal.unique_column_name.empty()) { - const auto default_name = literal.getColumnName(data.getContext()->getSettingsRef()); + const auto default_name = literal.getColumnName(); const auto & index = data.actions_stack.getLastActionsIndex(); const auto * existing_column = index.tryGetNode(default_name); @@ -1147,7 +1147,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su } /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. - String set_id = right_in_operand->getColumnName(data.getContext()->getSettingsRef()); + String set_id = right_in_operand->getColumnName(); SubqueryForSet & subquery_for_set = data.subqueries_for_sets[set_id]; @@ -1183,7 +1183,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su { const auto & last_actions = data.actions_stack.getLastActions(); const auto & index = data.actions_stack.getLastActionsIndex(); - if (index.contains(left_in_operand->getColumnName(data.getContext()->getSettingsRef()))) + if (index.contains(left_in_operand->getColumnName())) /// An explicit enumeration of values in parentheses. return makeExplicitSet(&node, last_actions, false, data.getContext(), data.set_size_limit, data.prepared_sets); else diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 66c1cb9ad7b..6b55c728a96 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -243,7 +243,7 @@ void ExpressionAnalyzer::analyzeAggregation() ssize_t size = group_asts.size(); getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false); - const auto & column_name = group_asts[i]->getColumnName(getContext()->getSettingsRef()); + const auto & column_name = group_asts[i]->getColumnName(); const auto * node = temp_actions->tryFindInIndex(column_name); if (!node) throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); @@ -408,7 +408,7 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) auto temp_actions = std::make_shared(columns_after_join); getRootActions(left_in_operand, true, temp_actions); - if (temp_actions->tryFindInIndex(left_in_operand->getColumnName(getContext()->getSettingsRef()))) + if (temp_actions->tryFindInIndex(left_in_operand->getColumnName())) makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, prepared_sets); } } @@ -456,7 +456,7 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions) if (node->arguments) getRootActionsNoMakeSet(node->arguments, true, actions); - aggregate.column_name = node->getColumnName(getContext()->getSettingsRef()); + aggregate.column_name = node->getColumnName(); const ASTs & arguments = node->arguments ? node->arguments->children : ASTs(); aggregate.argument_names.resize(arguments.size()); @@ -464,7 +464,7 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions) for (size_t i = 0; i < arguments.size(); ++i) { - const std::string & name = arguments[i]->getColumnName(getContext()->getSettingsRef()); + const std::string & name = arguments[i]->getColumnName(); const auto * dag_node = actions->tryFindInIndex(name); if (!dag_node) { @@ -657,7 +657,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) WindowFunctionDescription window_function; window_function.function_node = function_node; window_function.column_name - = window_function.function_node->getColumnName(getContext()->getSettingsRef()); + = window_function.function_node->getColumnName(); window_function.function_parameters = window_function.function_node->parameters ? getAggregateFunctionParametersArray( @@ -676,7 +676,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) window_function.argument_names.resize(arguments.size()); for (size_t i = 0; i < arguments.size(); ++i) { - const std::string & name = arguments[i]->getColumnName(getContext()->getSettingsRef()); + const std::string & name = arguments[i]->getColumnName(); const auto * node = actions->tryFindInIndex(name); if (!node) @@ -973,7 +973,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( auto & step = chain.lastStep(sourceColumns()); getRootActions(select_query->prewhere(), only_types, step.actions()); - String prewhere_column_name = select_query->prewhere()->getColumnName(getContext()->getSettingsRef()); + String prewhere_column_name = select_query->prewhere()->getColumnName(); step.addRequiredOutput(prewhere_column_name); const auto & node = step.actions()->findInIndex(prewhere_column_name); @@ -1059,7 +1059,7 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, getRootActions(select_query->where(), only_types, step.actions()); - auto where_column_name = select_query->where()->getColumnName(getContext()->getSettingsRef()); + auto where_column_name = select_query->where()->getColumnName(); step.addRequiredOutput(where_column_name); const auto & node = step.actions()->findInIndex(where_column_name); @@ -1084,7 +1084,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain ASTs asts = select_query->groupBy()->children; for (const auto & ast : asts) { - step.addRequiredOutput(ast->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(ast->getColumnName()); getRootActions(ast, only_types, step.actions()); } @@ -1112,7 +1112,7 @@ void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(Expression for (const auto & name : desc.argument_names) step.addRequiredOutput(name); - /// Collect aggregates removing duplicates by node.getColumnName(getContext()->getSettingsRef()) + /// Collect aggregates removing duplicates by node.getColumnName() /// It's not clear why we recollect aggregates (for query parts) while we're able to use previously collected ones (for entire query) /// @note The original recollection logic didn't remove duplicates. GetAggregatesVisitor::Data data; @@ -1167,7 +1167,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments( // (2b) Required function argument columns. for (const auto & a : f.function_node->arguments->children) { - step.addRequiredOutput(a->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(a->getColumnName()); } } @@ -1189,7 +1189,7 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); getRootActionsForHaving(select_query->having(), only_types, step.actions()); - step.addRequiredOutput(select_query->having()->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(select_query->having()->getColumnName()); return true; } @@ -1213,7 +1213,7 @@ void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, continue; } - step.addRequiredOutput(child->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(child->getColumnName()); } } @@ -1241,7 +1241,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai if (!ast || ast->children.empty()) throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); ASTPtr order_expression = ast->children.at(0); - step.addRequiredOutput(order_expression->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(order_expression->getColumnName()); if (ast->with_fill) with_fill = true; @@ -1291,7 +1291,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain for (const auto & child : select_query->limitBy()->children) { - auto child_name = child->getColumnName(getContext()->getSettingsRef()); + auto child_name = child->getColumnName(); if (!aggregated_names.count(child_name)) step.addRequiredOutput(std::move(child_name)); } @@ -1307,15 +1307,13 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio NamesWithAliases result_columns; - const auto & settings = getContext()->getSettingsRef(); - ASTs asts = select_query->select()->children; for (const auto & ast : asts) { - String result_name = ast->getAliasOrColumnName(settings); + String result_name = ast->getAliasOrColumnName(); if (required_result_columns.empty() || required_result_columns.count(result_name)) { - std::string source_name = ast->getColumnName(settings); + std::string source_name = ast->getColumnName(); /* * For temporary columns created by ExpressionAnalyzer for literals, @@ -1357,7 +1355,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const { ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns()); getRootActions(expr, only_types, step.actions()); - step.addRequiredOutput(expr->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(expr->getColumnName()); } @@ -1374,13 +1372,12 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r else asts = ASTs(1, query); - const auto & settings = getContext()->getSettingsRef(); for (const auto & ast : asts) { - std::string name = ast->getColumnName(settings); + std::string name = ast->getColumnName(); std::string alias; if (add_aliases) - alias = ast->getAliasOrColumnName(settings); + alias = ast->getAliasOrColumnName(); else alias = name; result_columns.emplace_back(name, alias); @@ -1509,7 +1506,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) { - prewhere_info = std::make_shared(actions, query.prewhere()->getColumnName(settings)); + prewhere_info = std::make_shared(actions, query.prewhere()->getColumnName()); if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings)) { @@ -1519,7 +1516,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( ExpressionActions( prewhere_info->prewhere_actions, ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); - auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName(settings)); + auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) prewhere_constant_filter_description = ConstantFilterDescription(*column_elem.column); @@ -1554,7 +1551,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( ExpressionActions( before_where, ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); - auto & column_elem = before_where_sample.getByName(query.where()->getColumnName(settings)); + auto & column_elem = before_where_sample.getByName(query.where()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) where_constant_filter_description = ConstantFilterDescription(*column_elem.column); @@ -1645,7 +1642,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( const auto * select_query = query_analyzer.getSelectQuery(); for (const auto & child : select_query->select()->children) { - step.addRequiredOutput(child->getColumnName(settings)); + step.addRequiredOutput(child->getColumnName()); } } @@ -1701,8 +1698,7 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si if (hasWhere()) { - const auto & settings = chain.getContext()->getSettingsRef(); - where_column_name = query.where()->getColumnName(settings); + where_column_name = query.where()->getColumnName(); remove_where_filter = chain.steps.at(where_step_num)->required_output.find(where_column_name)->second; } } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c126f3bca88..d24974885b8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -141,7 +141,7 @@ String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, co SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot); actions = analyzer.simpleSelectActions(); - auto column_name = expr_list->children.at(0)->getColumnName(context->getSettingsRef()); + auto column_name = expr_list->children.at(0)->getColumnName(); actions->removeUnusedActions(NameSet{column_name}); actions->projectInput(false); @@ -782,7 +782,7 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP order_descr.reserve(query.orderBy()->children.size()); for (const auto & elem : query.orderBy()->children) { - String name = elem->children.front()->getColumnName(context->getSettingsRef()); + String name = elem->children.front()->getColumnName(); const auto & order_by_elem = elem->as(); std::shared_ptr collator; @@ -801,14 +801,14 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP return order_descr; } -static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query, ContextPtr context) +static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query) { SortDescription order_descr; order_descr.reserve(query.groupBy()->children.size()); for (const auto & elem : query.groupBy()->children) { - String name = elem->getColumnName(context->getSettingsRef()); + String name = elem->getColumnName(); order_descr.emplace_back(name, 1, 1); } @@ -1918,13 +1918,13 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc { query_info.projection->order_optimizer = std::make_shared( query_info.projection->group_by_elements_actions, - getSortDescriptionFromGroupBy(query, context), + getSortDescriptionFromGroupBy(query), query_info.syntax_analyzer_result); } else { query_info.order_optimizer = std::make_shared( - analysis_result.group_by_elements_actions, getSortDescriptionFromGroupBy(query, context), query_info.syntax_analyzer_result); + analysis_result.group_by_elements_actions, getSortDescriptionFromGroupBy(query), query_info.syntax_analyzer_result); } } @@ -2003,7 +2003,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter) { auto where_step = std::make_unique( - query_plan.getCurrentDataStream(), expression, getSelectQuery().where()->getColumnName(context->getSettingsRef()), remove_filter); + query_plan.getCurrentDataStream(), expression, getSelectQuery().where()->getColumnName(), remove_filter); where_step->setStepDescription("WHERE"); query_plan.addStep(std::move(where_step)); @@ -2052,7 +2052,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac SortDescription group_by_sort_description; if (group_by_info && settings.optimize_aggregation_in_order) - group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery(), context); + group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery()); else group_by_info = nullptr; @@ -2100,7 +2100,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression) { auto having_step - = std::make_unique(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(context->getSettingsRef()), false); + = std::make_unique(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(), false); having_step->setStepDescription("HAVING"); query_plan.addStep(std::move(having_step)); @@ -2116,7 +2116,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving( query_plan.getCurrentDataStream(), overflow_row, expression, - has_having ? getSelectQuery().having()->getColumnName(context->getSettingsRef()) : "", + has_having ? getSelectQuery().having()->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold, final); @@ -2435,7 +2435,7 @@ void InterpreterSelectQuery::executeLimitBy(QueryPlan & query_plan) Names columns; for (const auto & elem : query.limitBy()->children) - columns.emplace_back(elem->getColumnName(context->getSettingsRef())); + columns.emplace_back(elem->getColumnName()); UInt64 length = getLimitUIntValue(query.limitByLength(), context, "LIMIT"); UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context, "OFFSET") : 0); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index cc345004f6f..a2b64d26d42 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -609,6 +609,27 @@ std::vector getWindowFunctions(ASTPtr & query, const ASTSel return data.window_functions; } +class MarkTupleLiteralsAsLegacyData +{ +public: + using TypeToVisit = ASTLiteral; + + void visit(ASTLiteral & literal, ASTPtr &) + { + if (literal.value.getType() == Field::Types::Tuple) + literal.use_legacy_column_name_of_tuple = true; + } +}; + +using MarkTupleLiteralsAsLegacyMatcher = OneTypeMatcher; +using MarkTupleLiteralsAsLegacyVisitor = InDepthNodeVisitor; + +void markTupleLiteralsAsLegacy(ASTPtr & query) +{ + MarkTupleLiteralsAsLegacyVisitor::Data data; + MarkTupleLiteralsAsLegacyVisitor(data).visit(query); +} + } TreeRewriterResult::TreeRewriterResult( @@ -927,6 +948,9 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( /// Executing scalar subqueries - replacing them with constant values. executeScalarSubqueries(query, getContext(), subquery_depth, result.scalars, select_options.only_analyze); + if (settings.legacy_column_name_of_tuple_literal) + markTupleLiteralsAsLegacy(query); + TreeOptimizer::apply(query, result, tables_with_columns, getContext()); /// array_join_alias_to_name, array_join_result_to_source. @@ -994,6 +1018,9 @@ TreeRewriterResultPtr TreeRewriter::analyze( /// Executing scalar subqueries. Column defaults could be a scalar subquery. executeScalarSubqueries(query, getContext(), 0, result.scalars, false); + if (settings.legacy_column_name_of_tuple_literal) + markTupleLiteralsAsLegacy(query); + TreeOptimizer::optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif); if (allow_aggregations) diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index f814e1d8c02..e46f644e836 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -39,7 +39,7 @@ std::pair> evaluateConstantExpression(co if (context->getSettingsRef().normalize_function_names) FunctionNameNormalizer().visit(ast.get()); - String name = ast->getColumnName(context->getSettingsRef()); + String name = ast->getColumnName(); auto syntax_result = TreeRewriter(context).analyze(ast, source_columns); ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(ast, syntax_result, context).getConstActions(); diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index daae3e76aa1..1ff27c61836 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -24,16 +24,6 @@ namespace ErrorCodes } void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const -{ - appendColumnNameImpl(ostr, nullptr); -} - -void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const -{ - appendColumnNameImpl(ostr, &settings); -} - -void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr, const Settings * settings) const { if (name == "view") throw Exception("Table function view cannot be used as an expression", ErrorCodes::UNEXPECTED_EXPRESSION); @@ -48,10 +38,7 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr, const Settings * sett if (it != parameters->children.begin()) writeCString(", ", ostr); - if (settings) - (*it)->appendColumnName(ostr, *settings); - else - (*it)->appendColumnName(ostr); + (*it)->appendColumnName(ostr); } writeChar(')', ostr); } @@ -64,10 +51,7 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr, const Settings * sett if (it != arguments->children.begin()) writeCString(", ", ostr); - if (settings) - (*it)->appendColumnName(ostr, *settings); - else - (*it)->appendColumnName(ostr); + (*it)->appendColumnName(ostr); } } diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 8e657afbf6e..685aaaadd26 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -54,10 +54,6 @@ public: protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; - void appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const override; - -private: - void appendColumnNameImpl(WriteBuffer & ostr, const Settings * settings) const; }; diff --git a/src/Parsers/ASTLiteral.cpp b/src/Parsers/ASTLiteral.cpp index c456cb3e933..93d490bc645 100644 --- a/src/Parsers/ASTLiteral.cpp +++ b/src/Parsers/ASTLiteral.cpp @@ -50,16 +50,14 @@ String FieldVisitorToColumnName::operator() (const Tuple & x) const } -void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const -{ - if (settings.legacy_column_name_of_tuple_literal) - appendColumnNameImplLegacy(ostr); - else - appendColumnNameImpl(ostr); -} - void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const { + if (use_legacy_column_name_of_tuple) + { + appendColumnNameImplLegacy(ostr); + return; + } + /// 100 - just arbitrary value. constexpr auto min_elements_for_hashing = 100; diff --git a/src/Parsers/ASTLiteral.h b/src/Parsers/ASTLiteral.h index c17310f719b..856bed61979 100644 --- a/src/Parsers/ASTLiteral.h +++ b/src/Parsers/ASTLiteral.h @@ -33,6 +33,10 @@ public: */ String unique_column_name; + /// For compatibility reasons in distributed queries, + /// we may need to use legacy column name for tuple literal. + bool use_legacy_column_name_of_tuple = false; + /** Get the text that identifies this element. */ String getID(char delim) const override { return "Literal" + (delim + applyVisitor(FieldVisitorDump(), value)); } @@ -44,7 +48,6 @@ protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; - void appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const override; private: /// Legacy version of 'appendColumnNameImpl'. It differs only with tuple literals. diff --git a/src/Parsers/ASTWithAlias.cpp b/src/Parsers/ASTWithAlias.cpp index 0f5b86763e0..88f6568a719 100644 --- a/src/Parsers/ASTWithAlias.cpp +++ b/src/Parsers/ASTWithAlias.cpp @@ -48,14 +48,6 @@ void ASTWithAlias::appendColumnName(WriteBuffer & ostr) const appendColumnNameImpl(ostr); } -void ASTWithAlias::appendColumnName(WriteBuffer & ostr, const Settings & settings) const -{ - if (prefer_alias_to_column_name && !alias.empty()) - writeString(alias, ostr); - else - appendColumnNameImpl(ostr, settings); -} - void ASTWithAlias::appendColumnNameWithoutAlias(WriteBuffer & ostr) const { appendColumnNameImpl(ostr); diff --git a/src/Parsers/ASTWithAlias.h b/src/Parsers/ASTWithAlias.h index 249be17b74c..ea4419402b0 100644 --- a/src/Parsers/ASTWithAlias.h +++ b/src/Parsers/ASTWithAlias.h @@ -21,10 +21,8 @@ public: using IAST::IAST; void appendColumnName(WriteBuffer & ostr) const final; - void appendColumnName(WriteBuffer & ostr, const Settings & settings) const final; void appendColumnNameWithoutAlias(WriteBuffer & ostr) const final; String getAliasOrColumnName() const override { return alias.empty() ? getColumnName() : alias; } - String getAliasOrColumnName(const Settings & settings) const override { return alias.empty() ? getColumnName(settings) : alias; } String tryGetAlias() const override { return alias; } void setAlias(const String & to) override { alias = to; } @@ -35,7 +33,6 @@ public: protected: virtual void appendColumnNameImpl(WriteBuffer & ostr) const = 0; - virtual void appendColumnNameImpl(WriteBuffer & ostr, const Settings &) const { appendColumnNameImpl(ostr); } }; /// helper for setting aliases and chaining result to other functions diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 0f38fcf98dd..3a21d704eb9 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -109,14 +109,6 @@ String IAST::getColumnName() const } -String IAST::getColumnName(const Settings & settings) const -{ - WriteBufferFromOwnString write_buffer; - appendColumnName(write_buffer, settings); - return write_buffer.str(); -} - - String IAST::getColumnNameWithoutAlias() const { WriteBufferFromOwnString write_buffer; diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 143094e1d7a..f805ae7d8eb 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -42,7 +42,6 @@ public: /** Get the canonical name of the column if the element is a column */ String getColumnName() const; - String getColumnName(const Settings & settings) const; /** Same as the above but ensure no alias names are used. This is for index analysis */ String getColumnNameWithoutAlias() const; @@ -52,8 +51,6 @@ public: throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::LOGICAL_ERROR); } - virtual void appendColumnName(WriteBuffer & ostr, const Settings &) const { appendColumnName(ostr); } - virtual void appendColumnNameWithoutAlias(WriteBuffer &) const { throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::LOGICAL_ERROR); @@ -61,7 +58,6 @@ public: /** Get the alias, if any, or the canonical name of the column, if it is not. */ virtual String getAliasOrColumnName() const { return getColumnName(); } - virtual String getAliasOrColumnName(const Settings & settings) const { return getColumnName(settings); } /** Get the alias, if any, or an empty string if it does not exist, or if the element does not support aliases. */ virtual String tryGetAlias() const { return String(); } diff --git a/tests/queries/0_stateless/02002_system_table_with_tuple.reference b/tests/queries/0_stateless/02002_system_table_with_tuple.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02002_system_table_with_tuple.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02002_system_table_with_tuple.sql b/tests/queries/0_stateless/02002_system_table_with_tuple.sql new file mode 100644 index 00000000000..42796e98110 --- /dev/null +++ b/tests/queries/0_stateless/02002_system_table_with_tuple.sql @@ -0,0 +1,2 @@ +SELECT count() > 0 FROM system.tables WHERE database IN ('system', 'system') +SETTINGS legacy_column_name_of_tuple_literal = 1; From bb8c6813a1c5e1225fda84a688575adf86a42297 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 4 Aug 2021 10:15:50 +0000 Subject: [PATCH 026/147] Update --- .../functions/other-functions.md | 114 +----------------- 1 file changed, 3 insertions(+), 111 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 63892bc712c..1826318217b 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2148,48 +2148,12 @@ Short description. ``` -Alias: ``. (Optional) - -More text (Optional). - -**Arguments** (Optional) - -- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). -- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). - -**Parameters** (Optional, only for parametric aggregate functions) - -- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). - -**Returned value(s)** +**Returned value** - Returned values list. Type: [Type name](relative/path/to/type/dscr.md#type). -**Example** - -The example must show usage and/or a use cases. The following text contains recommended parts of an example. - -Input table (Optional): - -``` text -``` - -Query: - -``` sql -``` - -Result: - -``` text -``` - -**See Also** (Optional) - -- [link](#) - ## enabledProfiles {#enabled-profiles} Short description. @@ -2200,48 +2164,12 @@ Short description. ``` -Alias: ``. (Optional) - -More text (Optional). - -**Arguments** (Optional) - -- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). -- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). - -**Parameters** (Optional, only for parametric aggregate functions) - -- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). - -**Returned value(s)** +**Returned value** - Returned values list. Type: [Type name](relative/path/to/type/dscr.md#type). -**Example** - -The example must show usage and/or a use cases. The following text contains recommended parts of an example. - -Input table (Optional): - -``` text -``` - -Query: - -``` sql -``` - -Result: - -``` text -``` - -**See Also** (Optional) - -- [link](#) - ## defaultProfiles {#default-profiles} Short description. @@ -2252,44 +2180,8 @@ Short description. ``` -Alias: ``. (Optional) - -More text (Optional). - -**Arguments** (Optional) - -- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). -- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). - -**Parameters** (Optional, only for parametric aggregate functions) - -- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). - -**Returned value(s)** +**Returned value** - Returned values list. Type: [Type name](relative/path/to/type/dscr.md#type). - -**Example** - -The example must show usage and/or a use cases. The following text contains recommended parts of an example. - -Input table (Optional): - -``` text -``` - -Query: - -``` sql -``` - -Result: - -``` text -``` - -**See Also** (Optional) - -- [link](#) \ No newline at end of file From 5c39ae981c9998209e882196a6c732bb78b09951 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 4 Aug 2021 11:26:24 +0000 Subject: [PATCH 027/147] Add desc for currentProfiles, enabledProfiles, defaultProfiles func --- .../en/engines/database-engines/replicated.md | 5 ---- .../functions/other-functions.md | 26 ++++++++++--------- .../ru/engines/database-engines/replicated.md | 5 ---- 3 files changed, 14 insertions(+), 22 deletions(-) diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index 1ac4d34b0de..ed8406e6a86 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -1,8 +1,3 @@ ---- -toc_priority: 40 -toc_title: Replicated ---- - # [experimental] Replicated {#replicated} The engine is based on the [Atomic](../../engines/database-engines/atomic.md) engine. It supports replication of metadata via DDL log being written to ZooKeeper and executed on all of the replicas for a given database. diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 1826318217b..fca5afaced3 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2140,48 +2140,50 @@ Result: ## currentProfiles {#current-profiles} -Short description. +Returns list of the current [setting profiles](../../operations/access-rights/#settings-profiles-management) for current user. The command [SET PROFILE](../../sql-reference/statements/set.md#set-statement#query-set) could be used to change the current setting profile. **Syntax** ``` sql - +currentProfiles() ``` **Returned value** -- Returned values list. +- List of the current setting profiles. -Type: [Type name](relative/path/to/type/dscr.md#type). +Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). ## enabledProfiles {#enabled-profiles} -Short description. + Returns setting profiles are assigned to the user both explicitly and implicitly. Implicitly assigned profiles include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the section `default_profile` in the main configuration file). + + If some profile occurs multiple times (with some other settings in between), the latest occurrence overrides all the previous ones. **Syntax** ``` sql - +enabledProfiles() ``` **Returned value** -- Returned values list. +- List of the enabled setting profiles. -Type: [Type name](relative/path/to/type/dscr.md#type). +Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). ## defaultProfiles {#default-profiles} -Short description. +Returns the default setting profiles, i.e. the settings which should be applied on the user's login. **Syntax** ``` sql - +defaultProfiles() ``` **Returned value** -- Returned values list. +- List of the default setting profiles. -Type: [Type name](relative/path/to/type/dscr.md#type). +Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). diff --git a/docs/ru/engines/database-engines/replicated.md b/docs/ru/engines/database-engines/replicated.md index c70d1aa25c0..c0e3c6af89c 100644 --- a/docs/ru/engines/database-engines/replicated.md +++ b/docs/ru/engines/database-engines/replicated.md @@ -1,8 +1,3 @@ ---- -toc_priority: 40 -toc_title: Replicated ---- - # [экспериментальный] Replicated {#replicated} Движок основан на движке [Atomic](../../engines/database-engines/atomic.md). Он поддерживает репликацию метаданных через журнал DDL, записываемый в ZooKeeper и выполняемый на всех репликах для данной базы данных. From 5bd665bfba463d97238c55d7e4e710187163b3e5 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 4 Aug 2021 11:47:27 +0000 Subject: [PATCH 028/147] Add ru docs --- .../functions/other-functions.md | 4 +- .../ru/engines/database-engines/replicated.md | 1 + .../functions/other-functions.md | 49 +++++++++++++++++++ 3 files changed, 52 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index fca5afaced3..7866dfc9c84 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2140,7 +2140,7 @@ Result: ## currentProfiles {#current-profiles} -Returns list of the current [setting profiles](../../operations/access-rights/#settings-profiles-management) for current user. The command [SET PROFILE](../../sql-reference/statements/set.md#set-statement#query-set) could be used to change the current setting profile. +Returns list of the current [setting profiles](../../operations/access-rights.md#settings-profiles-management) for current user. The command [SET PROFILE](../../sql-reference/statements/set.md#set-statement#query-set) could be used to change the current setting profile. **Syntax** @@ -2156,7 +2156,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## enabledProfiles {#enabled-profiles} - Returns setting profiles are assigned to the user both explicitly and implicitly. Implicitly assigned profiles include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the section `default_profile` in the main configuration file). + Returns setting profiles are assigned to the user both explicitly and implicitly. Implicitly assigned profiles to include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the section `default_profile` in the main configuration file). If some profile occurs multiple times (with some other settings in between), the latest occurrence overrides all the previous ones. diff --git a/docs/ru/engines/database-engines/replicated.md b/docs/ru/engines/database-engines/replicated.md index c0e3c6af89c..f1d5755647a 100644 --- a/docs/ru/engines/database-engines/replicated.md +++ b/docs/ru/engines/database-engines/replicated.md @@ -1,3 +1,4 @@ + # [экспериментальный] Replicated {#replicated} Движок основан на движке [Atomic](../../engines/database-engines/atomic.md). Он поддерживает репликацию метаданных через журнал DDL, записываемый в ZooKeeper и выполняемый на всех репликах для данной базы данных. diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index a07bd19faa1..f6f7578a187 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2088,3 +2088,52 @@ SELECT tcpPort(); - [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) +## currentProfiles {#current-profiles} + +Возвращает список [профилей настроек](../../operations/access-rights.md#settings-profiles-management) для текущего пользователя. Для изменения текущего профиля настроек может быть использована команда [SET PROFILE](../../sql-reference/statements/set.md#set-statement#query-set). + +**Синтаксис** + +``` sql +currentProfiles() +``` + +**Возвращаемое значение** + +- Список профилей настроек для текущего пользователя. + +Тип: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). + +## enabledProfiles {#enabled-profiles} + +Возвращает профили настроек, назначенные пользователю как явно, так и неявно. Неявно назначенные профили включают родительские профили других назначенных профилей, профили, назначенные с помощью предоставленных ролей, профили, назначенные с помощью собственных настроек, и основной профиль по умолчанию (см. секцию `default_profile` в основном конфигурационном файле). + + Если какой-либо профиль встречается несколько раз (с некоторыми другими настройками между ними), последнее событие перезаписывает все предыдущие. + +**Синтаксис** + +``` sql +enabledProfiles() +``` + +**Возвращаемое значение** + +- Список доступных профилей для текущего пользователя. + +Тип: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). + +## defaultProfiles {#default-profiles} + +Возвращает профили настроек по умолчанию, т. е. профили, которые должны быть применены при входе пользователя в систему. + +**Синтаксис** + +``` sql +defaultProfiles() +``` + +**Возвращаемое значение** + +- Список профилей по умолчанию. + +ТИп: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). \ No newline at end of file From 61ddc53e261c27e5d738d2a0be3fd9b04c72953b Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 4 Aug 2021 11:51:22 +0000 Subject: [PATCH 029/147] Fixed links --- docs/en/sql-reference/functions/other-functions.md | 2 +- docs/ru/sql-reference/functions/other-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 7866dfc9c84..dcd19cbf9b4 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2140,7 +2140,7 @@ Result: ## currentProfiles {#current-profiles} -Returns list of the current [setting profiles](../../operations/access-rights.md#settings-profiles-management) for current user. The command [SET PROFILE](../../sql-reference/statements/set.md#set-statement#query-set) could be used to change the current setting profile. +Returns list of the current [setting profiles](../../operations/access-rights.md#settings-profiles-management) for current user. The command [SET PROFILE](../../sql-reference/statements/set.md#query-set) could be used to change the current setting profile. **Syntax** diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index f6f7578a187..4df98f71458 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2090,7 +2090,7 @@ SELECT tcpPort(); ## currentProfiles {#current-profiles} -Возвращает список [профилей настроек](../../operations/access-rights.md#settings-profiles-management) для текущего пользователя. Для изменения текущего профиля настроек может быть использована команда [SET PROFILE](../../sql-reference/statements/set.md#set-statement#query-set). +Возвращает список [профилей настроек](../../operations/access-rights.md#settings-profiles-management) для текущего пользователя. Для изменения текущего профиля настроек может быть использована команда [SET PROFILE](../../sql-reference/statements/set.md#query-set). **Синтаксис** From d8ce46ea747a62c65c3c589d88c1d2f576fb3be5 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 4 Aug 2021 17:20:35 +0500 Subject: [PATCH 030/147] Show more details in testflows checks --- .../tests/array_tuple_map.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py index 04150d945f6..698e81ca1ac 100644 --- a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py +++ b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py @@ -1,5 +1,6 @@ import uuid +from testflows.asserts import error from extended_precision_data_types.requirements import * from extended_precision_data_types.common import * @@ -410,10 +411,11 @@ def map_func(self, data_type, node=None): sql = (f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3],", f"[{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)") - exitcode, message = 0, None + r = node.query(sql, exitcode=None, message=None) if data_type.startswith("Decimal"): - exitcode, message = 44, "Exception:" - node.query(sql, exitcode=exitcode, message=message) + assert r.exitcode == 44, error(r.output) + else: + assert r.exitcode == 0, error(f"sql: {sql}\noutput: {r.output}") execute_query(f"SELECT * FROM {table_name} ORDER BY a ASC") From 9b9c6e02d162969296371289391a8e0394e6276c Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 4 Aug 2021 12:34:50 +0000 Subject: [PATCH 031/147] Applied suggestion from code review --- docs/en/sql-reference/functions/other-functions.md | 6 +++--- docs/ru/sql-reference/functions/other-functions.md | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index dcd19cbf9b4..74eeb76577b 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2140,7 +2140,7 @@ Result: ## currentProfiles {#current-profiles} -Returns list of the current [setting profiles](../../operations/access-rights.md#settings-profiles-management) for current user. The command [SET PROFILE](../../sql-reference/statements/set.md#query-set) could be used to change the current setting profile. +Returns list of the current [setting profiles](../../operations/access-rights.md#settings-profiles-management) for current user. The command [SET PROFILE](../../sql-reference/statements/set.md#query-set) could be used to change the current setting profile. If the command `SET PROFILE` hasn't been called the function returns the profiles specified at the current user's definition (see [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement), `SETTINGS` section). **Syntax** @@ -2156,7 +2156,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## enabledProfiles {#enabled-profiles} - Returns setting profiles are assigned to the user both explicitly and implicitly. Implicitly assigned profiles to include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the section `default_profile` in the main configuration file). + Returns setting profiles are assigned to the user both explicitly and implicitly. Explicitly assigned profiles are the same profiles as returned by the function [currentProfiles](#current-profiles). Implicitly assigned profiles includes parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the section `default_profile` in the main configuration file). If some profile occurs multiple times (with some other settings in between), the latest occurrence overrides all the previous ones. @@ -2174,7 +2174,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## defaultProfiles {#default-profiles} -Returns the default setting profiles, i.e. the settings which should be applied on the user's login. +Returns all the profiles specified at the current user's definition (see [CREATE USER]()). **Syntax** diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 4df98f71458..4e3e5a638dc 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2090,7 +2090,7 @@ SELECT tcpPort(); ## currentProfiles {#current-profiles} -Возвращает список [профилей настроек](../../operations/access-rights.md#settings-profiles-management) для текущего пользователя. Для изменения текущего профиля настроек может быть использована команда [SET PROFILE](../../sql-reference/statements/set.md#query-set). +Возвращает список [профилей настроек](../../operations/access-rights.md#settings-profiles-management) для текущего пользователя. Для изменения текущего профиля настроек может быть использована команда [SET PROFILE](../../sql-reference/statements/set.md#query-set). Если команда `SET PROFILE` не использовалась, функция возвращает профили, указанные при объявлении текущего пользователя (см. [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)). **Синтаксис** @@ -2106,7 +2106,7 @@ currentProfiles() ## enabledProfiles {#enabled-profiles} -Возвращает профили настроек, назначенные пользователю как явно, так и неявно. Неявно назначенные профили включают родительские профили других назначенных профилей, профили, назначенные с помощью предоставленных ролей, профили, назначенные с помощью собственных настроек, и основной профиль по умолчанию (см. секцию `default_profile` в основном конфигурационном файле). +Возвращает профили настроек, назначенные пользователю как явно, так и неявно. Явно назначенные профили — это те же профили, которые возвращает функция [currentProfiles](#current-profiles). Неявно назначенные профили включают родительские профили других назначенных профилей, профили, назначенные с помощью предоставленных ролей, профили, назначенные с помощью собственных настроек, и основной профиль по умолчанию (см. секцию `default_profile` в основном конфигурационном файле). Если какой-либо профиль встречается несколько раз (с некоторыми другими настройками между ними), последнее событие перезаписывает все предыдущие. @@ -2124,7 +2124,7 @@ enabledProfiles() ## defaultProfiles {#default-profiles} -Возвращает профили настроек по умолчанию, т. е. профили, которые должны быть применены при входе пользователя в систему. +Возвращает все профили, указанные при объявлении текущего пользователя (см. [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement), секция `SETTINGS`) **Синтаксис** From b8558a1716a9c98b9bfdd1fe3e8259f815d5fd30 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 4 Aug 2021 16:44:39 +0300 Subject: [PATCH 032/147] Fix uninitialized memory in functions multiSearch* with empty array --- src/Functions/FunctionsMultiStringPosition.h | 2 +- src/Functions/MultiSearchFirstIndexImpl.h | 2 +- src/Functions/MultiSearchFirstPositionImpl.h | 2 +- src/Functions/MultiSearchImpl.h | 2 +- .../00233_position_function_family.reference | 16 ++++++++++++++++ .../00233_position_function_family.sql | 17 ++++++++++++++++- 6 files changed, 36 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionsMultiStringPosition.h b/src/Functions/FunctionsMultiStringPosition.h index f36f7639ccd..74496484c92 100644 --- a/src/Functions/FunctionsMultiStringPosition.h +++ b/src/Functions/FunctionsMultiStringPosition.h @@ -109,7 +109,7 @@ public: auto & vec_res = col_res->getData(); auto & offsets_res = col_offsets->getData(); - vec_res.resize(column_haystack_size * refs.size()); + vec_res.resize_fill(column_haystack_size * refs.size()); if (col_haystack_vector) Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res); diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index bb0bb57b9db..146cb551bb3 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -26,7 +26,7 @@ struct MultiSearchFirstIndexImpl { auto searcher = Impl::createMultiSearcherInBigHaystack(needles); const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); + res.resize_fill(haystack_string_size); size_t iteration = 0; while (searcher.hasMoreToSearch()) { diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index 4743518e61e..aee4da43958 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -30,7 +30,7 @@ struct MultiSearchFirstPositionImpl }; auto searcher = Impl::createMultiSearcherInBigHaystack(needles); const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); + res.resize_fill(haystack_string_size); size_t iteration = 0; while (searcher.hasMoreToSearch()) { diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index 5b881e2fd55..4580eda27e4 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -26,7 +26,7 @@ struct MultiSearchImpl { auto searcher = Impl::createMultiSearcherInBigHaystack(needles); const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); + res.resize_fill(haystack_string_size); size_t iteration = 0; while (searcher.hasMoreToSearch()) { diff --git a/tests/queries/0_stateless/00233_position_function_family.reference b/tests/queries/0_stateless/00233_position_function_family.reference index 1523094261f..f6cacbc7027 100644 --- a/tests/queries/0_stateless/00233_position_function_family.reference +++ b/tests/queries/0_stateless/00233_position_function_family.reference @@ -23796,3 +23796,19 @@ 1 1 1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/00233_position_function_family.sql b/tests/queries/0_stateless/00233_position_function_family.sql index 8e443f5cd10..36f8a5535d9 100644 --- a/tests/queries/0_stateless/00233_position_function_family.sql +++ b/tests/queries/0_stateless/00233_position_function_family.sql @@ -483,6 +483,22 @@ select 1 = position('abab', materialize('ab')); select 1 = position('abababababababababababab', materialize('abab')); select 1 = position('abababababababababababab', materialize('abababababababababa')); +select 0 = multiSearchAny('\0', CAST([], 'Array(String)')); +select 0 = multiSearchAnyCaseInsensitive('\0', CAST([], 'Array(String)')); +select 0 = multiSearchAnyCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); +select 0 = multiSearchAnyUTF8('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstIndex('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstIndexCaseInsensitive('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstIndexUTF8('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstPosition('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstPositionCaseInsensitive('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstPositionCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstPositionUTF8('\0', CAST([], 'Array(String)')); +select [] = multiSearchAllPositions('\0', CAST([], 'Array(String)')); +select [] = multiSearchAllPositionsCaseInsensitive('\0', CAST([], 'Array(String)')); +select [] = multiSearchAllPositionsCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); +select [] = multiSearchAllPositionsUTF8('\0', CAST([], 'Array(String)')); select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['b']); select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bc']); @@ -1177,7 +1193,6 @@ select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШНвпкфЗвгДж select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ФРХгаСлчЧОцкШгзмКЗшФфББвЧ'), ['кзШфФб', 'ГАслЧЧОцкшг', 'ФфббВЧ', 'ЦкШ', '', 'АслчЧОЦКШгзМкЗШффбБвч', 'РХгаслЧчОЦКШГз', 'РхгаслчЧОцКШгзМкзшФфБбВ', 'Шг', 'Ф', 'ХГАслчЧоцКШГзМкзш', 'ШгЗмКЗшфФб']) from system.numbers limit 10; select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЧдйШкхОлалщНйбССХКаФзОМрКЕЙР'), ['бссХкафзОм', 'ХОЛаЛщнйБссХкаФз', 'лаЛщнйБсСХ', 'ЩнЙбСсхКаФЗО', 'йБСсХКАФЗОмР', 'йшкХолаЛЩНйбсСхК', 'С', '', 'ЙшкхОлалщНЙБсСхКаФзом', 'Йр', 'щнЙБссхКАфзоМрК', 'рКе']) from system.numbers limit 10; - select 1 = multiSearchFirstIndex(materialize('alhpvldsiwsydwhfdasqju'), ['sydwh', 'dwh', 'dwhfdasqj', 'w', 'briozrtpq', 'fdasq', 'lnuvpuxdhhuxjbolw', 'vldsiws', 'dasqju', 'uancllygwoifwnnp', 'wfxputfnen', 'hzaclvjumecnmweungz']) from system.numbers limit 10; select 0 = multiSearchFirstIndex(materialize('kcwchxxyujbhrxkxgnomg'), ['jmvqipszutxfnhdfaxqwoxcz', 'nrgzkbsakdtdiiyphozjoauyughyvlz', 'qbszx', 'sllthykcnttqecpequommemygee', 'bvsbdiufrrrjxaxzxgbd', 'hdkpcmpdyjildw', 'frxkyukiywngfcxfzwkcun', 'dmvxf', 'esamivybor', 'eoggdynqwlnlxr']) from system.numbers limit 10; select 0 = multiSearchFirstIndex(materialize('zggbeilrfpkleafjjldgyfgn'), ['rpypxkpgvljhqthneremvabcd', 'qchzlnsctuwkdxqcrjgihvtfxhqxfqsxm', 'vtozkivjyqcqetmqenuihq', 'fixcvjyzbzejmwdivjf', 'lydoolvnuuamwlnzbyuuwpqqjaxf', 'elkodwthxqpcybwezm', 'wpiju', 'wdzuuwumlqfvga', 'iokphkai', 'wkbwdstplhivjyk', 'wxfbhfturuqoymwklohawgwltptytc', 'jehprkzofqvurepbvuwdqj']) from system.numbers limit 10; From 61016da2b11665b8dda2cf7af86918a676e2dbb0 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 29 Jul 2021 11:25:22 +0300 Subject: [PATCH 033/147] Set allow_remote_fs_zero_copy_replication to true by default --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- .../ReplicatedMergeTreeMergeStrategyPicker.cpp | 11 ++++++++--- .../configs/config.d/storage_conf.xml | 1 - .../configs/config.d/storage_conf.xml | 1 - .../configs/config.d/s3.xml | 1 - .../01700_system_zookeeper_path_in.reference | 2 ++ 6 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index d018059c248..531091bb7f9 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -124,7 +124,7 @@ struct Settings; M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ M(String, storage_policy, "default", "Name of storage disk policy", 0) \ M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ - M(Bool, allow_remote_fs_zero_copy_replication, false, "Allow Zero-copy replication over remote fs", 0) \ + M(Bool, allow_remote_fs_zero_copy_replication, true, "Allow Zero-copy replication over remote fs", 0) \ M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm", 0) \ M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp index 13e05681fd9..2d611dc0c97 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp @@ -144,9 +144,14 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() if (current_replica_index_tmp < 0 || active_replicas_tmp.size() < 2) { - LOG_WARNING(storage.log, "Can't find current replica in the active replicas list, or too few active replicas to use execute_merges_on_single_replica_time_threshold!"); - /// we can reset the settings w/o lock (it's atomic) - execute_merges_on_single_replica_time_threshold = 0; + if (execute_merges_on_single_replica_time_threshold > 0) + { + LOG_WARNING(storage.log, "Can't find current replica in the active replicas list, or too few active replicas to use execute_merges_on_single_replica_time_threshold!"); + /// we can reset the settings w/o lock (it's atomic) + execute_merges_on_single_replica_time_threshold = 0; + } + /// default value of remote_fs_execute_merges_on_single_replica_time_threshold is not 0 + /// so we write no warning in log here remote_fs_execute_merges_on_single_replica_time_threshold = 0; return; } diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml index 46a11a8fe16..1405c7274ca 100644 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml @@ -59,7 +59,6 @@ 1024000 1 - 1 diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml index 0cf9191c4af..20b750ffff3 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml @@ -21,7 +21,6 @@ 0 - 1 diff --git a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml index 89c97aa3360..e3d3d3543fa 100644 --- a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml +++ b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml @@ -66,7 +66,6 @@ 1024 1 - 1 diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference index e491dd9e091..dcee18b33e0 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference @@ -12,3 +12,5 @@ blocks failed_parts last_part parallel +shared +shared From 8e347bab99d0bf1d46d4a03dfab8250379ec4c0b Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 4 Aug 2021 20:21:37 +0500 Subject: [PATCH 034/147] Fix error in testflows related with mapPopulateSeries --- .../tests/array_tuple_map.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py index 698e81ca1ac..c39574ba75e 100644 --- a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py +++ b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py @@ -1,6 +1,5 @@ import uuid -from testflows.asserts import error from extended_precision_data_types.requirements import * from extended_precision_data_types.common import * @@ -408,14 +407,13 @@ def map_func(self, data_type, node=None): table(name = table_name, data_type = f'Tuple(Array({data_type}), Array({data_type}))') with When("I insert the output into a table"): - sql = (f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3],", + sql = (f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3]," f"[{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)") - r = node.query(sql, exitcode=None, message=None) + exitcode, message = 0, None if data_type.startswith("Decimal"): - assert r.exitcode == 44, error(r.output) - else: - assert r.exitcode == 0, error(f"sql: {sql}\noutput: {r.output}") + exitcode, message = 44, "Exception:" + node.query(sql, exitcode=exitcode, message=message) execute_query(f"SELECT * FROM {table_name} ORDER BY a ASC") From 9a9aebc6443f5fd37c672386cd6537273f6c23e1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 4 Aug 2021 20:38:10 +0300 Subject: [PATCH 035/147] fix tests --- src/Interpreters/TreeRewriter.cpp | 2 +- .../queries/0_stateless/02002_system_table_with_tuple.sh | 9 +++++++++ .../0_stateless/02002_system_table_with_tuple.sql | 2 -- 3 files changed, 10 insertions(+), 3 deletions(-) create mode 100755 tests/queries/0_stateless/02002_system_table_with_tuple.sh delete mode 100644 tests/queries/0_stateless/02002_system_table_with_tuple.sql diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index a2b64d26d42..9294cca7bb4 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -614,7 +614,7 @@ class MarkTupleLiteralsAsLegacyData public: using TypeToVisit = ASTLiteral; - void visit(ASTLiteral & literal, ASTPtr &) + static void visit(ASTLiteral & literal, ASTPtr &) { if (literal.value.getType() == Field::Types::Tuple) literal.use_legacy_column_name_of_tuple = true; diff --git a/tests/queries/0_stateless/02002_system_table_with_tuple.sh b/tests/queries/0_stateless/02002_system_table_with_tuple.sh new file mode 100755 index 00000000000..2abf5b8b525 --- /dev/null +++ b/tests/queries/0_stateless/02002_system_table_with_tuple.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "SELECT count() > 0 FROM system.tables \ + WHERE database IN ('system', '$CLICKHOUSE_DATABASE') \ + SETTINGS legacy_column_name_of_tuple_literal = 1" diff --git a/tests/queries/0_stateless/02002_system_table_with_tuple.sql b/tests/queries/0_stateless/02002_system_table_with_tuple.sql deleted file mode 100644 index 42796e98110..00000000000 --- a/tests/queries/0_stateless/02002_system_table_with_tuple.sql +++ /dev/null @@ -1,2 +0,0 @@ -SELECT count() > 0 FROM system.tables WHERE database IN ('system', 'system') -SETTINGS legacy_column_name_of_tuple_literal = 1; From d4fbd7107ee9e5a10c2b71dbfb3d802832b6016e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 4 Aug 2021 21:14:59 +0300 Subject: [PATCH 036/147] try make flags more readable --- src/Interpreters/InterpreterRenameQuery.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 926043ee819..bfd03028c91 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -72,22 +72,25 @@ BlockIO InterpreterRenameQuery::execute() BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions, TableGuards & ddl_guards) { + assert(!rename.rename_if_cannot_exchange || descriptions.size() == 1); + assert(!rename.rename_if_cannot_exchange || rename.exchange); auto & database_catalog = DatabaseCatalog::instance(); for (const auto & elem : descriptions) { - bool exchange_tables = rename.exchange; - if (exchange_tables) + bool exchange_tables; + if (rename.exchange) { - bool allow_rename_instead_of_exchange = descriptions.size() == 1 && rename.rename_if_cannot_exchange; - if (allow_rename_instead_of_exchange && !database_catalog.isTableExist(StorageID(elem.to_database_name, elem.to_table_name), getContext())) - { - exchange_tables = false; - renamed_instead_of_exchange = true; - } + exchange_tables = true; + } + else if (rename.rename_if_cannot_exchange) + { + exchange_tables = database_catalog.isTableExist(StorageID(elem.to_database_name, elem.to_table_name), getContext()); + renamed_instead_of_exchange = !exchange_tables; } else { + exchange_tables = false; database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name), getContext()); } From 2ed7e02079cbd2aa939e83a313e8302540baab70 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Thu, 5 Aug 2021 11:11:41 +0500 Subject: [PATCH 037/147] Fix snapshots for mapPopulateSeries function --- .../snapshots/common.py.tests.snapshot | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot b/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot index 18b58b0cfdc..e0414393111 100644 --- a/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot +++ b/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot @@ -1045,6 +1045,7 @@ a mapPopulateSeries_with_Int128_on_a_table = r""" a +([1,2,3,4,5],[1,2,3,0,0]) """ mapContains_with_Int128_on_a_table = r""" @@ -1575,6 +1576,7 @@ a mapPopulateSeries_with_Int256_on_a_table = r""" a +([1,2,3,4,5],[1,2,3,0,0]) """ mapContains_with_Int256_on_a_table = r""" @@ -2105,6 +2107,7 @@ a mapPopulateSeries_with_UInt128_on_a_table = r""" a +([1,2,3,4,5],[1,2,3,0,0]) """ mapContains_with_UInt128_on_a_table = r""" @@ -2635,6 +2638,7 @@ a mapPopulateSeries_with_UInt256_on_a_table = r""" a +([1,2,3,4,5],[1,2,3,0,0]) """ mapContains_with_UInt256_on_a_table = r""" From a60640af2f5510bb65c44c78f4f47e9a775b33fb Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Thu, 5 Aug 2021 09:25:52 +0300 Subject: [PATCH 038/147] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 74eeb76577b..3beabf4852c 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2140,7 +2140,9 @@ Result: ## currentProfiles {#current-profiles} -Returns list of the current [setting profiles](../../operations/access-rights.md#settings-profiles-management) for current user. The command [SET PROFILE](../../sql-reference/statements/set.md#query-set) could be used to change the current setting profile. If the command `SET PROFILE` hasn't been called the function returns the profiles specified at the current user's definition (see [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement), `SETTINGS` section). +Returns a list of the current [settings profiles](../../operations/access-rights.md#settings-profiles-management) for the current user. + +The settings profile can be specified at the user's definition (see [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement), `PROFILE` clause) and can be changed with the [SET profile](../../sql-reference/statements/set.md#query-set) statements lately. **Syntax** From 44b71efb72f94f51e57a340e35a3e9b6e72772ea Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Thu, 5 Aug 2021 09:27:48 +0300 Subject: [PATCH 039/147] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 3beabf4852c..a70c118cd86 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2152,7 +2152,7 @@ currentProfiles() **Returned value** -- List of the current setting profiles. +- List of the current user settings profiles. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). From 4f14ccf3e612d2ae1168c62abea2d6dcabe3610a Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Thu, 5 Aug 2021 09:28:52 +0300 Subject: [PATCH 040/147] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index a70c118cd86..fbdca692b73 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2158,7 +2158,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## enabledProfiles {#enabled-profiles} - Returns setting profiles are assigned to the user both explicitly and implicitly. Explicitly assigned profiles are the same profiles as returned by the function [currentProfiles](#current-profiles). Implicitly assigned profiles includes parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the section `default_profile` in the main configuration file). + Returns settings profiles, assigned to the current user both explicitly and implicitly. Explicitly assigned profiles are the same as returned by the [currentProfiles](#current-profiles) function. Implicitly assigned profiles include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the `default_profile` section in the main server configuration file). If some profile occurs multiple times (with some other settings in between), the latest occurrence overrides all the previous ones. From ca89cf656d18f093a9874122aab1fd9257ef5a5e Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Thu, 5 Aug 2021 09:28:58 +0300 Subject: [PATCH 041/147] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index fbdca692b73..12c26ccff5e 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2170,7 +2170,7 @@ enabledProfiles() **Returned value** -- List of the enabled setting profiles. +- List of the enabled settings profiles. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). From 1331537064f8de384b3db8d37aec233809dca6fe Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Thu, 5 Aug 2021 09:29:12 +0300 Subject: [PATCH 042/147] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 12c26ccff5e..732c403679d 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2176,7 +2176,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## defaultProfiles {#default-profiles} -Returns all the profiles specified at the current user's definition (see [CREATE USER]()). +Returns all the profiles specified at the current user's definition (see [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement) statement). **Syntax** From 71de6f3e03d265e2d509f9fd15a8e39d9d21f26f Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Thu, 5 Aug 2021 09:29:17 +0300 Subject: [PATCH 043/147] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 732c403679d..122d701e856 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2186,6 +2186,6 @@ defaultProfiles() **Returned value** -- List of the default setting profiles. +- List of the default settings profiles. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). From e232fea872e76a8682a2cd416db69df3362f7ad5 Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Thu, 5 Aug 2021 09:29:29 +0300 Subject: [PATCH 044/147] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 4e3e5a638dc..3ecac367dbf 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2090,7 +2090,9 @@ SELECT tcpPort(); ## currentProfiles {#current-profiles} -Возвращает список [профилей настроек](../../operations/access-rights.md#settings-profiles-management) для текущего пользователя. Для изменения текущего профиля настроек может быть использована команда [SET PROFILE](../../sql-reference/statements/set.md#query-set). Если команда `SET PROFILE` не использовалась, функция возвращает профили, указанные при объявлении текущего пользователя (см. [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)). +Возвращает список [профилей настроек](../../operations/access-rights.md#settings-profiles-management) для текущего пользователя. + +Профиль настроек может быть указан при создании пользователя (см. [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)). Позднее для изменения настроек могут применяться другие профили с помощью выражения [SET profile](../../sql-reference/statements/set.md#query-set). **Синтаксис** @@ -2136,4 +2138,4 @@ defaultProfiles() - Список профилей по умолчанию. -ТИп: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). \ No newline at end of file +ТИп: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). From f6a64e9ea484bc906b6a2e4478198b3e0c19e3cc Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Thu, 5 Aug 2021 09:29:40 +0300 Subject: [PATCH 045/147] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 3ecac367dbf..044a8929927 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2108,7 +2108,7 @@ currentProfiles() ## enabledProfiles {#enabled-profiles} -Возвращает профили настроек, назначенные пользователю как явно, так и неявно. Явно назначенные профили — это те же профили, которые возвращает функция [currentProfiles](#current-profiles). Неявно назначенные профили включают родительские профили других назначенных профилей, профили, назначенные с помощью предоставленных ролей, профили, назначенные с помощью собственных настроек, и основной профиль по умолчанию (см. секцию `default_profile` в основном конфигурационном файле). +Возвращает профили настроек, назначенные пользователю как явно, так и неявно. Явно назначенные профили — это те же профили, которые возвращает функция [currentProfiles](#current-profiles). Неявно назначенные профили включают родительские профили других назначенных профилей; профили, назначенные с помощью предоставленных ролей; профили, назначенные с помощью собственных настроек; основной профиль по умолчанию (см. секцию `default_profile` в основном конфигурационном файле сервера). Если какой-либо профиль встречается несколько раз (с некоторыми другими настройками между ними), последнее событие перезаписывает все предыдущие. From 617cdd5d4d6082b15e15b8e6cb63f8a3c2cd2db6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Aug 2021 10:11:25 +0300 Subject: [PATCH 046/147] check-style: add a check for ReplicatedMergeTree in *.sql --- utils/check-style/check-style | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 551368e5fd8..e541d2cbfb1 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -132,14 +132,8 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.sql) - # NOTE: *.sql is not supported because it is not possible right now, because: - # - ReplicatedMergeTree supports only ASTLiteral for zookeeper path - # (and adding support of other nodes, with evaluating them are not that easy, due to zk_prefix is "optional") - # - Hence concat(currentDatabase(), 'foo') - # - Also params cannot be used, because the are wrapped with CAST() - # - # But hopefully they will not be a problem - # (since they do not do any "stressing" and overlap probability should be lower). + test_case_zk_prefix="{[ ]*database[ ]*}" + grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.py) # Right now there is not such tests anyway From 6c61d3731e90e91b75858a19c7d285c3d75da3c1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Aug 2021 10:24:23 +0300 Subject: [PATCH 047/147] check-style: cover *.j2 too --- utils/check-style/check-style | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index e541d2cbfb1..a096076a3f0 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -89,7 +89,7 @@ find $ROOT_PATH/tests/queries -iname '*fail*' | # Queries to system.query_log/system.query_thread_log should have current_database = currentDatabase() condition # NOTE: it is not that accuate, but at least something. tests_with_query_log=( $( - find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' | + find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | xargs grep --with-filename -e system.query_log -e system.query_thread_log | cut -d: -f1 | sort -u ) ) @@ -102,7 +102,7 @@ done # Queries to system.tables/system.parts/system.detached_parts/system.parts_columns/system.columns should have database = currentDatabase() condition # NOTE: it is not that accuate, but at least something. tests_with_database_column=( $( - find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' | + find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | grep -v -x -e $ROOT_PATH/tests/queries/query_test.py | xargs grep --with-filename -e system.tables -e system.parts -e system.detached_parts -e system.parts_columns -e system.columns | cut -d: -f1 | sort -u @@ -121,7 +121,7 @@ done # Queries with ReplicatedMergeTree # NOTE: it is not that accuate, but at least something. tests_with_replicated_merge_tree=( $( - find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' | + find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | xargs grep --with-filename -e ReplicatedMergeTree | cut -d: -f1 | sort -u ) ) @@ -131,7 +131,7 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do test_case_zk_prefix="\$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; - *.sql) + *.sql|*.sql.j2) test_case_zk_prefix="{[ ]*database[ ]*}" grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; @@ -180,7 +180,7 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' | while read file; do [[ $(head -n1 $file) != '#pragma once' ]] && echo "File $file must have '#pragma once' in first line"; done # Check for executable bit on non-executable files -find $ROOT_PATH/{src,base,programs,utils,tests,docs,website,cmake} '(' -name '*.cpp' -or -name '*.h' -or -name '*.sql' -or -name '*.xml' -or -name '*.reference' -or -name '*.txt' -or -name '*.md' ')' -and -executable | grep -P '.' && echo "These files should not be executable." +find $ROOT_PATH/{src,base,programs,utils,tests,docs,website,cmake} '(' -name '*.cpp' -or -name '*.h' -or -name '*.sql' -or -name '*.j2' -or -name '*.xml' -or -name '*.reference' -or -name '*.txt' -or -name '*.md' ')' -and -executable | grep -P '.' && echo "These files should not be executable." # Check for BOM find $ROOT_PATH/{src,base,programs,utils,tests,docs,website,cmake} -name '*.md' -or -name '*.cpp' -or -name '*.h' | xargs grep -l -F $'\xEF\xBB\xBF' | grep -P '.' && echo "Files should not have UTF-8 BOM" From a783b05755b64f51422a8aed0618739356e4011d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Aug 2021 10:28:37 +0300 Subject: [PATCH 048/147] check-style: improve check for ReplicatedMergeTree --- utils/check-style/check-style | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index a096076a3f0..6ab8feab61c 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -129,11 +129,11 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do case "$test_case" in *.sh) test_case_zk_prefix="\$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" - grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" + grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.sql|*.sql.j2) test_case_zk_prefix="{[ ]*database[ ]*}" - grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" + grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.py) # Right now there is not such tests anyway From 507eae9aed045395cde49569cfd324cce50c9df9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Aug 2021 10:32:51 +0300 Subject: [PATCH 049/147] check-style: skip *.gen for ReplicatedMergeTree check --- utils/check-style/check-style | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 6ab8feab61c..d338dd4a2f3 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -127,6 +127,8 @@ tests_with_replicated_merge_tree=( $( ) ) for test_case in "${tests_with_replicated_merge_tree[@]}"; do case "$test_case" in + *.gen.*) + ;; *.sh) test_case_zk_prefix="\$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" From 09fa82c24446bdc330a731d7417f5c49cadee7db Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 09:58:04 +0300 Subject: [PATCH 050/147] check-style: fix {database} macro pattern --- utils/check-style/check-style | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index d338dd4a2f3..1f2354efe5a 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -134,7 +134,7 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.sql|*.sql.j2) - test_case_zk_prefix="{[ ]*database[ ]*}" + test_case_zk_prefix="{database}" grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.py) From 3cda5960ec14751fbb4e12b9131f0b75f9b18c7a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 10:07:32 +0300 Subject: [PATCH 051/147] check-style: allow currentDatabase() for ReplicatedMergeTree definition too --- utils/check-style/check-style | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 1f2354efe5a..05b8a1485e4 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -134,7 +134,7 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.sql|*.sql.j2) - test_case_zk_prefix="{database}" + test_case_zk_prefix="\({database}\|currentDatabase()\)" grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.py) From bc0af176feeecd504d1ff86a624dbc8358850c0e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 09:55:07 +0300 Subject: [PATCH 052/147] tests/stateless: add {database} macro for ReplicatedMergeTree in *.sql --- ...cated_merge_tree_alter_zookeeper.reference | 48 +++++++++---------- ..._replicated_merge_tree_alter_zookeeper.sql | 4 +- .../00083_create_merge_tree_zookeeper.sql | 4 +- .../00121_drop_column_zookeeper.sql | 4 +- .../00215_primary_key_order_zookeeper.sql | 2 +- ...per_deduplication_and_unexpected_parts.sql | 2 +- ...eplicated_drop_on_non_leader_zookeeper.sql | 4 +- ...46_clear_column_in_partition_zookeeper.sql | 4 +- ...stom_partitioning_replicated_zookeeper.sql | 20 ++++---- ...ed_storage_definition_syntax_zookeeper.sql | 4 +- ...lized_view_and_deduplication_zookeeper.sql | 4 +- ...ication_after_drop_partition_zookeeper.sql | 2 +- ...00563_insert_into_remote_and_zookeeper.sql | 2 +- ...23_replicated_truncate_table_zookeeper.sql | 4 +- .../00643_cast_zookeeper.reference | 2 +- .../0_stateless/00643_cast_zookeeper.sql | 4 +- ...identical_result_after_merge_zookeeper.sql | 4 +- .../0_stateless/00725_comment_columns.sql | 2 +- ...sert_have_data_before_quorum_zookeeper.sql | 4 +- ...ert_lost_part_and_alive_part_zookeeper.sql | 4 +- ...0732_quorum_insert_lost_part_zookeeper.sql | 4 +- ..._old_data_and_without_quorum_zookeeper.sql | 4 +- ...m_insert_simple_test_1_parts_zookeeper.sql | 4 +- ...m_insert_simple_test_2_parts_zookeeper.sql | 4 +- .../00753_comment_columns_zookeeper.reference | 4 +- .../00753_comment_columns_zookeeper.sql | 2 +- ...r_modify_order_by_replicated_zookeeper.sql | 2 +- ...dices_alter_replicated_zookeeper.reference | 24 +++++----- ...836_indices_alter_replicated_zookeeper.sql | 8 ++-- ...0837_minmax_index_replicated_zookeeper.sql | 4 +- ...om_compression_codecs_replicated.reference | 2 +- ...r_custom_compression_codecs_replicated.sql | 12 ++--- ...per_test_alter_compression_codecs_long.sql | 4 +- ...y_replicated_merge_tree_optimize_final.sql | 4 +- ...ndex_granularity_replicated_merge_tree.sql | 12 ++--- .../00933_ttl_replicated_zookeeper.reference | 2 +- .../00933_ttl_replicated_zookeeper.sql | 4 +- ...0988_constraints_replication_zookeeper.sql | 4 +- .../01037_zookeeper_check_table_empty_pk.sql | 2 +- .../01049_zookeeper_synchronous_mutations.sql | 4 +- .../01062_alter_on_mutataion_zookeeper.sql | 4 +- .../01079_alter_default_zookeeper.reference | 16 +++---- .../01079_alter_default_zookeeper.sql | 2 +- ..._zookeeper_mutations_and_insert_quorum.sql | 4 +- ...1135_default_and_alter_zookeeper.reference | 2 +- .../01135_default_and_alter_zookeeper.sql | 2 +- ...lumn_compact_part_replicated_zookeeper.sql | 2 +- ...213_alter_rename_primary_key_zookeeper.sql | 4 +- ...er_rename_with_default_zookeeper.reference | 6 +-- ...13_alter_rename_with_default_zookeeper.sql | 2 +- ...67_alter_default_key_columns_zookeeper.sql | 4 +- ...name_column_constraint_zookeeper.reference | 4 +- ...ter_rename_column_constraint_zookeeper.sql | 2 +- .../01319_manual_write_to_replicas.sql | 4 +- ...num_partition_key_replicated_zookeeper.sql | 4 +- .../01378_alter_rename_with_ttl_zookeeper.sql | 2 +- .../01430_modify_sample_by_zookeeper.sql | 2 +- ...1451_replicated_detach_drop_and_quorum.sql | 4 +- ...01451_replicated_detach_drop_part_long.sql | 4 +- ...ter_remove_no_property_zookeeper.reference | 4 +- ...493_alter_remove_no_property_zookeeper.sql | 2 +- ...lter_remove_properties_zookeeper.reference | 24 +++++----- ...1493_alter_remove_properties_zookeeper.sql | 4 +- ...509_parallel_quorum_insert_no_replicas.sql | 4 +- ...12_create_replicate_merge_tree_one_arg.sql | 2 +- ..._select_sequence_consistency_zookeeper.sql | 6 +-- .../01530_drop_database_atomic_sync.sql | 12 ++--- ...01532_execute_merges_on_single_replica.sql | 4 +- ...1581_deduplicate_by_columns_replicated.sql | 4 +- .../01646_system_restart_replicas_smoke.sql | 2 +- ..._drop_part_and_deduplication_zookeeper.sql | 10 ++-- ...0_fetch_patition_with_macro_in_zk_path.sql | 8 ++-- tests/queries/0_stateless/01666_blns.sql | 4 +- .../01669_columns_declaration_serde.sql | 4 +- .../01700_system_zookeeper_path_in.sql | 18 +++---- .../01710_projection_fetch.reference | 8 ++-- .../0_stateless/01710_projection_fetch.sql | 4 +- .../01713_table_ttl_old_syntax_zookeeper.sql | 6 +-- ...5_background_checker_blather_zookeeper.sql | 4 +- ...747_alter_partition_key_enum_zookeeper.sql | 2 +- .../01761_alter_decimal_zookeeper.reference | 4 +- .../01761_alter_decimal_zookeeper.sql | 2 +- .../01901_test_attach_partition_from.sql | 2 +- 83 files changed, 228 insertions(+), 228 deletions(-) diff --git a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference index cb61ab3e9b9..59e3e1e4a3f 100644 --- a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference +++ b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference @@ -1,22 +1,22 @@ d Date k UInt64 i32 Int32 -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 10 42 d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 9 41 1992-01-01 08:00:00 2015-01-01 10 42 1970-01-01 00:00:00 d Date @@ -25,14 +25,14 @@ i32 Int32 dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] 2015-01-01 10 42 1970-01-01 00:00:00 [] [] @@ -43,7 +43,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -51,7 +51,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['1970-01-01','1970-01-01','1970-01-01'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] [] @@ -64,7 +64,7 @@ n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) s String DEFAULT \'0\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -73,7 +73,7 @@ n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) s String DEFAULT \'0\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 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 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['1970-01-01','1970-01-01','1970-01-01'] 0 @@ -86,7 +86,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) s Int64 DEFAULT \'0\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -94,7 +94,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) s Int64 DEFAULT \'0\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 @@ -108,7 +108,7 @@ n.ui8 Array(UInt8) n.s Array(String) s UInt32 DEFAULT \'0\' n.d Array(Date) -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -117,7 +117,7 @@ n.ui8 Array(UInt8) n.s Array(String) s UInt32 DEFAULT \'0\' n.d Array(Date) -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 ['1970-01-01','1970-01-01','1970-01-01'] 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 ['1970-01-01','1970-01-01','1970-01-01'] 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 ['1970-01-01','1970-01-01','1970-01-01'] @@ -129,14 +129,14 @@ i32 Int32 dt DateTime(\'UTC\') n.s Array(String) s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') n.s Array(String) s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 ['asd','qwe','qwe'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 ['120','130','140'] 0 2015-01-01 8 40 2012-12-12 12:12:12 ['12','13','14'] 0 @@ -147,13 +147,13 @@ k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 2015-01-01 7 39 2014-07-14 13:26:50 0 2015-01-01 8 40 2012-12-12 12:12:12 0 @@ -166,7 +166,7 @@ dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' n.s Array(String) n.d Array(Date) -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -174,7 +174,7 @@ dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' n.s Array(String) n.d Array(Date) -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 [] [] 2015-01-01 7 39 2014-07-14 13:26:50 0 [] [] 2015-01-01 8 40 2012-12-12 12:12:12 0 [] [] @@ -185,13 +185,13 @@ k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 2015-01-01 7 39 2014-07-14 13:26:50 0 2015-01-01 8 40 2012-12-12 12:12:12 0 @@ -202,13 +202,13 @@ k UInt64 i32 Int32 dt Date s DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt Date s DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 1970-01-02 03:55:00 2015-01-01 7 39 2014-07-14 1970-01-01 00:00:00 2015-01-01 8 40 2012-12-12 1970-01-01 00:00:00 diff --git a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql index ac56b3416cd..9901dfad9c8 100644 --- a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql +++ b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql @@ -3,8 +3,8 @@ DROP TABLE IF EXISTS replicated_alter2; SET replication_alter_partitions_sync = 2; -CREATE TABLE replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00062/alter', 'r1', d, k, 8192); -CREATE TABLE replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00062/alter', 'r2', d, k, 8192); +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); INSERT INTO replicated_alter1 VALUES ('2015-01-01', 10, 42); diff --git a/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql b/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql index 998a4517163..65247d7b12c 100644 --- a/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql +++ b/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql @@ -56,7 +56,7 @@ CREATE TABLE aggregating_merge_tree_with_sampling (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = AggregatingMergeTree(d, sipHash64(a) + b, (a, sipHash64(a) + b), 111); CREATE TABLE replicated_merge_tree - (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00083/01/replicated_merge_tree/', 'r1', d, (a, b), 111); + (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_merge_tree/', 'r1', d, (a, b), 111); CREATE TABLE replicated_collapsing_merge_tree (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test_00083/01/replicated_collapsing_merge_tree/', 'r1', d, (a, b), 111, y); CREATE TABLE replicated_versioned_collapsing_merge_tree @@ -69,7 +69,7 @@ CREATE TABLE replicated_aggregating_merge_tree (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00083/01/replicated_aggregating_merge_tree/', 'r1', d, (a, b), 111); CREATE TABLE replicated_merge_tree_with_sampling - (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00083/01/replicated_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111); + (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111); CREATE TABLE replicated_collapsing_merge_tree_with_sampling (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test_00083/01/replicated_collapsing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111, y); CREATE TABLE replicated_versioned_collapsing_merge_tree_with_sampling diff --git a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql index 7ccf69e46b2..2aee56135f2 100644 --- a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql +++ b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql @@ -1,12 +1,12 @@ DROP TABLE IF EXISTS alter_00121; -CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter_00121/t1', 'r1', d, (d), 8192); +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); ALTER TABLE alter_00121 DROP COLUMN x; DROP TABLE alter_00121; -CREATE TABLE alter_00121 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter_00121/t2', 'r1', d, (d), 8192); +CREATE TABLE alter_00121 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t2', 'r1', d, (d), 8192); INSERT INTO alter_00121 VALUES ('2014-01-01'); SELECT * FROM alter_00121 ORDER BY d; diff --git a/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql b/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql index 8e36cbc85e0..86b84f3f63c 100644 --- a/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql +++ b/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS primary_key; -CREATE TABLE primary_key (d Date DEFAULT today(), x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00215/primary_key', 'r1', d, -x, 1); +CREATE TABLE primary_key (d Date DEFAULT today(), x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00215/primary_key', 'r1', d, -x, 1); INSERT INTO primary_key (x) VALUES (1), (2), (3); INSERT INTO primary_key (x) VALUES (1), (3), (2); diff --git a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql index c14ce53d4a3..d70f337213e 100644 --- a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql +++ b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS deduplication; -CREATE TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00226/deduplication', 'r1', d, x, 1); +CREATE TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00226/deduplication', 'r1', d, x, 1); INSERT INTO deduplication (x) VALUES (1); INSERT INTO deduplication (x) VALUES (1); diff --git a/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql b/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql index 95a5c7c97f1..96a131514f8 100644 --- a/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql +++ b/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql @@ -3,8 +3,8 @@ SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS attach_r1; DROP TABLE IF EXISTS attach_r2; -CREATE TABLE attach_r1 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00236/01/attach', 'r1', d, d, 8192); -CREATE TABLE attach_r2 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00236/01/attach', 'r2', d, d, 8192); +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); INSERT INTO attach_r1 VALUES ('2014-01-01'), ('2014-02-01'), ('2014-03-01'); diff --git a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql index 5d8c4de1c06..e4d60f1b960 100644 --- a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql @@ -24,8 +24,8 @@ SELECT '===Replicated case==='; DROP TABLE IF EXISTS clear_column1; DROP TABLE IF EXISTS clear_column2; SELECT sleep(1) FORMAT Null; -CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '1') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; -CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '2') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_00446/tables/clear_column', '1') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_00446/tables/clear_column', '2') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; INSERT INTO clear_column1 (d) VALUES ('2000-01-01'), ('2000-02-01'); SYSTEM SYNC REPLICA clear_column2; diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql index a2a771e6cc0..bd92fcefcff 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql @@ -4,8 +4,8 @@ SELECT '*** Not partitioned ***'; DROP TABLE IF EXISTS not_partitioned_replica1_00502; DROP TABLE IF EXISTS not_partitioned_replica2_00502; -CREATE TABLE not_partitioned_replica1_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/not_partitioned_00502', '1') ORDER BY x; -CREATE TABLE not_partitioned_replica2_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/not_partitioned_00502', '2') ORDER BY x; +CREATE TABLE not_partitioned_replica1_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '1') ORDER BY x; +CREATE TABLE not_partitioned_replica2_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '2') ORDER BY x; INSERT INTO not_partitioned_replica1_00502 VALUES (1), (2), (3); INSERT INTO not_partitioned_replica1_00502 VALUES (4), (5); @@ -30,8 +30,8 @@ SELECT '*** Partitioned by week ***'; DROP TABLE IF EXISTS partitioned_by_week_replica1; DROP TABLE IF EXISTS partitioned_by_week_replica2; -CREATE TABLE partitioned_by_week_replica1(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_week_00502', '1') PARTITION BY toMonday(d) ORDER BY x; -CREATE TABLE partitioned_by_week_replica2(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_week_00502', '2') PARTITION BY toMonday(d) ORDER BY x; +CREATE TABLE partitioned_by_week_replica1(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '1') PARTITION BY toMonday(d) ORDER BY x; +CREATE TABLE partitioned_by_week_replica2(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '2') PARTITION BY toMonday(d) ORDER BY x; -- 2000-01-03 belongs to a different week than 2000-01-01 and 2000-01-02 INSERT INTO partitioned_by_week_replica1 VALUES ('2000-01-01', 1), ('2000-01-02', 2), ('2000-01-03', 3); @@ -57,8 +57,8 @@ SELECT '*** Partitioned by a (Date, UInt8) tuple ***'; DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00502; DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00502; -CREATE TABLE partitioned_by_tuple_replica1_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_tuple_00502', '1') ORDER BY x PARTITION BY (d, x); -CREATE TABLE partitioned_by_tuple_replica2_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_tuple_00502', '2') ORDER BY x PARTITION BY (d, x); +CREATE TABLE partitioned_by_tuple_replica1_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '1') ORDER BY x PARTITION BY (d, x); +CREATE TABLE partitioned_by_tuple_replica2_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '2') ORDER BY x PARTITION BY (d, x); INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-01', 1, 1), ('2000-01-01', 2, 2), ('2000-01-02', 1, 3); INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-02', 1, 4), ('2000-01-01', 1, 5); @@ -84,8 +84,8 @@ SELECT '*** Partitioned by String ***'; DROP TABLE IF EXISTS partitioned_by_string_replica1; DROP TABLE IF EXISTS partitioned_by_string_replica2; -CREATE TABLE partitioned_by_string_replica1(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_string_00502', '1') PARTITION BY s ORDER BY x; -CREATE TABLE partitioned_by_string_replica2(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_string_00502', '2') PARTITION BY s ORDER BY x; +CREATE TABLE partitioned_by_string_replica1(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '1') PARTITION BY s ORDER BY x; +CREATE TABLE partitioned_by_string_replica2(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '2') PARTITION BY s ORDER BY x; INSERT INTO partitioned_by_string_replica1 VALUES ('aaa', 1), ('aaa', 2), ('bbb', 3); INSERT INTO partitioned_by_string_replica1 VALUES ('bbb', 4), ('aaa', 5); @@ -110,8 +110,8 @@ SELECT '*** Table without columns with fixed size ***'; DROP TABLE IF EXISTS without_fixed_size_columns_replica1; DROP TABLE IF EXISTS without_fixed_size_columns_replica2; -CREATE TABLE without_fixed_size_columns_replica1(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/without_fixed_size_columns_00502', '1') PARTITION BY length(s) ORDER BY s; -CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/without_fixed_size_columns_00502', '2') PARTITION BY length(s) ORDER BY s; +CREATE TABLE without_fixed_size_columns_replica1(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '1') PARTITION BY length(s) ORDER BY s; +CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '2') PARTITION BY length(s) ORDER BY s; INSERT INTO without_fixed_size_columns_replica1 VALUES ('a'), ('aa'), ('b'), ('cc'); diff --git a/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql b/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql index ef8655a1861..48678329404 100644 --- a/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql +++ b/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql @@ -5,7 +5,7 @@ SELECT '*** Replicated with sampling ***'; DROP TABLE IF EXISTS replicated_with_sampling; CREATE TABLE replicated_with_sampling(x UInt8) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00509/replicated_with_sampling', 'r1') + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00509/replicated_with_sampling', 'r1') ORDER BY x SAMPLE BY x; @@ -72,7 +72,7 @@ SELECT '*** Table definition with SETTINGS ***'; DROP TABLE IF EXISTS with_settings; CREATE TABLE with_settings(x UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00509/with_settings', 'r1') + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00509/with_settings', 'r1') ORDER BY x SETTINGS replicated_can_become_leader = 0; diff --git a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql index 8df012a8588..24f64a577f1 100644 --- a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql +++ b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql @@ -4,9 +4,9 @@ DROP TABLE IF EXISTS with_deduplication_mv; DROP TABLE IF EXISTS without_deduplication_mv; CREATE TABLE with_deduplication(x UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00510/with_deduplication', 'r1') ORDER BY x; + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/with_deduplication', 'r1') ORDER BY x; CREATE TABLE without_deduplication(x UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00510/without_deduplication', 'r1') ORDER BY x SETTINGS replicated_deduplication_window = 0; + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/without_deduplication', 'r1') ORDER BY x SETTINGS replicated_deduplication_window = 0; CREATE MATERIALIZED VIEW with_deduplication_mv UUID '00000510-1000-4000-8000-000000000001' ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00510/with_deduplication_mv', 'r1') ORDER BY dummy 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 2a9ebb992a8..71ed99f78fa 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 @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS deduplication_by_partition; CREATE TABLE deduplication_by_partition(d Date, x UInt32) ENGINE = - ReplicatedMergeTree('/clickhouse/tables/test_00516/deduplication_by_partition', 'r1', d, x, 8192); + ReplicatedMergeTree('/clickhouse/tables/{database}/test_00516/deduplication_by_partition', 'r1', d, x, 8192); INSERT INTO deduplication_by_partition VALUES ('2000-01-01', 1); INSERT INTO deduplication_by_partition VALUES ('2000-01-01', 2), ('2000-01-01', 3); diff --git a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql index 9c00ee68ba9..4cfbc1d0fb0 100644 --- a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql +++ b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql @@ -1,6 +1,6 @@ -- Check that settings are correctly passed through Distributed table DROP TABLE IF EXISTS simple; -CREATE TABLE simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/test_00563/tables/simple', '1') ORDER BY d; +CREATE TABLE simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_00563/tables/simple', '1') ORDER BY d; -- TODO: replace '127.0.0.2' -> '127.0.0.1' after a fix INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (1); diff --git a/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql b/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql index 0f6d51b8716..06e88754888 100644 --- a/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql +++ b/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS replicated_truncate1; DROP TABLE IF EXISTS replicated_truncate2; -CREATE TABLE replicated_truncate1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00623/truncate', 'r1', d, k, 8192); -CREATE TABLE replicated_truncate2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00623/truncate', 'r2', d, k, 8192); +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); SELECT '======Before Truncate======'; INSERT INTO replicated_truncate1 VALUES ('2015-01-01', 10, 42); diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.reference b/tests/queries/0_stateless/00643_cast_zookeeper.reference index 9123463de1a..4eb4a4e0365 100644 --- a/tests/queries/0_stateless/00643_cast_zookeeper.reference +++ b/tests/queries/0_stateless/00643_cast_zookeeper.reference @@ -3,7 +3,7 @@ CREATE TABLE default.cast1 `x` UInt8, `e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)') ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r1') +ENGINE = ReplicatedMergeTree('/clickhouse/tables/default/test_00643/cast', 'r1') ORDER BY e SETTINGS index_granularity = 8192 x UInt8 diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.sql b/tests/queries/0_stateless/00643_cast_zookeeper.sql index c9760f00ca7..21e7a89c603 100644 --- a/tests/queries/0_stateless/00643_cast_zookeeper.sql +++ b/tests/queries/0_stateless/00643_cast_zookeeper.sql @@ -22,7 +22,7 @@ CREATE TABLE cast1 'world' = 2 ) ) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r1') ORDER BY e; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00643/cast', 'r1') ORDER BY e; SHOW CREATE TABLE cast1 FORMAT TSVRaw; DESC TABLE cast1; @@ -30,7 +30,7 @@ DESC TABLE cast1; INSERT INTO cast1 (x) VALUES (1); SELECT * FROM cast1; -CREATE TABLE cast2 AS cast1 ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r2') ORDER BY e; +CREATE TABLE cast2 AS cast1 ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00643/cast', 'r2') ORDER BY e; SYSTEM SYNC REPLICA cast2; diff --git a/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql b/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql index 498896d31a7..50f51510d61 100644 --- a/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql +++ b/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS byte_identical_r1; DROP TABLE IF EXISTS byte_identical_r2; -CREATE TABLE byte_identical_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00721/byte_identical', 'r1') ORDER BY x; -CREATE TABLE byte_identical_r2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00721/byte_identical', 'r2') ORDER BY x; +CREATE TABLE byte_identical_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00721/byte_identical', 'r1') ORDER BY x; +CREATE TABLE byte_identical_r2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00721/byte_identical', 'r2') ORDER BY x; INSERT INTO byte_identical_r1(x) VALUES (1), (2), (3); SYSTEM SYNC REPLICA byte_identical_r2; diff --git a/tests/queries/0_stateless/00725_comment_columns.sql b/tests/queries/0_stateless/00725_comment_columns.sql index 6fe095d9935..3eec0ba31b7 100644 --- a/tests/queries/0_stateless/00725_comment_columns.sql +++ b/tests/queries/0_stateless/00725_comment_columns.sql @@ -87,4 +87,4 @@ FORMAT PrettyCompactNoEscapes; DROP TABLE IF EXISTS check_query_comment_column; --- TODO: add here tests with ReplicatedMergeTree +-- TODO: add here tests with ReplicatedMergeTree({database}) diff --git a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql index a1fc25fbf0b..a92646a41fc 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_have_data', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_have_data', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_have_data', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_have_data', '2') ORDER BY x PARTITION BY y; INSERT INTO quorum1 VALUES (1, '1990-11-15'); INSERT INTO quorum1 VALUES (2, '1990-11-15'); diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql index 914d98ff3f5..4f9afb93f3a 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost_alive', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost_alive', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost_alive', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost_alive', '2') ORDER BY x PARTITION BY y; SET insert_quorum=2, insert_quorum_parallel=0; SET select_sequential_consistency=1; diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql index 18cad0dda41..1374fa776ca 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost', '2') ORDER BY x PARTITION BY y; SET insert_quorum=2, insert_quorum_parallel=0; SET select_sequential_consistency=1; diff --git a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql index 11c8324ac25..0cc162b6010 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_old_data', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_old_data', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_old_data', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_old_data', '2') ORDER BY x PARTITION BY y; INSERT INTO quorum1 VALUES (1, '1990-11-15'); INSERT INTO quorum1 VALUES (2, '1990-11-15'); diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql index a4e262ce3c4..0ceca3743b2 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum1', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum1', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum1', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum1', '2') ORDER BY x PARTITION BY y; SET insert_quorum=2, insert_quorum_parallel=0; SET select_sequential_consistency=1; diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql index 40c08fd926f..a95d992705e 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum2', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum2', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum2', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum2', '2') ORDER BY x PARTITION BY y; SET insert_quorum=2, insert_quorum_parallel=0; SET select_sequential_consistency=1; diff --git a/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference b/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference index 74aa60a41f2..9bf0304d7d7 100644 --- a/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference +++ b/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference @@ -1,6 +1,6 @@ -CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192 +CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192 column_name1 UInt8 DEFAULT 1 comment column_name2 UInt8 non default comment -CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'another comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192 +CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'another comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192 column_name1 UInt8 DEFAULT 1 another comment column_name2 UInt8 non default comment diff --git a/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql b/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql index e98630bb00d..93e2c9fb9d5 100644 --- a/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql +++ b/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql @@ -4,7 +4,7 @@ CREATE TABLE check_comments ( column_name1 UInt8 DEFAULT 1 COMMENT 'comment', column_name2 UInt8 COMMENT 'non default comment' - ) ENGINE = ReplicatedMergeTree('clickhouse/tables/test_00753/comments', 'r1') + ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00753/comments', 'r1') ORDER BY column_name1; SHOW CREATE check_comments; diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql index 809adfaa498..9e08406202e 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql @@ -3,7 +3,7 @@ SET optimize_on_insert = 0; SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS old_style; -CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00754/old_style', 'r1', d, x, 8192); +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/00836_indices_alter_replicated_zookeeper.reference b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference index 838bd93ebaf..fbe0e7f564f 100644 --- a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference +++ b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference @@ -1,5 +1,5 @@ -CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 2 1 2 @@ -14,8 +14,8 @@ CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n I 3 2 19 9 65 75 -CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 4 1 5 @@ -28,10 +28,10 @@ CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n I 3 2 19 9 65 75 -CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 4 1 5 @@ -44,14 +44,14 @@ CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n I 3 2 19 9 65 75 -CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 3 1 2 1 3 -CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 3 1 2 diff --git a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql index e038d2d425e..6274a70a381 100644 --- a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql @@ -9,14 +9,14 @@ CREATE TABLE minmax_idx ( u64 UInt64, i32 Int32 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter1', 'r1') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter1', 'r1') ORDER BY u64; CREATE TABLE minmax_idx_r ( u64 UInt64, i32 Int32 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter1', 'r2') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter1', 'r2') ORDER BY u64; INSERT INTO minmax_idx VALUES (1, 2); @@ -74,7 +74,7 @@ CREATE TABLE minmax_idx2 i32 Int32, INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter2', 'r1') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter2', 'r1') ORDER BY u64; CREATE TABLE minmax_idx2_r @@ -83,7 +83,7 @@ CREATE TABLE minmax_idx2_r i32 Int32, INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter2', 'r2') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter2', 'r2') ORDER BY u64; diff --git a/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql b/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql index 2213ccecd4e..0dd8dd41f97 100644 --- a/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql @@ -15,7 +15,7 @@ CREATE TABLE minmax_idx1 idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 1, INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00837/minmax', 'r1') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00837/minmax', 'r1') ORDER BY u64 SETTINGS index_granularity = 2; @@ -32,7 +32,7 @@ CREATE TABLE minmax_idx2 idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 1, INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00837/minmax', 'r2') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00837/minmax', 'r2') ORDER BY u64 SETTINGS index_granularity = 2; diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference index 29e7b23d3dd..3b7faecbba4 100644 --- a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference +++ b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference @@ -20,7 +20,7 @@ 274972506.6 9175437371954010821 9175437371954010821 -CREATE TABLE default.compression_codec_multiple_more_types_replicated\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00910/compression_codec_multiple_more_types_replicated\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.compression_codec_multiple_more_types_replicated\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00910/compression_codec_multiple_more_types_replicated\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1.5555555555555 hello world! [77] ['John'] 7.1000000000000 xxxxxxxxxxxx [127] ['Henry'] ! diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql index 52eb1d4e411..3fe121edc94 100644 --- a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql +++ b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql @@ -11,7 +11,7 @@ CREATE TABLE compression_codec_replicated1( somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), othernum Int64 CODEC(Delta) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_replicated', '1') ORDER BY tuple(); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_replicated', '1') ORDER BY tuple(); CREATE TABLE compression_codec_replicated2( id UInt64 CODEC(LZ4), @@ -20,7 +20,7 @@ CREATE TABLE compression_codec_replicated2( somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), othernum Int64 CODEC(Delta) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_replicated', '2') ORDER BY tuple(); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_replicated', '2') ORDER BY tuple(); INSERT INTO compression_codec_replicated1 VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); @@ -57,14 +57,14 @@ CREATE TABLE compression_codec_multiple_replicated1 ( data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple', '1') ORDER BY tuple(); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple', '1') ORDER BY tuple(); CREATE TABLE compression_codec_multiple_replicated2 ( id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple', '2') ORDER BY tuple(); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple', '2') ORDER BY tuple(); INSERT INTO compression_codec_multiple_replicated2 VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); @@ -106,7 +106,7 @@ CREATE TABLE compression_codec_multiple_more_types_replicated ( id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC), data FixedString(12) CODEC(ZSTD, ZSTD, Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC), ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8)) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple_more_types_replicated', '1') ORDER BY tuple(); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple_more_types_replicated', '1') ORDER BY tuple(); SHOW CREATE TABLE compression_codec_multiple_more_types_replicated; @@ -124,7 +124,7 @@ CREATE TABLE compression_codec_multiple_with_key_replicated ( somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta), id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta), data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple_with_key_replicated', '1') PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple_with_key_replicated', '1') PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; INSERT INTO compression_codec_multiple_with_key_replicated VALUES(toDate('2018-10-12'), 100000, 'hello'), (toDate('2018-10-12'), 100002, 'world'), (toDate('2018-10-12'), 1111, '!'); diff --git a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql index 548f26eadd0..e4994ff8d75 100644 --- a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql +++ b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql @@ -7,12 +7,12 @@ DROP TABLE IF EXISTS alter_compression_codec2; CREATE TABLE alter_compression_codec1 ( somedate Date CODEC(LZ4), id UInt64 CODEC(NONE) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '1_{replica}') PARTITION BY somedate ORDER BY id; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '1_{replica}') PARTITION BY somedate ORDER BY id; CREATE TABLE alter_compression_codec2 ( somedate Date CODEC(LZ4), id UInt64 CODEC(NONE) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '2_{replica}') PARTITION BY somedate ORDER BY id; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '2_{replica}') PARTITION BY somedate ORDER BY id; INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 1); INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 2); diff --git a/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql b/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql index e227222a939..2b119836ebf 100644 --- a/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql +++ b/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS replicated_optimize1; DROP TABLE IF EXISTS replicated_optimize2; -CREATE TABLE replicated_optimize1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00925/optimize', 'r1', d, k, 8192); -CREATE TABLE replicated_optimize2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00925/optimize', 'r2', d, k, 8192); +CREATE TABLE replicated_optimize1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00925/optimize', 'r1', d, k, 8192); +CREATE TABLE replicated_optimize2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00925/optimize', 'r2', d, k, 8192); OPTIMIZE TABLE replicated_optimize1 FINAL; diff --git a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql index 92b34a243a4..1e04679e1f3 100644 --- a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql +++ b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql @@ -8,14 +8,14 @@ CREATE TABLE zero_rows_per_granule1 ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/zero_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/zero_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0; CREATE TABLE zero_rows_per_granule2 ( p Date, k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/zero_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/zero_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0; INSERT INTO zero_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -68,14 +68,14 @@ CREATE TABLE four_rows_per_granule1 ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/four_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/four_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; CREATE TABLE four_rows_per_granule2 ( p Date, k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/four_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100 ,write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/four_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100 ,write_final_mark = 0; INSERT INTO four_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -135,14 +135,14 @@ CREATE TABLE adaptive_granularity_alter1 ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/adaptive_granularity_alter', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/adaptive_granularity_alter', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; CREATE TABLE adaptive_granularity_alter2 ( p Date, k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/adaptive_granularity_alter', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/adaptive_granularity_alter', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference index e84c3beabee..3dc07236d2b 100644 --- a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference +++ b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference @@ -1,3 +1,3 @@ 200 400 -CREATE TABLE default.ttl_repl2\n(\n `d` Date,\n `x` UInt32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00933/ttl_repl\', \'2\')\nPARTITION BY toDayOfMonth(d)\nORDER BY x\nTTL d + toIntervalDay(1)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.ttl_repl2\n(\n `d` Date,\n `x` UInt32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00933/ttl_repl\', \'2\')\nPARTITION BY toDayOfMonth(d)\nORDER BY x\nTTL d + toIntervalDay(1)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql index dbbbe887e9f..6085545dc02 100644 --- a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql @@ -1,9 +1,9 @@ DROP TABLE IF EXISTS ttl_repl1; DROP TABLE IF EXISTS ttl_repl2; -CREATE TABLE ttl_repl1(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00933/ttl_repl', '1') +CREATE TABLE ttl_repl1(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00933/ttl_repl', '1') PARTITION BY toDayOfMonth(d) ORDER BY x TTL d + INTERVAL 1 DAY; -CREATE TABLE ttl_repl2(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00933/ttl_repl', '2') +CREATE TABLE ttl_repl2(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00933/ttl_repl', '2') PARTITION BY toDayOfMonth(d) ORDER BY x TTL d + INTERVAL 1 DAY; INSERT INTO TABLE ttl_repl1 VALUES (toDate('2000-10-10 00:00:00'), 100); diff --git a/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql b/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql index f8895843dd3..c36bee7fe4f 100644 --- a/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql +++ b/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql @@ -6,14 +6,14 @@ CREATE TABLE replicated_constraints1 a UInt32, b UInt32, CONSTRAINT a_constraint CHECK a < 10 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00988/alter_constraints', 'r1') ORDER BY (a); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00988/alter_constraints', 'r1') ORDER BY (a); CREATE TABLE replicated_constraints2 ( a UInt32, b UInt32, CONSTRAINT a_constraint CHECK a < 10 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00988/alter_constraints', 'r2') ORDER BY (a); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00988/alter_constraints', 'r2') ORDER BY (a); INSERT INTO replicated_constraints1 VALUES (1, 2); INSERT INTO replicated_constraints2 VALUES (3, 4); diff --git a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql index 60c6d55c4f5..262ccecc7d4 100644 --- a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql +++ b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql @@ -13,7 +13,7 @@ DROP TABLE IF EXISTS mt_without_pk; DROP TABLE IF EXISTS replicated_mt_without_pk; -CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01037/replicated_mt_without_pk', '1') ORDER BY tuple(); +CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01037/replicated_mt_without_pk', '1') ORDER BY tuple(); INSERT INTO replicated_mt_without_pk VALUES (1, 2); diff --git a/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql b/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql index c84fb22043f..e369b500c35 100644 --- a/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql +++ b/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql @@ -3,9 +3,9 @@ DROP TABLE IF EXISTS table_for_synchronous_mutations2; SELECT 'Replicated'; -CREATE TABLE table_for_synchronous_mutations1(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01049/table_for_synchronous_mutations', '1') ORDER BY k; +CREATE TABLE table_for_synchronous_mutations1(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01049/table_for_synchronous_mutations', '1') ORDER BY k; -CREATE TABLE table_for_synchronous_mutations2(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01049/table_for_synchronous_mutations', '2') ORDER BY k; +CREATE TABLE table_for_synchronous_mutations2(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01049/table_for_synchronous_mutations', '2') ORDER BY k; INSERT INTO table_for_synchronous_mutations1 select number, number from numbers(100000); diff --git a/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql index b830c549dc3..53d7100a38e 100644 --- a/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql +++ b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql @@ -6,7 +6,7 @@ CREATE TABLE test_alter_on_mutation key UInt64, value String ) -ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01062/alter_on_mutation', '1') +ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01062/alter_on_mutation', '1') ORDER BY key PARTITION BY date; INSERT INTO test_alter_on_mutation select toDate('2020-01-05'), number, toString(number) from system.numbers limit 100; @@ -58,7 +58,7 @@ DROP TABLE IF EXISTS test_alter_on_mutation; DROP TABLE IF EXISTS nested_alter; -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/test_01062/nested_alter', 'r2', d, k, 8192); +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/01079_alter_default_zookeeper.reference b/tests/queries/0_stateless/01079_alter_default_zookeeper.reference index 7dd539b2683..32d31af0058 100644 --- a/tests/queries/0_stateless/01079_alter_default_zookeeper.reference +++ b/tests/queries/0_stateless/01079_alter_default_zookeeper.reference @@ -1,11 +1,11 @@ -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` String DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` String DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 1000 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 1000 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt16 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt16 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 10000 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\',\n `other_date` String DEFAULT 1\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\',\n `other_date` String DEFAULT 1\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01079_alter_default_zookeeper.sql b/tests/queries/0_stateless/01079_alter_default_zookeeper.sql index 6fa9d2bf4e0..7257f86c4e8 100644 --- a/tests/queries/0_stateless/01079_alter_default_zookeeper.sql +++ b/tests/queries/0_stateless/01079_alter_default_zookeeper.sql @@ -5,7 +5,7 @@ CREATE TABLE alter_default date Date, key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01079/alter_default', '1') +ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01079/alter_default', '1') ORDER BY key; INSERT INTO alter_default select toDate('2020-01-05'), number from system.numbers limit 100; diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql index 5597bf523e4..d38c639458d 100644 --- a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql +++ b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS mutations_and_quorum1; DROP TABLE IF EXISTS mutations_and_quorum2; -CREATE TABLE mutations_and_quorum1 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01090/mutations_and_quorum', '1') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); -CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01090/mutations_and_quorum', '2') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); +CREATE TABLE mutations_and_quorum1 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '1') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); +CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '2') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); SET insert_quorum=2, insert_quorum_parallel=0; diff --git a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.reference b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.reference index cf69f1c5896..30cdc08e87d 100644 --- a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.reference +++ b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.reference @@ -1,2 +1,2 @@ 4 -CREATE TABLE default.default_table\n(\n `id` UInt64,\n `enum_column` Enum8(\'undefined\' = 0, \'fox\' = 1, \'index\' = 2) DEFAULT \'fox\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01135/default_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.default_table\n(\n `id` UInt64,\n `enum_column` Enum8(\'undefined\' = 0, \'fox\' = 1, \'index\' = 2) DEFAULT \'fox\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01135/default_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql index 982c72eb333..ea8c943b9a4 100644 --- a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql +++ b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql @@ -5,7 +5,7 @@ CREATE TABLE default_table id UInt64, enum_column Enum8('undefined' = 0, 'fox' = 1, 'index' = 2) ) -ENGINE ReplicatedMergeTree('/clickhouse/test_01135/default_table', '1') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/test_01135/default_table', '1') ORDER BY tuple(); INSERT INTO default_table VALUES(1, 'index'), (2, 'fox'); diff --git a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql index 2aa903d99f0..9464ac24dff 100644 --- a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql @@ -3,7 +3,7 @@ set replication_alter_partitions_sync = 2; drop table if exists mt_compact; create table mt_compact(a UInt64, b UInt64 DEFAULT a * a, s String, n Nested(x UInt32, y String), lc LowCardinality(String)) -engine = ReplicatedMergeTree('/clickhouse/test_01201/mt_compact_replicated', '1') +engine = ReplicatedMergeTree('/clickhouse/{database}/test_01201/mt_compact_replicated', '1') order by a partition by a % 10 settings index_granularity = 8, min_rows_for_wide_part = 10; diff --git a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql index 616a213e46c..91dd6e67512 100644 --- a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql +++ b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql @@ -9,7 +9,7 @@ CREATE TABLE table_for_rename_pk value1 String, value2 String ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01213/table_for_rename_pk1', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01213/table_for_rename_pk1', '1') PARTITION BY date ORDER BY (key1, pow(key2, 2), key3); @@ -37,7 +37,7 @@ CREATE TABLE table_for_rename_with_primary_key value2 String, INDEX idx (value1) TYPE set(1) GRANULARITY 1 ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01213/table_for_rename_pk2', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01213/table_for_rename_pk2', '1') PARTITION BY date ORDER BY (key1, key2, key3) PRIMARY KEY (key1, key2); diff --git a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference index da3dad5cb16..2a6b00cdddb 100644 --- a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference +++ b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference @@ -8,10 +8,10 @@ Hello 1 Word 1 date1 date2 value1 value2 2019-10-02 2018-10-02 1 1 -CREATE TABLE default.table_rename_with_ttl\n(\n `date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_rename_with_ttl\n(\n `date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 renamed_date1 date2 value1 value2 2019-10-02 2018-10-02 1 1 -CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 renamed_date1 renamed_date2 value1 value2 2019-10-02 2018-10-02 1 1 -CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `renamed_date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL renamed_date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `renamed_date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL renamed_date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql index e5701077770..cb9e918e335 100644 --- a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql +++ b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql @@ -38,7 +38,7 @@ CREATE TABLE table_rename_with_ttl value1 String, value2 String TTL date1 + INTERVAL 10000 MONTH ) -ENGINE = ReplicatedMergeTree('/clickhouse/test_01213/table_rename_with_ttl', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01213/table_rename_with_ttl', '1') ORDER BY tuple() TTL date2 + INTERVAL 10000 MONTH; diff --git a/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql b/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql index d96085bc086..6e7f4d55a53 100644 --- a/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql +++ b/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql @@ -8,8 +8,8 @@ DROP TABLE test_alter; DROP TABLE IF EXISTS test_alter_r1; DROP TABLE IF EXISTS test_alter_r2; -CREATE TABLE test_alter_r1 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01267/alter', 'r1') ORDER BY s PARTITION BY x; -CREATE TABLE test_alter_r2 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01267/alter', 'r2') ORDER BY s PARTITION BY x; +CREATE TABLE test_alter_r1 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01267/alter', 'r1') ORDER BY s PARTITION BY x; +CREATE TABLE test_alter_r2 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01267/alter', 'r2') ORDER BY s PARTITION BY x; ALTER TABLE test_alter_r1 MODIFY COLUMN s DEFAULT 'Hello' SETTINGS replication_alter_partitions_sync = 2; ALTER TABLE test_alter_r2 MODIFY COLUMN x DEFAULT '2000-01-01' SETTINGS replication_alter_partitions_sync = 2; diff --git a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference index 84ca8273128..382ccb592af 100644 --- a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference +++ b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference @@ -7,7 +7,7 @@ 2019-10-01 6 6 7 8 2019-10-02 7 7 8 9 2019-10-03 8 8 9 10 -CREATE TABLE default.table_for_rename1\n(\n `date` Date,\n `key` UInt64,\n `value4` String,\n `value5` String,\n `value3` String,\n CONSTRAINT cs_value1 CHECK toInt64(value4) < toInt64(value5),\n CONSTRAINT cs_value2 CHECK toInt64(value5) < toInt64(value3)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01277/test_for_rename\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_for_rename1\n(\n `date` Date,\n `key` UInt64,\n `value4` String,\n `value5` String,\n `value3` String,\n CONSTRAINT cs_value1 CHECK toInt64(value4) < toInt64(value5),\n CONSTRAINT cs_value2 CHECK toInt64(value5) < toInt64(value3)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01277/test_for_rename\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 2019-10-01 0 0 1 2 2019-10-02 1 1 2 3 2019-10-03 2 2 3 4 @@ -38,7 +38,7 @@ CREATE TABLE default.table_for_rename1\n(\n `date` Date,\n `key` UInt64,\n 2019-10-01 18 18 19 20 2019-10-02 19 19 20 21 -- rename columns back -- -CREATE TABLE default.table_for_rename1\n(\n `date` Date,\n `key` UInt64,\n `value1` String,\n `value2` String,\n `value3` String,\n CONSTRAINT cs_value1 CHECK toInt64(value1) < toInt64(value2),\n CONSTRAINT cs_value2 CHECK toInt64(value2) < toInt64(value3)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01277/test_for_rename\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_for_rename1\n(\n `date` Date,\n `key` UInt64,\n `value1` String,\n `value2` String,\n `value3` String,\n CONSTRAINT cs_value1 CHECK toInt64(value1) < toInt64(value2),\n CONSTRAINT cs_value2 CHECK toInt64(value2) < toInt64(value3)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01277/test_for_rename\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 2019-10-01 0 0 1 2 2019-10-02 1 1 2 3 2019-10-03 2 2 3 4 diff --git a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql index 28f17dced97..10ab75e42bd 100644 --- a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql +++ b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql @@ -10,7 +10,7 @@ CREATE TABLE table_for_rename1 CONSTRAINT cs_value1 CHECK toInt64(value1) < toInt64(value2), CONSTRAINT cs_value2 CHECK toInt64(value2) < toInt64(value3) ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01277/test_for_rename', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01277/test_for_rename', '1') PARTITION BY date ORDER BY key; diff --git a/tests/queries/0_stateless/01319_manual_write_to_replicas.sql b/tests/queries/0_stateless/01319_manual_write_to_replicas.sql index 5388f0017c0..7fb4d0b7d61 100644 --- a/tests/queries/0_stateless/01319_manual_write_to_replicas.sql +++ b/tests/queries/0_stateless/01319_manual_write_to_replicas.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS r1; DROP TABLE IF EXISTS r2; -CREATE TABLE r1 (x String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r1') ORDER BY x; -CREATE TABLE r2 (x String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r2') ORDER BY x; +CREATE TABLE r1 (x String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/r', 'r1') ORDER BY x; +CREATE TABLE r2 (x String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/r', 'r2') ORDER BY x; SYSTEM STOP REPLICATED SENDS; diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql index d185973f564..d0b6fbe3dde 100644 --- a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS test; DROP TABLE IF EXISTS test2; -CREATE TABLE test (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/test_01346/table', 'r1') PARTITION BY x ORDER BY y; -CREATE TABLE test2 (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/test_01346/table', 'r2') PARTITION BY x ORDER BY y; +CREATE TABLE test (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r1') PARTITION BY x ORDER BY y; +CREATE TABLE test2 (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r2') PARTITION BY x ORDER BY y; INSERT INTO test VALUES ('hello', 'test'); SELECT * FROM test; diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql index 0cd6feb9da1..79d1276ae61 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql @@ -5,7 +5,7 @@ CREATE TABLE table_rename_with_ttl date1 Date, value1 String ) -ENGINE = ReplicatedMergeTree('/clickhouse/test/table_rename_with_ttl_01378', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test/table_rename_with_ttl_01378', '1') ORDER BY tuple(); INSERT INTO table_rename_with_ttl SELECT toDate('2018-10-01') + number % 3, toString(number) from numbers(9); diff --git a/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql b/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql index dc7f5017bfd..288fa97d218 100644 --- a/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql +++ b/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql @@ -11,7 +11,7 @@ SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM modify_sample SAMPLE 0 ALTER TABLE modify_sample MODIFY SAMPLE BY x; SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM modify_sample SAMPLE 0.1; -CREATE TABLE modify_sample_replicated (d Date DEFAULT '2000-01-01', x UInt8, y UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01430', 'modify_sample') PARTITION BY d ORDER BY (x, y); +CREATE TABLE modify_sample_replicated (d Date DEFAULT '2000-01-01', x UInt8, y UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01430', 'modify_sample') PARTITION BY d ORDER BY (x, y); INSERT INTO modify_sample_replicated (x, y) SELECT toUInt8(number) AS x, toUInt64(number) as y FROM system.numbers LIMIT 256; diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql index 72e588c5c02..2c6bafe4936 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql @@ -4,8 +4,8 @@ SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS replica1; DROP TABLE IF EXISTS replica2; -CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/quorum', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; -CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/quorum', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/01451/quorum', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/01451/quorum', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; INSERT INTO replica1 VALUES (0); diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql index cd8267ce59a..5da86f45fde 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql @@ -3,8 +3,8 @@ SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS replica1; DROP TABLE IF EXISTS replica2; -CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; -CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; INSERT INTO replica1 VALUES (0); INSERT INTO replica1 VALUES (1); diff --git a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference index 82f6fbd6615..a770460c8b8 100644 --- a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference +++ b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference @@ -1,4 +1,4 @@ CREATE TABLE default.no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 CREATE TABLE default.no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql index aceb4e8140d..8d37900aaf8 100644 --- a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql +++ b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql @@ -29,7 +29,7 @@ CREATE TABLE r_no_prop_table ( some_column UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/test/01493_r_no_prop_table', '1') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/test/01493_r_no_prop_table', '1') ORDER BY tuple(); SHOW CREATE TABLE r_no_prop_table; diff --git a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference index 7cd2c81b460..42f9b24715f 100644 --- a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference +++ b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference @@ -1,21 +1,21 @@ -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 ====== remove column comment ====== -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 ====== remove column codec ====== -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 ====== remove column default ====== 42 str 1 0 tsr 2 -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 ====== remove column TTL ====== -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 ====== remove table TTL ====== -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 3 3 diff --git a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql index b810e2a8737..55f610eb7d0 100644 --- a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql +++ b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql @@ -10,7 +10,7 @@ CREATE TABLE r_prop_table1 column_comment Date COMMENT 'Some comment', column_ttl UInt64 TTL column_comment + INTERVAL 1 MONTH ) -ENGINE ReplicatedMergeTree('/clickhouse/test_01493/r_prop_table', '1') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/test_01493/r_prop_table', '1') ORDER BY tuple() TTL column_comment + INTERVAL 2 MONTH; @@ -21,7 +21,7 @@ CREATE TABLE r_prop_table2 column_comment Date COMMENT 'Some comment', column_ttl UInt64 TTL column_comment + INTERVAL 1 MONTH ) -ENGINE ReplicatedMergeTree('/clickhouse/test_01493/r_prop_table', '2') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/test_01493/r_prop_table', '2') ORDER BY tuple() TTL column_comment + INTERVAL 2 MONTH; diff --git a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql index 16c4a4df936..ba2b774b718 100644 --- a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql +++ b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql @@ -4,13 +4,13 @@ DROP TABLE IF EXISTS r2 SYNC; CREATE TABLE r1 ( key UInt64, value String ) -ENGINE = ReplicatedMergeTree('/clickhouse/01509_parallel_quorum_insert_no_replicas', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01509_parallel_quorum_insert_no_replicas', '1') ORDER BY tuple(); CREATE TABLE r2 ( key UInt64, value String ) -ENGINE = ReplicatedMergeTree('/clickhouse/01509_parallel_quorum_insert_no_replicas', '2') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01509_parallel_quorum_insert_no_replicas', '2') ORDER BY tuple(); SET insert_quorum_parallel=1; diff --git a/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql b/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql index f33c1534a44..2ee73865122 100644 --- a/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql +++ b/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql @@ -1,3 +1,3 @@ -CREATE TABLE mt (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01497/mt') +CREATE TABLE mt (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01497/mt') ORDER BY tuple() -- { serverError 36 } diff --git a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql index 0eb9f20ad91..925dd042435 100644 --- a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql +++ b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql @@ -4,9 +4,9 @@ DROP TABLE IF EXISTS quorum1 SYNC; DROP TABLE IF EXISTS quorum2 SYNC; DROP TABLE IF EXISTS quorum3 SYNC; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01513/sequence_consistency', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01513/sequence_consistency', '2') ORDER BY x PARTITION BY y; -CREATE TABLE quorum3(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01513/sequence_consistency', '3') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01513/sequence_consistency', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01513/sequence_consistency', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum3(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01513/sequence_consistency', '3') ORDER BY x PARTITION BY y; INSERT INTO quorum1 VALUES (1, '1990-11-15'); INSERT INTO quorum1 VALUES (2, '1990-11-15'); diff --git a/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql b/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql index d5fb25a9241..07fe00693d1 100644 --- a/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql +++ b/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql @@ -1,33 +1,33 @@ drop database if exists db_01530_atomic sync; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; drop database db_01530_atomic sync; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; drop database db_01530_atomic sync; set database_atomic_wait_for_drop_and_detach_synchronously=1; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; drop database db_01530_atomic; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; drop database db_01530_atomic; set database_atomic_wait_for_drop_and_detach_synchronously=0; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; drop database db_01530_atomic; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; -- { serverError 253; } +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; -- { serverError 253; } set database_atomic_wait_for_drop_and_detach_synchronously=1; diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql index 69369321d26..1acae560c93 100644 --- a/tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql +++ b/tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS execute_on_single_replica_r1 NO DELAY; DROP TABLE IF EXISTS execute_on_single_replica_r2 NO DELAY; -/* that test requires fixed zookeeper path */ +/* that test requires fixed zookeeper path, so we cannot use ReplicatedMergeTree({database}) */ CREATE TABLE execute_on_single_replica_r1 (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_01532/execute_on_single_replica', 'r1') ORDER BY tuple() SETTINGS execute_merges_on_single_replica_time_threshold=10; CREATE TABLE execute_on_single_replica_r2 (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_01532/execute_on_single_replica', 'r2') ORDER BY tuple() SETTINGS execute_merges_on_single_replica_time_threshold=10; @@ -124,4 +124,4 @@ ORDER BY part_name FORMAT Vertical; DROP TABLE execute_on_single_replica_r1 NO DELAY; -DROP TABLE execute_on_single_replica_r2 NO DELAY; \ No newline at end of file +DROP TABLE execute_on_single_replica_r2 NO DELAY; diff --git a/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql b/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql index 9779dca90a2..cc8c721b6a8 100644 --- a/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql +++ b/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql @@ -11,11 +11,11 @@ SET replication_alter_partitions_sync = 2; -- IRL insert_replica_id were filled from hostname CREATE TABLE IF NOT EXISTS replicated_deduplicate_by_columns_r1 ( id Int32, val UInt32, unique_value UInt64 MATERIALIZED rowNumberInBlock() -) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_01581/replicated_deduplicate', 'r1') ORDER BY id; +) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_01581/replicated_deduplicate', 'r1') ORDER BY id; CREATE TABLE IF NOT EXISTS replicated_deduplicate_by_columns_r2 ( id Int32, val UInt32, unique_value UInt64 MATERIALIZED rowNumberInBlock() -) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_01581/replicated_deduplicate', 'r2') ORDER BY id; +) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_01581/replicated_deduplicate', 'r2') ORDER BY id; -- insert some data, 2 records: (3, 1003), (4, 1004) are duplicated and have difference in unique_value / insert_replica_id diff --git a/tests/queries/0_stateless/01646_system_restart_replicas_smoke.sql b/tests/queries/0_stateless/01646_system_restart_replicas_smoke.sql index cfd70df8dd4..0b7fba5b02e 100644 --- a/tests/queries/0_stateless/01646_system_restart_replicas_smoke.sql +++ b/tests/queries/0_stateless/01646_system_restart_replicas_smoke.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS data_01646; -CREATE TABLE data_01646 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01646/data_01646', 'r') ORDER BY s PARTITION BY x; +CREATE TABLE data_01646 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01646/data_01646', 'r') ORDER BY s PARTITION BY x; SYSTEM RESTART REPLICAS; DESCRIBE TABLE data_01646; DROP TABLE data_01646; diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql index c3e459dfc49..a839883e683 100644 --- a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql +++ b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql @@ -5,7 +5,7 @@ CREATE TABLE partitioned_table ( partitioner UInt8, value String ) -ENGINE ReplicatedMergeTree('/clickhouse/01650_drop_part_and_deduplication_partitioned_table', '1') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/01650_drop_part_and_deduplication_partitioned_table', '1') ORDER BY key PARTITION BY partitioner; @@ -16,24 +16,24 @@ INSERT INTO partitioned_table VALUES (11, 1, 'AA'), (22, 2, 'BB'), (33, 3, 'CC') SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; -SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; +SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; INSERT INTO partitioned_table VALUES (33, 3, 'CC'); -- must be deduplicated SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; -SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; +SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; ALTER TABLE partitioned_table DROP PART '3_1_1_0'; SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; -SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; +SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; INSERT INTO partitioned_table VALUES (33, 3, 'CC'); -- mustn't be deduplicated SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; -SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; +SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; DROP TABLE IF EXISTS partitioned_table; diff --git a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql index 2c50a7a8516..85b37cee15a 100644 --- a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql +++ b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql @@ -2,17 +2,17 @@ DROP TABLE IF EXISTS test_01640; DROP TABLE IF EXISTS restore_01640; CREATE TABLE test_01640(i Int64, d Date, s String) -ENGINE = ReplicatedMergeTree('/clickhouse/{shard}/tables/test_01640','{replica}') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/{shard}/tables/test_01640','{replica}') PARTITION BY toYYYYMM(d) ORDER BY i; insert into test_01640 values (1, '2021-01-01','some'); CREATE TABLE restore_01640(i Int64, d Date, s String) -ENGINE = ReplicatedMergeTree('/clickhouse/{shard}/tables/restore_01640','{replica}') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/{shard}/tables/restore_01640','{replica}') PARTITION BY toYYYYMM(d) ORDER BY i; -ALTER TABLE restore_01640 FETCH PARTITION tuple(toYYYYMM(toDate('2021-01-01'))) - FROM '/clickhouse/{shard}/tables/test_01640'; +ALTER TABLE restore_01640 FETCH PARTITION tuple(toYYYYMM(toDate('2021-01-01'))) + FROM '/clickhouse/{database}/{shard}/tables/test_01640'; SELECT partition_id FROM system.detached_parts diff --git a/tests/queries/0_stateless/01666_blns.sql b/tests/queries/0_stateless/01666_blns.sql index be9632092bc..19caf45832f 100644 --- a/tests/queries/0_stateless/01666_blns.sql +++ b/tests/queries/0_stateless/01666_blns.sql @@ -554,9 +554,9 @@ SELECT count() FROM test; DROP TABLE IF EXISTS test_r1; DROP TABLE IF EXISTS test_r2; -CREATE TABLE test_r1 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test_01666', 'r1') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; +CREATE TABLE test_r1 AS test ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01666', 'r1') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; INSERT INTO test_r1 SELECT * FROM test; -CREATE TABLE test_r2 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test_01666', 'r2') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; +CREATE TABLE test_r2 AS test ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01666', 'r2') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; SYSTEM SYNC REPLICA test_r2; diff --git a/tests/queries/0_stateless/01669_columns_declaration_serde.sql b/tests/queries/0_stateless/01669_columns_declaration_serde.sql index a6bf1184e9f..b1a85754fce 100644 --- a/tests/queries/0_stateless/01669_columns_declaration_serde.sql +++ b/tests/queries/0_stateless/01669_columns_declaration_serde.sql @@ -22,12 +22,12 @@ DROP TABLE IF EXISTS test_r1; DROP TABLE IF EXISTS test_r2; CREATE TABLE test_r1 (x UInt64, "\\" String DEFAULT '\r\n\t\\' || ' -') ENGINE = ReplicatedMergeTree('/clickhouse/test_01669', 'r1') ORDER BY "\\"; +') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01669', 'r1') ORDER BY "\\"; INSERT INTO test_r1 ("\\") VALUES ('\\'); CREATE TABLE test_r2 (x UInt64, "\\" String DEFAULT '\r\n\t\\' || ' -') ENGINE = ReplicatedMergeTree('/clickhouse/test_01669', 'r2') ORDER BY "\\"; +') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01669', 'r2') ORDER BY "\\"; SYSTEM SYNC REPLICA test_r2; diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql index 02457a956a1..a147952dfe1 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql @@ -3,20 +3,20 @@ DROP TABLE IF EXISTS sample_table; CREATE TABLE sample_table ( key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/01700_system_zookeeper_path_in/{shard}', '{replica}') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/01700_system_zookeeper_path_in/{shard}', '{replica}') ORDER BY tuple(); -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1' AND name like 'block%' ORDER BY name; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1/replicas' AND name LIKE '%r1%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1' AND name like 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/replicas' AND name LIKE '%r1%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1') AND name LIKE 'block%' ORDER BY name; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE '%r1%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1') AND name LIKE 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE '%r1%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1', - '/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1', + '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE 'block%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/01700_system_zookeeper_path_in/s1/', name) - FROM system.zookeeper WHERE (name != 'replicas' AND name NOT LIKE 'leader_election%' AND path = '/clickhouse/01700_system_zookeeper_path_in/s1')) ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/', name) + FROM system.zookeeper WHERE (name != 'replicas' AND name NOT LIKE 'leader_election%' AND path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1')) ORDER BY name; DROP TABLE IF EXISTS sample_table; diff --git a/tests/queries/0_stateless/01710_projection_fetch.reference b/tests/queries/0_stateless/01710_projection_fetch.reference index abce5410b26..6fc199d6ba2 100644 --- a/tests/queries/0_stateless/01710_projection_fetch.reference +++ b/tests/queries/0_stateless/01710_projection_fetch.reference @@ -10,8 +10,8 @@ 3 3 4 4 0 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 2 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 diff --git a/tests/queries/0_stateless/01710_projection_fetch.sql b/tests/queries/0_stateless/01710_projection_fetch.sql index 7e4f6cc1d9a..5781d63788f 100644 --- a/tests/queries/0_stateless/01710_projection_fetch.sql +++ b/tests/queries/0_stateless/01710_projection_fetch.sql @@ -1,9 +1,9 @@ drop table if exists tp_1; drop table if exists tp_2; -create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '1_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; +create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/{shard}/01710_projection_fetch_' || currentDatabase(), '1_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; -create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '2_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; +create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/{shard}/01710_projection_fetch_' || currentDatabase(), '2_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; insert into tp_1 select number, number from numbers(3); 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 7d4c83c9d3a..71898bdedcb 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 @@ -5,7 +5,7 @@ CREATE TABLE ttl_table date Date, value UInt64 ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01713_table_ttl', '1', date, date, 8192) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01713_table_ttl', '1', date, date, 8192) TTL date + INTERVAL 2 MONTH; --{ serverError 36 } CREATE TABLE ttl_table @@ -13,7 +13,7 @@ CREATE TABLE ttl_table date Date, value UInt64 ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01713_table_ttl', '1', date, date, 8192) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01713_table_ttl', '1', date, date, 8192) PARTITION BY date; --{ serverError 42 } CREATE TABLE ttl_table @@ -21,7 +21,7 @@ CREATE TABLE ttl_table date Date, value UInt64 ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01713_table_ttl', '1', date, date, 8192) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01713_table_ttl', '1', date, date, 8192) ORDER BY value; --{ serverError 42 } SELECT 1; diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql index 66b53369517..c0335ae3753 100644 --- a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql +++ b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql @@ -2,11 +2,11 @@ DROP TABLE IF EXISTS i20203_1; DROP TABLE IF EXISTS i20203_2; CREATE TABLE i20203_1 (a Int8) -ENGINE = ReplicatedMergeTree('/clickhouse/01715_background_checker_i20203', 'r1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01715_background_checker_i20203', 'r1') ORDER BY tuple(); CREATE TABLE i20203_2 (a Int8) -ENGINE = ReplicatedMergeTree('/clickhouse/01715_background_checker_i20203', 'r2') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01715_background_checker_i20203', 'r2') ORDER BY tuple(); DETACH TABLE i20203_2; diff --git a/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql index 759c8ba3a0b..033a66a477f 100644 --- a/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql +++ b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql @@ -39,7 +39,7 @@ CREATE TABLE replicated_report `branch` String, `generated_time` DateTime ) -ENGINE = ReplicatedMergeTree('/clickhouse/01747_alter_partition_key/t', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01747_alter_partition_key/t', '1') PARTITION BY (product, toYYYYMM(generated_time)) ORDER BY (product, machine, branch, generated_time); diff --git a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference b/tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference index 5dcc95fd7b7..ea3f608b6c7 100644 --- a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference +++ b/tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference @@ -1,9 +1,9 @@ 1 5.00000000 2 6.00000000 -CREATE TABLE default.test_alter_decimal\n(\n `n` UInt64,\n `d` Decimal(18, 8)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/01761_alter_decimal_zookeeper\', \'r1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.test_alter_decimal\n(\n `n` UInt64,\n `d` Decimal(18, 8)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/01761_alter_decimal_zookeeper\', \'r1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1 5.00000000 2 6.00000000 -CREATE TABLE default.test_alter_decimal\n(\n `n` UInt64,\n `d` Decimal(18, 8)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/01761_alter_decimal_zookeeper\', \'r1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.test_alter_decimal\n(\n `n` UInt64,\n `d` Decimal(18, 8)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/01761_alter_decimal_zookeeper\', \'r1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1 5.00000000 2 6.00000000 3 7.00000000 diff --git a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql b/tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql index 01766f0d6c2..d1bd3b65182 100644 --- a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql +++ b/tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS test_alter_decimal; CREATE TABLE test_alter_decimal (n UInt64, d Decimal(15, 8)) -ENGINE = ReplicatedMergeTree('/clickhouse/01761_alter_decimal_zookeeper', 'r1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01761_alter_decimal_zookeeper', 'r1') ORDER BY tuple(); INSERT INTO test_alter_decimal VALUES (1, toDecimal32(5, 5)); diff --git a/tests/queries/0_stateless/01901_test_attach_partition_from.sql b/tests/queries/0_stateless/01901_test_attach_partition_from.sql index 0ffa4cdecf2..c15edacc6c8 100644 --- a/tests/queries/0_stateless/01901_test_attach_partition_from.sql +++ b/tests/queries/0_stateless/01901_test_attach_partition_from.sql @@ -5,7 +5,7 @@ CREATE TABLE test_alter_attach_01901S (A Int64, D date) ENGINE = MergeTree PARTI INSERT INTO test_alter_attach_01901S VALUES (1, '2020-01-01'); CREATE TABLE test_alter_attach_01901D (A Int64, D date) -Engine=ReplicatedMergeTree('/clickhouse/tables/test_alter_attach_01901D', 'r1') +Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/test_alter_attach_01901D', 'r1') PARTITION BY D ORDER BY A; ALTER TABLE test_alter_attach_01901D ATTACH PARTITION '2020-01-01' FROM test_alter_attach_01901S; From c806534eeed384cd47106ee7b4b5b1b75dda8aad Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 10:28:43 +0300 Subject: [PATCH 053/147] Update parallel skip_list --- tests/queries/skip_list.json | 29 ----------------------------- 1 file changed, 29 deletions(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index d40d9a940d0..5a4c982e13d 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -191,29 +191,23 @@ [ /// Pessimistic list of tests which work badly in parallel. /// Probably they need better investigation. - "00062_replicated_merge_tree_alter_zookeeper", "00080_show_tables_and_system_tables", "00101_materialized_views_and_insert_without_explicit_database", "00109_shard_totals_after_having", "00110_external_sort", "00116_storage_set", - "00121_drop_column_zookeeper", "00133_long_shard_memory_tracker_and_exception_safety", "00158_buffer_and_nonexistent_table", "00180_attach_materialized_view", - "00226_zookeeper_deduplication_and_unexpected_parts", - "00236_replicated_drop_on_non_leader_zookeeper", "00305_http_and_readonly", "00311_array_primary_key", "00417_kill_query", "00423_storage_log_single_thread", "00429_long_http_bufferization", "00446_clear_column_in_partition_concurrent_zookeeper", - "00446_clear_column_in_partition_zookeeper", "00463_long_sessions_in_http_interface", "00505_shard_secure", "00508_materialized_view_to", - "00516_deduplication_after_drop_partition_zookeeper", "00534_functions_bad_arguments10", "00552_or_nullable", "00564_versioned_collapsing_merge_tree", @@ -231,7 +225,6 @@ "00626_replace_partition_from_table", "00626_replace_partition_from_table_zookeeper", "00633_materialized_view_and_too_many_parts_zookeeper", - "00643_cast_zookeeper", "00652_mergetree_mutations", "00652_replicated_mutations_zookeeper", "00682_empty_parts_merge", @@ -247,7 +240,6 @@ "00751_default_databasename_for_view", "00753_alter_attach", "00754_alter_modify_column_partitions", - "00754_alter_modify_order_by_replicated_zookeeper", "00763_long_lock_buffer_alter_destination_table", "00800_versatile_storage_join", "00804_test_alter_compression_codecs", @@ -262,22 +254,18 @@ "00857_global_joinsavel_table_alias", "00899_long_attach_memory_limit", "00910_buffer_prewhere", - "00910_zookeeper_custom_compression_codecs_replicated", "00926_adaptive_index_granularity_merge_tree", "00926_adaptive_index_granularity_pk", "00926_adaptive_index_granularity_replacing_merge_tree", - "00926_zookeeper_adaptive_index_granularity_replicated_merge_tree", "00933_alter_ttl", "00933_reserved_word", "00933_test_fix_extra_seek_on_compressed_cache", - "00933_ttl_replicated_zookeeper", "00933_ttl_with_default", "00950_dict_get", "00955_test_final_mark", "00976_ttl_with_old_parts", "00980_merge_alter_settings", "00980_zookeeper_merge_tree_alter_settings", - "00988_constraints_replication_zookeeper", "00989_parallel_parts_loading", "00992_system_parts_race_condition_zookeeper_long", "00993_system_parts_race_condition_drop_zookeeper", @@ -349,7 +337,6 @@ "01125_dict_ddl_cannot_add_column", "01127_month_partitioning_consistency_select", "01130_in_memory_parts_partitons", - "01135_default_and_alter_zookeeper", "01148_zookeeper_path_macros_unfolding", "01150_ddl_guard_rwr", "01153_attach_mv_uuid", @@ -374,11 +361,9 @@ "01254_dict_load_after_detach_attach", "01257_dictionary_mismatch_types", "01259_dictionary_custom_settings_ddl", - "01267_alter_default_key_columns_zookeeper", "01268_dictionary_direct_layout", "01269_alias_type_differs", "01272_suspicious_codecs", - "01277_alter_rename_column_constraint_zookeeper", "01280_ssd_complex_key_dictionary", "01280_ttl_where_group_by", "01281_group_by_limit_memory_tracking", @@ -392,7 +377,6 @@ "01305_replica_create_drop_zookeeper", "01307_multiple_leaders_zookeeper", "01318_long_unsuccessful_mutation_zookeeper", - "01319_manual_write_to_replicas", "01320_create_sync_race_condition_zookeeper", "01338_long_select_and_alter", "01338_long_select_and_alter_zookeeper", @@ -401,7 +385,6 @@ "01357_version_collapsing_attach_detach_zookeeper", "01375_compact_parts_codecs", "01376_GROUP_BY_injective_elimination_dictGet", - "01378_alter_rename_with_ttl_zookeeper", "01383_remote_ambiguous_column_shard", "01388_clear_all_columns", "01391_join_on_dict_crash", @@ -413,7 +396,6 @@ "01415_sticking_mutations", "01417_freeze_partition_verbose", "01417_freeze_partition_verbose_zookeeper", - "01430_modify_sample_by_zookeeper", "01444_create_table_drop_database_race", "01454_storagememory_data_race_challenge", "01455_rank_correlation_spearman", @@ -425,7 +407,6 @@ "01470_show_databases_like", "01471_calculate_ttl_during_merge", "01487_distributed_in_not_default_db", - "01493_alter_remove_properties_zookeeper", "01493_storage_set_persistency", "01494_storage_join_persistency", "01501_cache_dictionary_all_fields", @@ -437,7 +418,6 @@ "01526_complex_key_dict_direct_layout", "01527_clickhouse_local_optimize", "01527_dist_sharding_key_dictGet_reload", - "01530_drop_database_atomic_sync", "01541_max_memory_usage_for_user_long", "01542_dictionary_load_exception_race", "01545_system_errors", // looks at the difference of values in system.errors @@ -453,30 +433,23 @@ "01601_detach_permanently", "01602_show_create_view", "01603_rename_overwrite_bug", - "01666_blns", "01646_system_restart_replicas_smoke", // system restart replicas is a global query "01656_test_query_log_factories_info", "01658_read_file_to_stringcolumn", - "01669_columns_declaration_serde", "01676_dictget_in_default_expression", "01681_cache_dictionary_simple_key", "01682_cache_dictionary_complex_key", "01683_flat_dictionary", "01684_ssd_cache_dictionary_simple_key", "01685_ssd_cache_dictionary_complex_key", - "01700_system_zookeeper_path_in", "01702_system_query_log", // It's ok to execute in parallel with oter tests but not several instances of the same test. "01702_system_query_log", // Runs many global system queries - "01715_background_checker_blather_zookeeper", "01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test. "01722_long_brotli_http_compression_json_format", // It is broken in some unimaginable way with the genius error 'cannot write to ofstream'. Not sure how to debug this - "01747_alter_partition_key_enum_zookeeper", "01748_dictionary_table_dot", // creates database "01760_polygon_dictionaries", "01760_system_dictionaries", - "01761_alter_decimal_zookeeper", "01360_materialized_view_with_join_on_query_log", // creates and drops MVs on query_log, which may interrupt flushes. - "01509_parallel_quorum_insert_no_replicas", // It's ok to execute in parallel with oter tests but not several instances of the same test. "attach", "ddl_dictionaries", "dictionary", @@ -508,7 +481,6 @@ "01804_dictionary_decimal256_type", "01850_dist_INSERT_preserve_error", // uses cluster with different static databases shard_0/shard_1 "01821_table_comment", - "01710_projection_fetch", "01824_prefer_global_in_and_join", "01870_modulo_partition_key", "01870_buffer_flush", // creates database @@ -526,7 +498,6 @@ "01902_table_function_merge_db_repr", "01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer", "01946_test_wrong_host_name_access", - "01213_alter_rename_with_default_zookeeper", /// Warning: Removing leftovers from table. "02001_add_default_database_to_system_users" ///create user ] } From 8e1e53fb68509e32adcfb7c8fc8edf61b6e3e136 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 5 Aug 2021 14:14:50 +0300 Subject: [PATCH 054/147] fix --- src/Interpreters/InterpreterCreateQuery.cpp | 16 +++++++++++++--- src/Interpreters/InterpreterRenameQuery.cpp | 2 +- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 066258af73b..bf2cf6338aa 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1156,10 +1156,20 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, }; ast_rename->elements.push_back(std::move(elem)); - ast_rename->exchange = true; ast_rename->dictionary = create.is_dictionary; - /// Will execute ordinary RENAME instead of EXCHANGE if the target table does not exist - ast_rename->rename_if_cannot_exchange = create.create_or_replace; + if (create.create_or_replace) + { + /// CREATE OR REPLACE TABLE + /// Will execute ordinary RENAME instead of EXCHANGE if the target table does not exist + ast_rename->rename_if_cannot_exchange = true; + ast_rename->exchange = false; + } + else + { + /// REPLACE TABLE + /// Will execute EXCHANGE query and fail if the target table does not exist + ast_rename->exchange = true; + } InterpreterRenameQuery interpreter_rename{ast_rename, current_context}; interpreter_rename.execute(); diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index bfd03028c91..e3d52487a52 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -73,7 +73,7 @@ BlockIO InterpreterRenameQuery::execute() BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions, TableGuards & ddl_guards) { assert(!rename.rename_if_cannot_exchange || descriptions.size() == 1); - assert(!rename.rename_if_cannot_exchange || rename.exchange); + assert(!(rename.rename_if_cannot_exchange && rename.exchange)); auto & database_catalog = DatabaseCatalog::instance(); for (const auto & elem : descriptions) From 04da62707ed0748fbaedf4f466c047f67c430625 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 5 Aug 2021 14:34:13 +0300 Subject: [PATCH 055/147] more careful handling of reconnects in fuzzer --- programs/client/Client.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 6ed6d5404a3..47c2fed8686 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1449,7 +1449,12 @@ private: "Error while reconnecting to the server: {}\n", getCurrentExceptionMessage(true)); - assert(!connection->isConnected()); + // The reconnection might fail, but we'll still be connected + // in the sense of `connection->isConnected() = true`, + // in case when the requested database doesn't exist. + // Disconnect manually now, so that the following code doesn't + // have any doubts, and the connection state is predictable. + connection->disconnect(); } } From 0f832fcf8bdcaf0b6cb564914fe7ab693dd27ac6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Aug 2021 14:45:38 +0300 Subject: [PATCH 056/147] Add missing file --- src/Common/clickhouse_malloc.cpp | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) create mode 100644 src/Common/clickhouse_malloc.cpp diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp new file mode 100644 index 00000000000..5e37c8b3a2d --- /dev/null +++ b/src/Common/clickhouse_malloc.cpp @@ -0,0 +1,29 @@ +#if 0 + +#include +#include + + +extern "C" void * clickhouse_malloc(size_t size) +{ + void * res = malloc(size); + if (res) + Memory::trackMemory(size); + return res; +} + +extern "C" void clickhouse_free(void * ptr) +{ + Memory::untrackMemory(ptr); + free(ptr); +} + +extern "C" int clickhouse_posix_memalign(void ** memptr, size_t alignment, size_t size) +{ + int res = posix_memalign(memptr, alignment, size); + if (res == 0) + Memory::trackMemory(size); + return res; +} + +#endif From 1ae59b6ccee0656abf21dfa8f55fd33645998e22 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 5 Aug 2021 17:05:03 +0300 Subject: [PATCH 057/147] DatabaseReplicatedWorker logs_to_keep race fix --- src/Databases/DatabaseReplicatedWorker.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 4020906f9b2..773612e403c 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -43,7 +43,7 @@ private: mutable std::mutex mutex; std::condition_variable wait_current_task_change; String current_task; - UInt32 logs_to_keep = std::numeric_limits::max(); + std::atomic logs_to_keep = std::numeric_limits::max(); }; } From fc12310ee802d977839a3ece2b8f9ce46ef3913c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Aug 2021 17:17:22 +0300 Subject: [PATCH 058/147] Revert #24095. User-level settings will affect queries from view. --- src/Storages/StorageView.cpp | 7 +--- .../01866_view_persist_settings.reference | 38 ++++++++++++++++--- .../01866_view_persist_settings.sql | 36 ++++++++++++++++++ 3 files changed, 70 insertions(+), 11 deletions(-) diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 5119c5b121d..6d4fbc1dff3 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -86,12 +86,7 @@ void StorageView::read( current_inner_query = query_info.view_query->clone(); } - auto modified_context = Context::createCopy(context); - /// Use settings from global context, - /// because difference between settings set on VIEW creation and query execution can break queries - modified_context->setSettings(context->getGlobalContext()->getSettingsRef()); - - InterpreterSelectWithUnionQuery interpreter(current_inner_query, modified_context, {}, column_names); + InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names); interpreter.buildQueryPlan(query_plan); /// It's expected that the columns read from storage are not constant. diff --git a/tests/queries/0_stateless/01866_view_persist_settings.reference b/tests/queries/0_stateless/01866_view_persist_settings.reference index 07c96e76875..4d7e1356807 100644 --- a/tests/queries/0_stateless/01866_view_persist_settings.reference +++ b/tests/queries/0_stateless/01866_view_persist_settings.reference @@ -1,8 +1,5 @@ join_use_nulls = 1 - -1 11 0 -2 12 22 -3 0 23 - 1 11 0 2 12 22 @@ -12,9 +9,40 @@ join_use_nulls = 1 2 12 22 3 \N 23 - -1 11 0 +1 11 \N 2 12 22 -3 0 23 +3 \N 23 +join_use_nulls = 0 +- +1 11 0 +2 12 22 +3 0 23 +- +1 11 0 +2 12 22 +3 0 23 +- +1 11 \N +2 12 22 +3 \N 23 +- +1 11 0 +2 12 22 +3 0 23 +join_use_nulls = 1 +- +- +1 11 0 +2 12 22 +3 0 23 +- +1 11 \N +2 12 22 +3 \N 23 +- +1 11 \N +2 12 22 +3 \N 23 join_use_nulls = 0 - 1 11 0 diff --git a/tests/queries/0_stateless/01866_view_persist_settings.sql b/tests/queries/0_stateless/01866_view_persist_settings.sql index 71d6c856b9b..74c94ec6369 100644 --- a/tests/queries/0_stateless/01866_view_persist_settings.sql +++ b/tests/queries/0_stateless/01866_view_persist_settings.sql @@ -33,6 +33,19 @@ SET join_use_nulls = 1; SELECT 'join_use_nulls = 1'; +SELECT '-'; +SELECT * FROM view_no_nulls; -- { serverError 349 } +SELECT '-'; +SELECT * FROM view_no_nulls_set; +SELECT '-'; +SELECT * FROM view_nulls_set; +SELECT '-'; +SELECT * FROM view_nulls; + +SET join_use_nulls = 0; + +SELECT 'join_use_nulls = 0'; + SELECT '-'; SELECT * FROM view_no_nulls; SELECT '-'; @@ -42,6 +55,29 @@ SELECT * FROM view_nulls_set; SELECT '-'; SELECT * FROM view_nulls; +DETACH TABLE view_no_nulls; +DETACH TABLE view_no_nulls_set; +DETACH TABLE view_nulls_set; +DETACH TABLE view_nulls; + +ATTACH TABLE view_no_nulls; +ATTACH TABLE view_no_nulls_set; +ATTACH TABLE view_nulls_set; +ATTACH TABLE view_nulls; + +SET join_use_nulls = 1; + +SELECT 'join_use_nulls = 1'; + +SELECT '-'; +SELECT * FROM view_no_nulls; -- { serverError 349 } +SELECT '-'; +SELECT * FROM view_no_nulls_set; +SELECT '-'; +SELECT * FROM view_nulls_set; +SELECT '-'; +SELECT * FROM view_nulls; + SET join_use_nulls = 0; SELECT 'join_use_nulls = 0'; From a427b19d3a9d482feeda84223a64eb4f5af57044 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 5 Aug 2021 18:38:52 +0300 Subject: [PATCH 059/147] Check join_use_nulls changed in StorageView::read --- src/Storages/StorageView.cpp | 42 ++++++++++++++++++- .../01866_view_persist_settings.sql | 4 +- 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 6d4fbc1dff3..a0ab532ece0 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -29,6 +30,36 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + +bool isNullableOrLcNullable(DataTypePtr type) +{ + if (type->isNullable()) + return true; + + if (const auto * lc_type = typeid_cast(type.get())) + return lc_type->getDictionaryType()->isNullable(); + + return false; +} + +/// Returns `true` if there are nullable column in src but corresponding column in dst is not +bool changedNullabilityOneWay(const Block & src_block, const Block & dst_block) +{ + std::unordered_map src_nullable; + for (const auto & col : src_block) + src_nullable[col.name] = isNullableOrLcNullable(col.type); + + for (const auto & col : dst_block) + { + if (!isNullableOrLcNullable(col.type) && src_nullable[col.name]) + return true; + } + return false; +} + +} StorageView::StorageView( const StorageID & table_id_, @@ -99,10 +130,17 @@ void StorageView::read( query_plan.addStep(std::move(materializing)); /// And also convert to expected structure. - auto header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); + const auto & expected_header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); + const auto & header = query_plan.getCurrentDataStream().header; + + if (changedNullabilityOneWay(header, expected_header)) + { + throw DB::Exception(ErrorCodes::INCORRECT_QUERY, + "Joined columns is nullable, but setting `join_use_nulls` wans't set on CREATE VIEW"); + } auto convert_actions_dag = ActionsDAG::makeConvertingActions( - query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), + expected_header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); auto converting = std::make_unique(query_plan.getCurrentDataStream(), convert_actions_dag); diff --git a/tests/queries/0_stateless/01866_view_persist_settings.sql b/tests/queries/0_stateless/01866_view_persist_settings.sql index 74c94ec6369..c58b802494d 100644 --- a/tests/queries/0_stateless/01866_view_persist_settings.sql +++ b/tests/queries/0_stateless/01866_view_persist_settings.sql @@ -34,7 +34,7 @@ SET join_use_nulls = 1; SELECT 'join_use_nulls = 1'; SELECT '-'; -SELECT * FROM view_no_nulls; -- { serverError 349 } +SELECT * FROM view_no_nulls; -- { serverError 80 } SELECT '-'; SELECT * FROM view_no_nulls_set; SELECT '-'; @@ -70,7 +70,7 @@ SET join_use_nulls = 1; SELECT 'join_use_nulls = 1'; SELECT '-'; -SELECT * FROM view_no_nulls; -- { serverError 349 } +SELECT * FROM view_no_nulls; -- { serverError 80 } SELECT '-'; SELECT * FROM view_no_nulls_set; SELECT '-'; From 0e040ca40c7b5840b3129c533a34b948b9be8925 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 5 Aug 2021 12:35:46 +0300 Subject: [PATCH 060/147] Add 10 seconds cache for S3 proxy resolver --- src/Disks/S3/ProxyConfiguration.h | 1 + src/Disks/S3/ProxyListConfiguration.h | 1 + src/Disks/S3/ProxyResolverConfiguration.cpp | 48 +++++++++++++++---- src/Disks/S3/ProxyResolverConfiguration.h | 11 ++++- src/Disks/S3/registerDiskS3.cpp | 7 ++- src/IO/S3/PocoHTTPClient.cpp | 3 ++ src/IO/S3/PocoHTTPClient.h | 3 ++ .../configs/config.d/storage_conf.xml | 1 + 8 files changed, 65 insertions(+), 10 deletions(-) diff --git a/src/Disks/S3/ProxyConfiguration.h b/src/Disks/S3/ProxyConfiguration.h index 888f4c6faf9..793170e727c 100644 --- a/src/Disks/S3/ProxyConfiguration.h +++ b/src/Disks/S3/ProxyConfiguration.h @@ -19,6 +19,7 @@ public: virtual ~ProxyConfiguration() = default; /// Returns proxy configuration on each HTTP request. virtual Aws::Client::ClientConfigurationPerRequest getConfiguration(const Aws::Http::HttpRequest & request) = 0; + virtual void errorReport(const Aws::Client::ClientConfigurationPerRequest & config) = 0; }; } diff --git a/src/Disks/S3/ProxyListConfiguration.h b/src/Disks/S3/ProxyListConfiguration.h index 4d37d2b6d69..bd5bbba19a4 100644 --- a/src/Disks/S3/ProxyListConfiguration.h +++ b/src/Disks/S3/ProxyListConfiguration.h @@ -20,6 +20,7 @@ class ProxyListConfiguration : public ProxyConfiguration public: explicit ProxyListConfiguration(std::vector proxies_); Aws::Client::ClientConfigurationPerRequest getConfiguration(const Aws::Http::HttpRequest & request) override; + void errorReport(const Aws::Client::ClientConfigurationPerRequest &) override {} private: /// List of configured proxies. diff --git a/src/Disks/S3/ProxyResolverConfiguration.cpp b/src/Disks/S3/ProxyResolverConfiguration.cpp index b959d8b4415..17dd19fe444 100644 --- a/src/Disks/S3/ProxyResolverConfiguration.cpp +++ b/src/Disks/S3/ProxyResolverConfiguration.cpp @@ -16,8 +16,10 @@ namespace DB::ErrorCodes namespace DB::S3 { -ProxyResolverConfiguration::ProxyResolverConfiguration(const Poco::URI & endpoint_, String proxy_scheme_, unsigned proxy_port_) - : endpoint(endpoint_), proxy_scheme(std::move(proxy_scheme_)), proxy_port(proxy_port_) + +ProxyResolverConfiguration::ProxyResolverConfiguration(const Poco::URI & endpoint_, String proxy_scheme_ + , unsigned proxy_port_, unsigned cache_ttl_) + : endpoint(endpoint_), proxy_scheme(std::move(proxy_scheme_)), proxy_port(proxy_port_), cache_ttl(cache_ttl_) { } @@ -25,16 +27,25 @@ Aws::Client::ClientConfigurationPerRequest ProxyResolverConfiguration::getConfig { LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Obtain proxy using resolver: {}", endpoint.toString()); + std::unique_lock lock(cache_mutex); + + std::chrono::time_point now = std::chrono::system_clock::now(); + + if (cache_ttl.count() && cache_valid && now <= cache_timestamp + cache_ttl && now >= cache_timestamp) + { + LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Use cached proxy: {}://{}:{}", Aws::Http::SchemeMapper::ToString(cached_config.proxyScheme), cached_config.proxyHost, cached_config.proxyPort); + return cached_config; + } + /// 1 second is enough for now. /// TODO: Make timeouts configurable. ConnectionTimeouts timeouts( Poco::Timespan(1000000), /// Connection timeout. Poco::Timespan(1000000), /// Send timeout. - Poco::Timespan(1000000) /// Receive timeout. + Poco::Timespan(1000000) /// Receive timeout. ); auto session = makeHTTPSession(endpoint, timeouts); - Aws::Client::ClientConfigurationPerRequest cfg; try { /// It should be just empty GET request. @@ -53,20 +64,41 @@ Aws::Client::ClientConfigurationPerRequest ProxyResolverConfiguration::getConfig LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Use proxy: {}://{}:{}", proxy_scheme, proxy_host, proxy_port); - cfg.proxyScheme = Aws::Http::SchemeMapper::FromString(proxy_scheme.c_str()); - cfg.proxyHost = proxy_host; - cfg.proxyPort = proxy_port; + cached_config.proxyScheme = Aws::Http::SchemeMapper::FromString(proxy_scheme.c_str()); + cached_config.proxyHost = proxy_host; + cached_config.proxyPort = proxy_port; + cache_timestamp = std::chrono::system_clock::now(); + cache_valid = true; - return cfg; + return cached_config; } catch (...) { tryLogCurrentException("AWSClient", "Failed to obtain proxy"); /// Don't use proxy if it can't be obtained. + Aws::Client::ClientConfigurationPerRequest cfg; return cfg; } } +void ProxyResolverConfiguration::errorReport(const Aws::Client::ClientConfigurationPerRequest & config) +{ + if (config.proxyHost.empty()) + return; + + std::unique_lock lock(cache_mutex); + + if (!cache_ttl.count() || !cache_valid) + return; + + if (cached_config.proxyScheme != config.proxyScheme || cached_config.proxyHost != config.proxyHost + || cached_config.proxyPort != config.proxyPort) + return; + + /// Invalidate cached proxy when got error with this proxy + cache_valid = false; +} + } #endif diff --git a/src/Disks/S3/ProxyResolverConfiguration.h b/src/Disks/S3/ProxyResolverConfiguration.h index 8eea662f257..f7eba8d028a 100644 --- a/src/Disks/S3/ProxyResolverConfiguration.h +++ b/src/Disks/S3/ProxyResolverConfiguration.h @@ -8,6 +8,8 @@ #include "ProxyConfiguration.h" +#include + namespace DB::S3 { /** @@ -18,8 +20,9 @@ namespace DB::S3 class ProxyResolverConfiguration : public ProxyConfiguration { public: - ProxyResolverConfiguration(const Poco::URI & endpoint_, String proxy_scheme_, unsigned proxy_port_); + ProxyResolverConfiguration(const Poco::URI & endpoint_, String proxy_scheme_, unsigned proxy_port_, unsigned cache_ttl_); Aws::Client::ClientConfigurationPerRequest getConfiguration(const Aws::Http::HttpRequest & request) override; + void errorReport(const Aws::Client::ClientConfigurationPerRequest & config) override; private: /// Endpoint to obtain a proxy host. @@ -28,6 +31,12 @@ private: const String proxy_scheme; /// Port for obtained proxy. const unsigned proxy_port; + + std::mutex cache_mutex; + bool cache_valid = false; + std::chrono::time_point cache_timestamp; + const std::chrono::seconds cache_ttl{0}; + Aws::Client::ClientConfigurationPerRequest cached_config; }; } diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 49a11b1dbb9..01b2cea2045 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -56,11 +56,12 @@ std::shared_ptr getProxyResolverConfiguration( if (proxy_scheme != "http" && proxy_scheme != "https") throw Exception("Only HTTP/HTTPS schemas allowed in proxy resolver config: " + proxy_scheme, ErrorCodes::BAD_ARGUMENTS); auto proxy_port = proxy_resolver_config.getUInt(prefix + ".proxy_port"); + auto cache_ttl = proxy_resolver_config.getUInt(prefix + ".proxy_cache_time", 10); LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Configured proxy resolver: {}, Scheme: {}, Port: {}", endpoint.toString(), proxy_scheme, proxy_port); - return std::make_shared(endpoint, proxy_scheme, proxy_port); + return std::make_shared(endpoint, proxy_scheme, proxy_port, cache_ttl); } std::shared_ptr getProxyListConfiguration( @@ -128,8 +129,12 @@ getClient(const Poco::Util::AbstractConfiguration & config, const String & confi auto proxy_config = getProxyConfiguration(config_prefix, config); if (proxy_config) + { client_configuration.perRequestConfiguration = [proxy_config](const auto & request) { return proxy_config->getConfiguration(request); }; + client_configuration.error_report + = [proxy_config](const auto & request_config) { proxy_config->errorReport(request_config); }; + } client_configuration.retryStrategy = std::make_shared(config.getUInt(config_prefix + ".retry_attempts", 10)); diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 618d9ab7661..2eb208e2587 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -89,6 +89,7 @@ void PocoHTTPClientConfiguration::updateSchemeAndRegion() PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & clientConfiguration) : per_request_configuration(clientConfiguration.perRequestConfiguration) + , error_report(clientConfiguration.error_report) , timeouts(ConnectionTimeouts( Poco::Timespan(clientConfiguration.connectTimeoutMs * 1000), /// connection timeout. Poco::Timespan(clientConfiguration.requestTimeoutMs * 1000), /// send timeout. @@ -296,6 +297,8 @@ void PocoHTTPClient::makeRequestInternal( else if (status_code >= 300) { ProfileEvents::increment(select_metric(S3MetricType::Errors)); + if (status_code >= 500) + error_report(request_configuration); } response->SetResponseBody(response_body_stream, session); diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index e374863cf00..12f5af60ed4 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -37,6 +37,8 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration void updateSchemeAndRegion(); + std::function error_report; + private: PocoHTTPClientConfiguration(const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_); @@ -95,6 +97,7 @@ private: Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; std::function per_request_configuration; + std::function error_report; ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; diff --git a/tests/integration/test_s3_with_proxy/configs/config.d/storage_conf.xml b/tests/integration/test_s3_with_proxy/configs/config.d/storage_conf.xml index ccae67c7c09..a8d36a53bd5 100644 --- a/tests/integration/test_s3_with_proxy/configs/config.d/storage_conf.xml +++ b/tests/integration/test_s3_with_proxy/configs/config.d/storage_conf.xml @@ -26,6 +26,7 @@ http://resolver:8080/hostname http 80 + 10 From 8b9778ebb63c182956fcc985de31f43420f4d8b0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 5 Aug 2021 18:52:00 +0200 Subject: [PATCH 061/147] Shadow lambda arguments In case of ambiguity, lambda functions should prefer its arguments to other aliases or identifiers --- src/Interpreters/QueryNormalizer.cpp | 24 ++++++++++++++++++ src/Interpreters/QueryNormalizer.h | 4 +-- .../01960_lambda_precedence.reference | 3 +++ .../0_stateless/01960_lambda_precedence.sql | 25 +++++++++++++++++++ 4 files changed, 54 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01960_lambda_precedence.reference create mode 100644 tests/queries/0_stateless/01960_lambda_precedence.sql diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index aae714198b5..ea61ade2b49 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -170,6 +171,24 @@ void QueryNormalizer::visitChildren(IAST * node, Data & data) /// Don't go into query argument. return; } + + /// For lambda functions we need to avoid replacing lambda parameters with external aliases, for example, + /// Select 1 as x, arrayMap(x -> x + 2, [1, 2, 3]) + /// shouldn't be replaced with Select 1 as x, arrayMap(x -> **(1 as x)** + 2, [1, 2, 3]) + Aliases extracted_aliases; + if (func_node->name == "lambda") + { + Names lambda_aliases = RequiredSourceColumnsMatcher::extractNamesFromLambda(*func_node); + for (const auto & name : lambda_aliases) + { + auto it = data.aliases.find(name); + if (it != data.aliases.end()) + { + extracted_aliases.insert(data.aliases.extract(it)); + } + } + } + /// We skip the first argument. We also assume that the lambda function can not have parameters. size_t first_pos = 0; if (func_node->name == "lambda") @@ -192,6 +211,11 @@ void QueryNormalizer::visitChildren(IAST * node, Data & data) { visitChildren(func_node->window_definition.get(), data); } + + for (auto & it : extracted_aliases) + { + data.aliases.insert(it); + } } else if (!node->as()) { diff --git a/src/Interpreters/QueryNormalizer.h b/src/Interpreters/QueryNormalizer.h index 7fc0f4bdf82..eebcff62cde 100644 --- a/src/Interpreters/QueryNormalizer.h +++ b/src/Interpreters/QueryNormalizer.h @@ -39,7 +39,7 @@ public: using SetOfASTs = std::set; using MapOfASTs = std::map; - const Aliases & aliases; + Aliases & aliases; const NameSet & source_columns_set; ExtractedSettings settings; @@ -53,7 +53,7 @@ public: /// It's Ok to have "c + 1 AS c" in queries, but not in table definition const bool allow_self_aliases; /// for constructs like "SELECT column + 1 AS column" - Data(const Aliases & aliases_, const NameSet & source_columns_set_, bool ignore_alias_, ExtractedSettings && settings_, bool allow_self_aliases_) + Data(Aliases & aliases_, const NameSet & source_columns_set_, bool ignore_alias_, ExtractedSettings && settings_, bool allow_self_aliases_) : aliases(aliases_) , source_columns_set(source_columns_set_) , settings(settings_) diff --git a/tests/queries/0_stateless/01960_lambda_precedence.reference b/tests/queries/0_stateless/01960_lambda_precedence.reference new file mode 100644 index 00000000000..552cf82b9ba --- /dev/null +++ b/tests/queries/0_stateless/01960_lambda_precedence.reference @@ -0,0 +1,3 @@ +1000 [2,3,4] +1 +1 diff --git a/tests/queries/0_stateless/01960_lambda_precedence.sql b/tests/queries/0_stateless/01960_lambda_precedence.sql new file mode 100644 index 00000000000..858ca7f5dd0 --- /dev/null +++ b/tests/queries/0_stateless/01960_lambda_precedence.sql @@ -0,0 +1,25 @@ +SELECT + 1000 AS a, + arrayMap(a -> (a + 1), [1, 2, 3]); + + +-- https://github.com/ClickHouse/ClickHouse/issues/5046 +SELECT sum(c1) AS v +FROM + ( + SELECT + 1 AS c1, + ['v'] AS c2 + ) +WHERE arrayExists(v -> (v = 'v'), c2); + + +SELECT sum(c1) AS v +FROM + ( + SELECT + 1 AS c1, + ['v'] AS c2, + ['d'] AS d + ) +WHERE arrayExists(i -> (d = ['d']), c2); From fd76966715eaedd0d5d8d76c05e711d3af9779be Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 5 Aug 2021 19:03:42 +0200 Subject: [PATCH 062/147] Test shadow lambda recovery Be a good citizen and leave a test checking for the recovery after the lambda function has been processed --- tests/queries/0_stateless/01960_lambda_precedence.reference | 2 +- tests/queries/0_stateless/01960_lambda_precedence.sql | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01960_lambda_precedence.reference b/tests/queries/0_stateless/01960_lambda_precedence.reference index 552cf82b9ba..96e36988183 100644 --- a/tests/queries/0_stateless/01960_lambda_precedence.reference +++ b/tests/queries/0_stateless/01960_lambda_precedence.reference @@ -1,3 +1,3 @@ -1000 [2,3,4] +1000 [2,3,4] 1010 1 1 diff --git a/tests/queries/0_stateless/01960_lambda_precedence.sql b/tests/queries/0_stateless/01960_lambda_precedence.sql index 858ca7f5dd0..a3ff1424cf2 100644 --- a/tests/queries/0_stateless/01960_lambda_precedence.sql +++ b/tests/queries/0_stateless/01960_lambda_precedence.sql @@ -1,6 +1,7 @@ SELECT 1000 AS a, - arrayMap(a -> (a + 1), [1, 2, 3]); + arrayMap(a -> (a + 1), [1, 2, 3]), + a + 10 as c; -- https://github.com/ClickHouse/ClickHouse/issues/5046 From fbcf0962eebec993694b247fee27e8111418d234 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:01:35 +0300 Subject: [PATCH 063/147] Mark some tests as long --- ...=> 00062_replicated_merge_tree_alter_zookeeper_long.reference} | 0 ...r.sql => 00062_replicated_merge_tree_alter_zookeeper_long.sql} | 0 ...reference => 00083_create_merge_tree_zookeeper_long.reference} | 0 ...e_zookeeper.sql => 00083_create_merge_tree_zookeeper_long.sql} | 0 ...reference => 00215_primary_key_order_zookeeper_long.reference} | 0 ...r_zookeeper.sql => 00215_primary_key_order_zookeeper_long.sql} | 0 ...00502_custom_partitioning_replicated_zookeeper_long.reference} | 0 ...ql => 00502_custom_partitioning_replicated_zookeeper_long.sql} | 0 ...st_zookeeper.reference => 00643_cast_zookeeper_long.reference} | 0 .../{00643_cast_zookeeper.sql => 00643_cast_zookeeper_long.sql} | 0 ...ect_with_old_data_and_without_quorum_zookeeper_long.reference} | 0 ...rt_select_with_old_data_and_without_quorum_zookeeper_long.sql} | 0 ...ce => 00836_indices_alter_replicated_zookeeper_long.reference} | 0 ...eper.sql => 00836_indices_alter_replicated_zookeeper_long.sql} | 0 ...zookeeper_custom_compression_codecs_replicated_long.reference} | 0 ...00910_zookeeper_custom_compression_codecs_replicated_long.sql} | 0 ...aptive_index_granularity_replicated_merge_tree_long.reference} | 0 ...per_adaptive_index_granularity_replicated_merge_tree_long.sql} | 0 ...=> 01090_zookeeper_mutations_and_insert_quorum_long.reference} | 0 ...m.sql => 01090_zookeeper_mutations_and_insert_quorum_long.sql} | 0 ..._alter_enum_partition_key_replicated_zookeeper_long.reference} | 0 ... 01346_alter_enum_partition_key_replicated_zookeeper_long.sql} | 0 ....reference => 01430_modify_sample_by_zookeeper_long.reference} | 0 ...by_zookeeper.sql => 01430_modify_sample_by_zookeeper_long.sql} | 0 ...ence => 01532_execute_merges_on_single_replica_long.reference} | 0 ...eplica.sql => 01532_execute_merges_on_single_replica_long.sql} | 0 ...nce => 01581_deduplicate_by_columns_replicated_long.reference} | 0 ...cated.sql => 01581_deduplicate_by_columns_replicated_long.sql} | 0 28 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00062_replicated_merge_tree_alter_zookeeper.reference => 00062_replicated_merge_tree_alter_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00062_replicated_merge_tree_alter_zookeeper.sql => 00062_replicated_merge_tree_alter_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00083_create_merge_tree_zookeeper.reference => 00083_create_merge_tree_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00083_create_merge_tree_zookeeper.sql => 00083_create_merge_tree_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00215_primary_key_order_zookeeper.reference => 00215_primary_key_order_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00215_primary_key_order_zookeeper.sql => 00215_primary_key_order_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00502_custom_partitioning_replicated_zookeeper.reference => 00502_custom_partitioning_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00502_custom_partitioning_replicated_zookeeper.sql => 00502_custom_partitioning_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00643_cast_zookeeper.reference => 00643_cast_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00643_cast_zookeeper.sql => 00643_cast_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.reference => 00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql => 00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00836_indices_alter_replicated_zookeeper.reference => 00836_indices_alter_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00836_indices_alter_replicated_zookeeper.sql => 00836_indices_alter_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00910_zookeeper_custom_compression_codecs_replicated.reference => 00910_zookeeper_custom_compression_codecs_replicated_long.reference} (100%) rename tests/queries/0_stateless/{00910_zookeeper_custom_compression_codecs_replicated.sql => 00910_zookeeper_custom_compression_codecs_replicated_long.sql} (100%) rename tests/queries/0_stateless/{00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.reference => 00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.reference} (100%) rename tests/queries/0_stateless/{00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql => 00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.sql} (100%) rename tests/queries/0_stateless/{01090_zookeeper_mutations_and_insert_quorum.reference => 01090_zookeeper_mutations_and_insert_quorum_long.reference} (100%) rename tests/queries/0_stateless/{01090_zookeeper_mutations_and_insert_quorum.sql => 01090_zookeeper_mutations_and_insert_quorum_long.sql} (100%) rename tests/queries/0_stateless/{01346_alter_enum_partition_key_replicated_zookeeper.reference => 01346_alter_enum_partition_key_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01346_alter_enum_partition_key_replicated_zookeeper.sql => 01346_alter_enum_partition_key_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01430_modify_sample_by_zookeeper.reference => 01430_modify_sample_by_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01430_modify_sample_by_zookeeper.sql => 01430_modify_sample_by_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01532_execute_merges_on_single_replica.reference => 01532_execute_merges_on_single_replica_long.reference} (100%) rename tests/queries/0_stateless/{01532_execute_merges_on_single_replica.sql => 01532_execute_merges_on_single_replica_long.sql} (100%) rename tests/queries/0_stateless/{01581_deduplicate_by_columns_replicated.reference => 01581_deduplicate_by_columns_replicated_long.reference} (100%) rename tests/queries/0_stateless/{01581_deduplicate_by_columns_replicated.sql => 01581_deduplicate_by_columns_replicated_long.sql} (100%) diff --git a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference rename to tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql rename to tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.reference b/tests/queries/0_stateless/00083_create_merge_tree_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00083_create_merge_tree_zookeeper.reference rename to tests/queries/0_stateless/00083_create_merge_tree_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql b/tests/queries/0_stateless/00083_create_merge_tree_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql rename to tests/queries/0_stateless/00083_create_merge_tree_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00215_primary_key_order_zookeeper.reference b/tests/queries/0_stateless/00215_primary_key_order_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00215_primary_key_order_zookeeper.reference rename to tests/queries/0_stateless/00215_primary_key_order_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql b/tests/queries/0_stateless/00215_primary_key_order_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql rename to tests/queries/0_stateless/00215_primary_key_order_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference rename to tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql rename to tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.reference b/tests/queries/0_stateless/00643_cast_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00643_cast_zookeeper.reference rename to tests/queries/0_stateless/00643_cast_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.sql b/tests/queries/0_stateless/00643_cast_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00643_cast_zookeeper.sql rename to tests/queries/0_stateless/00643_cast_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference rename to tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql rename to tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.reference similarity index 100% rename from tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference rename to tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.reference diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.sql similarity index 100% rename from tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql rename to tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.sql diff --git a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.reference b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.reference similarity index 100% rename from tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.reference rename to tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.reference diff --git a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.sql similarity index 100% rename from tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql rename to tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.sql diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.reference b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.reference similarity index 100% rename from tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.reference rename to tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.reference diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql similarity index 100% rename from tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql rename to tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.reference b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.reference rename to tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql rename to tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.reference b/tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01430_modify_sample_by_zookeeper.reference rename to tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql b/tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql rename to tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica.reference b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.reference similarity index 100% rename from tests/queries/0_stateless/01532_execute_merges_on_single_replica.reference rename to tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.reference diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql similarity index 100% rename from tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql rename to tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql diff --git a/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.reference b/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated_long.reference similarity index 100% rename from tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.reference rename to tests/queries/0_stateless/01581_deduplicate_by_columns_replicated_long.reference diff --git a/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql b/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated_long.sql similarity index 100% rename from tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql rename to tests/queries/0_stateless/01581_deduplicate_by_columns_replicated_long.sql From d5987b7a0695862371946b7e796fc279419c70b0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:01:36 +0300 Subject: [PATCH 064/147] Fix 00754_alter_modify_order_by_replicated_zookeeper --- ...00754_alter_modify_order_by_replicated_zookeeper.reference | 4 ++-- .../00754_alter_modify_order_by_replicated_zookeeper.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference index 48fe2d30bf3..a1fecd72e30 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference +++ b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference @@ -9,6 +9,6 @@ 1 2 1 30 1 2 4 90 *** Check SHOW CREATE TABLE *** -CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, -z)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/default/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, -z)\nSETTINGS index_granularity = 8192 *** Check SHOW CREATE TABLE after offline ALTER *** -CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `t` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, t * t)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `t` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/default/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, t * t)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql index 9e08406202e..78986338cd9 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql @@ -9,8 +9,8 @@ DROP TABLE old_style; DROP TABLE IF EXISTS summing_r1; DROP TABLE IF EXISTS summing_r2; -CREATE TABLE summing_r1(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test_00754/summing', 'r1') ORDER BY (x, y); -CREATE TABLE summing_r2(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test_00754/summing', 'r2') ORDER BY (x, y); +CREATE TABLE summing_r1(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_00754/summing', 'r1') ORDER BY (x, y); +CREATE TABLE summing_r2(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_00754/summing', 'r2') ORDER BY (x, y); /* Can't add an expression with existing column to ORDER BY. */ ALTER TABLE summing_r1 MODIFY ORDER BY (x, y, -val); -- { serverError 36 } From 685c7b8df29f3e57ed61600237beafcb349ccd67 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:01:36 +0300 Subject: [PATCH 065/147] Add some tests to parallel list excludes --- tests/queries/skip_list.json | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 5a4c982e13d..24acb86299f 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -498,6 +498,11 @@ "01902_table_function_merge_db_repr", "01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer", "01946_test_wrong_host_name_access", + "00510_materizlized_view_and_deduplication_zookeeper", /// static UUID + "00509_extended_storage_definition_syntax_zookeeper", /// leftovers + "00083_create_merge_tree_zookeeper", /// leftovers + "01532_execute_merges_on_single_replica", /// static zk path + "01530_drop_database_atomic_sync", /// creates database "02001_add_default_database_to_system_users" ///create user ] } From 3eee01f157d5a2963a934f0279ae8a93dd34381c Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 5 Aug 2021 17:08:08 -0300 Subject: [PATCH 066/147] Update graphitemergetree.md add info GraphiteMergeTree descrption --- .../table-engines/mergetree-family/graphitemergetree.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/ru/engines/table-engines/mergetree-family/graphitemergetree.md index f3e915a413b..891d5227100 100644 --- a/docs/ru/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/graphitemergetree.md @@ -134,7 +134,7 @@ default - `regexp` – шаблон имени метрики. - `age` – минимальный возраст данных в секундах. - `precision` – точность определения возраста данных в секундах. Должен быть делителем для 86400 (количество секунд в сутках). -- `function` – имя агрегирующей функции, которую следует применить к данным, чей возраст оказался в интервале `[age, age + precision]`. +- `function` – имя агрегирующей функции, которую следует применить к данным, чей возраст оказался в интервале `[age, age + precision]`. Допустимые функции: min/max/any/avg. Avg вычисляется неточно, как среднее от средних. ### Пример конфигурации {#configuration-example} @@ -171,3 +171,6 @@ default ``` + +!!! warning "Внимание" + Прореживание данных производится во время слияний. Обычно для старых партций слияния не запускаются, поэтому для прореживания надо иницировать незапланированное слияние используя [optimize](../../../sql-reference/statements/optimize/). Или использовать дополнительные инструменты, например [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer). From bd776e432f33907a557ad7e6d999712d774a5dd8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Aug 2021 23:36:46 +0300 Subject: [PATCH 067/147] Advancements --- contrib/croaring-cmake/CMakeLists.txt | 8 +++++- src/Common/clickhouse_malloc.cpp | 38 +++++++++++++++++++++++---- 2 files changed, 40 insertions(+), 6 deletions(-) diff --git a/contrib/croaring-cmake/CMakeLists.txt b/contrib/croaring-cmake/CMakeLists.txt index 522540e5e40..1c61eb39cc8 100644 --- a/contrib/croaring-cmake/CMakeLists.txt +++ b/contrib/croaring-cmake/CMakeLists.txt @@ -25,4 +25,10 @@ target_include_directories(roaring PRIVATE "${LIBRARY_DIR}/include/roaring") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/include") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/cpp") -target_compile_definitions(roaring PRIVATE -Dmalloc=clickhouse_malloc -Dfree=clickhouse_free -Dposix_memalign=clickhouse_posix_memalign) +target_compile_definitions(roaring PRIVATE + -Dmalloc=clickhouse_malloc + -Dcalloc=clickhouse_calloc + -Drealloc=clickhouse_realloc + -Dreallocarray=clickhouse_reallocarray + -Dfree=clickhouse_free + -Dposix_memalign=clickhouse_posix_memalign) diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp index 5e37c8b3a2d..95b1d56974b 100644 --- a/src/Common/clickhouse_malloc.cpp +++ b/src/Common/clickhouse_malloc.cpp @@ -1,7 +1,5 @@ -#if 0 - #include -#include +#include extern "C" void * clickhouse_malloc(size_t size) @@ -12,6 +10,38 @@ extern "C" void * clickhouse_malloc(size_t size) return res; } +extern "C" void * clickhouse_calloc(size_t number_of_members, size_t size) +{ + void * res = calloc(number_of_members, size); + if (res) + Memory::trackMemory(number_of_members * size); + return res; +} + +extern "C" void * clickhouse_realloc(void * ptr, size_t size) +{ + void * res = realloc(ptr, size); + if (res) + { + if (ptr) + Memory::untrackMemory(ptr); + Memory::trackMemory(size); + } + return res; +} + +extern "C" void * clickhouse_reallocarray(void * ptr, size_t number_of_members, size_t size) +{ + void * res = reallocarray(ptr, number_of_members, size); + if (res) + { + if (ptr) + Memory::untrackMemory(ptr); + Memory::trackMemory(number_of_members * size); + } + return res; +} + extern "C" void clickhouse_free(void * ptr) { Memory::untrackMemory(ptr); @@ -25,5 +55,3 @@ extern "C" int clickhouse_posix_memalign(void ** memptr, size_t alignment, size_ Memory::trackMemory(size); return res; } - -#endif From 9a5533a0880c8d0e5e7639067ea2647874846625 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Aug 2021 23:44:14 +0300 Subject: [PATCH 068/147] Improve performance --- src/AggregateFunctions/AggregateFunctionGroupBitmapData.h | 2 -- src/Interpreters/Aggregator.cpp | 3 +++ 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index d5484c4db79..a10cb7ebac5 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -68,14 +68,12 @@ public: { toLarge(); rb->add(static_cast(value)); - CurrentMemoryTracker::check(); } } } else { rb->add(static_cast(value)); - CurrentMemoryTracker::check(); } } diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 7ffae761c0c..69ad1d56359 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1193,6 +1193,9 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const } } + /// Some aggregate functions cannot throw exceptions on allocations (e.g. from C malloc) + /// but still tracks memory. Check it here. + CurrentMemoryTracker::check(); return true; } From 638b229e275c837f600a32152533541eeecde90f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Aug 2021 23:50:09 +0300 Subject: [PATCH 069/147] Remove debug headers --- src/AggregateFunctions/AggregateFunctionGroupBitmapData.h | 1 - src/Common/CurrentMemoryTracker.cpp | 1 - 2 files changed, 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index a10cb7ebac5..95c7e6075d7 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -7,7 +7,6 @@ #include #include #include -#include // Include this header last, because it is an auto-generated dump of questionable // garbage that breaks the build (e.g. it changes _POSIX_C_SOURCE). diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index 1156c6e56bc..bf0745e667e 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -2,7 +2,6 @@ #include #include -#include namespace From b1eb3fde6cae81c02ce3ae83434056386805bac5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Aug 2021 23:51:39 +0300 Subject: [PATCH 070/147] Add comment --- src/Common/clickhouse_malloc.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp index 95b1d56974b..cb6419ae6da 100644 --- a/src/Common/clickhouse_malloc.cpp +++ b/src/Common/clickhouse_malloc.cpp @@ -2,6 +2,9 @@ #include +/** These functions can be substituted instead of regular ones when memory tracking is needed. + */ + extern "C" void * clickhouse_malloc(size_t size) { void * res = malloc(size); From c4e2342a065a3432338d40a9ddee0228a38dfacf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:05:46 +0300 Subject: [PATCH 071/147] Add verbose description for pre-LIMIT into EXPLAIN query --- src/Interpreters/InterpreterSelectQuery.cpp | 5 ++++- ...1562_optimize_monotonous_functions_in_order_by.reference | 6 +++--- .../0_stateless/01576_alias_column_rewrite.reference | 6 +++--- 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 33f9deaf805..051dff19bff 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2429,7 +2429,10 @@ void InterpreterSelectQuery::executePreLimit(QueryPlan & query_plan, bool do_not } auto limit = std::make_unique(query_plan.getCurrentDataStream(), limit_length, limit_offset); - limit->setStepDescription("preliminary LIMIT"); + if (do_not_skip_offset) + limit->setStepDescription("preliminary LIMIT (with OFFSET)"); + else + limit->setStepDescription("preliminary LIMIT (without OFFSET)"); query_plan.addStep(std::move(limit)); } } diff --git a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference index 0eb7e06f724..bf9bff06959 100644 --- a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference @@ -5,7 +5,7 @@ FROM test_order_by ORDER BY timestamp ASC LIMIT 10 Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) MergingSorted (Merge sorted streams for ORDER BY) MergeSorting (Merge sorted blocks for ORDER BY) PartialSorting (Sort each block for ORDER BY) @@ -19,7 +19,7 @@ FROM test_order_by ORDER BY toDate(timestamp) ASC LIMIT 10 Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) FinishSorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) @@ -33,7 +33,7 @@ ORDER BY timestamp ASC LIMIT 10 Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) FinishSorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) diff --git a/tests/queries/0_stateless/01576_alias_column_rewrite.reference b/tests/queries/0_stateless/01576_alias_column_rewrite.reference index c9a4c04b352..2a824e62158 100644 --- a/tests/queries/0_stateless/01576_alias_column_rewrite.reference +++ b/tests/queries/0_stateless/01576_alias_column_rewrite.reference @@ -22,7 +22,7 @@ lambda 1 optimize_read_in_order Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) MergingSorted (Merge sorted streams for ORDER BY) MergeSorting (Merge sorted blocks for ORDER BY) PartialSorting (Sort each block for ORDER BY) @@ -30,13 +30,13 @@ Expression (Projection) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) FinishSorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) FinishSorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) From 1358a2db92b60339b5081825b3e2cafec0373089 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:05:46 +0300 Subject: [PATCH 072/147] Fix WithMergeableStateAfterAggregationAndLimit with LIMIT BY and LIMIT OFFSET In case of LIMIT BY, pre LIMIT cannot be applied (that is done on the shard before), and hence shard applies regular LIMIT, but it goes with applying OFFSET, and also the initiator will do apply OFFSET too. --- src/Interpreters/InterpreterSelectQuery.cpp | 28 ++++++++++----- ...onAndLimit_LIMIT_BY_LIMIT_OFFSET.reference | 36 +++++++++++++++++++ ...regationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql | 26 ++++++++++++++ 3 files changed, 81 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference create mode 100644 tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 051dff19bff..ed76074c496 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1327,15 +1327,20 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu } bool apply_limit = options.to_stage != QueryProcessingStage::WithMergeableStateAfterAggregation; + bool apply_prelimit = apply_limit && + query.limitLength() && !query.limit_with_ties && + !hasWithTotalsInAnySubqueryInFromClause(query) && + !query.arrayJoinExpressionList() && + !query.distinct && + !expressions.hasLimitBy() && + !settings.extremes && + !has_withfill; bool apply_offset = options.to_stage != QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; - bool has_prelimit = false; - if (apply_limit && - query.limitLength() && !query.limit_with_ties && !hasWithTotalsInAnySubqueryInFromClause(query) && - !query.arrayJoinExpressionList() && !query.distinct && !expressions.hasLimitBy() && !settings.extremes && - !has_withfill) + bool limit_applied = false; + if (apply_prelimit) { executePreLimit(query_plan, /* do_not_skip_offset= */!apply_offset); - has_prelimit = true; + limit_applied = true; } /** If there was more than one stream, @@ -1354,10 +1359,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu /// If we have 'WITH TIES', we need execute limit before projection, /// because in that case columns from 'ORDER BY' are used. - if (query.limit_with_ties) + if (query.limit_with_ties && apply_offset) { executeLimit(query_plan); - has_prelimit = true; + limit_applied = true; } /// Projection not be done on the shards, since then initiator will not find column in blocks. @@ -1372,7 +1377,12 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu executeExtremes(query_plan); /// Limit is no longer needed if there is prelimit. - if (apply_limit && !has_prelimit) + /// + /// NOTE: that LIMIT cannot be applied of OFFSET should not be applied, + /// since LIMIT will apply OFFSET too. + /// This is the case for various optimizations for distributed queries, + /// and when LIMIT cannot be applied it will be applied on the initiator anyway. + if (apply_limit && !limit_applied && apply_offset) executeLimit(query_plan); if (apply_offset) diff --git a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference new file mode 100644 index 00000000000..08fe4251b69 --- /dev/null +++ b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference @@ -0,0 +1,36 @@ +-- { echo } +SELECT * +FROM remote('127.{1,2}', view( + SELECT * + FROM numbers(10) +), number) +GROUP BY number +ORDER BY number ASC +LIMIT 1 BY number +LIMIT 5, 5 +SETTINGS + optimize_skip_unused_shards = 1, + optimize_distributed_group_by_sharding_key = 1, + distributed_push_down_limit=1; +5 +6 +7 +8 +9 +SELECT * +FROM remote('127.{1,2}', view( + SELECT * + FROM numbers(10) +), number) +GROUP BY number +ORDER BY number ASC +LIMIT 1 BY number +LIMIT 5, 5 +SETTINGS + distributed_group_by_no_merge=2, + distributed_push_down_limit=1; +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql new file mode 100644 index 00000000000..75c06704ab9 --- /dev/null +++ b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql @@ -0,0 +1,26 @@ +-- { echo } +SELECT * +FROM remote('127.{1,2}', view( + SELECT * + FROM numbers(10) +), number) +GROUP BY number +ORDER BY number ASC +LIMIT 1 BY number +LIMIT 5, 5 +SETTINGS + optimize_skip_unused_shards = 1, + optimize_distributed_group_by_sharding_key = 1, + distributed_push_down_limit=1; +SELECT * +FROM remote('127.{1,2}', view( + SELECT * + FROM numbers(10) +), number) +GROUP BY number +ORDER BY number ASC +LIMIT 1 BY number +LIMIT 5, 5 +SETTINGS + distributed_group_by_no_merge=2, + distributed_push_down_limit=1; From d0b6b167f5d1cd2bca23237d616cef3a1ffe7d97 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 00:19:04 +0300 Subject: [PATCH 073/147] Exclude 01493_alter_remove_properties_zookeeper from parallel run too --- tests/queries/skip_list.json | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 24acb86299f..aebe9a83e42 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -498,6 +498,7 @@ "01902_table_function_merge_db_repr", "01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer", "01946_test_wrong_host_name_access", + "01493_alter_remove_properties_zookeeper", "00510_materizlized_view_and_deduplication_zookeeper", /// static UUID "00509_extended_storage_definition_syntax_zookeeper", /// leftovers "00083_create_merge_tree_zookeeper", /// leftovers From 735b730a526169f6438a76ff4fe4634ba029193a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 00:20:48 +0300 Subject: [PATCH 074/147] Mark more tests as long --- ...6_zookeeper_deduplication_and_unexpected_parts_long.reference} | 0 ...> 00226_zookeeper_deduplication_and_unexpected_parts_long.sql} | 0 ... 00236_replicated_drop_on_non_leader_zookeeper_long.reference} | 0 ...sql => 00236_replicated_drop_on_non_leader_zookeeper_long.sql} | 0 ...e => 00446_clear_column_in_partition_zookeeper_long.reference} | 0 ...per.sql => 00446_clear_column_in_partition_zookeeper_long.sql} | 0 ...e => 00623_replicated_truncate_table_zookeeper_long.reference} | 0 ...per.sql => 00623_replicated_truncate_table_zookeeper_long.sql} | 0 ...orce_by_identical_result_after_merge_zookeeper_long.reference} | 0 ...0721_force_by_identical_result_after_merge_zookeeper_long.sql} | 0 ...ent_columns.reference => 00725_comment_columns_long.reference} | 0 .../{00725_comment_columns.sql => 00725_comment_columns_long.sql} | 0 ...uorum_insert_have_data_before_quorum_zookeeper_long.reference} | 0 ...0732_quorum_insert_have_data_before_quorum_zookeeper_long.sql} | 0 ...nce => 00732_quorum_insert_lost_part_zookeeper_long.reference} | 0 ...eeper.sql => 00732_quorum_insert_lost_part_zookeeper_long.sql} | 0 ...32_quorum_insert_simple_test_2_parts_zookeeper_long.reference} | 0 ...=> 00732_quorum_insert_simple_test_2_parts_zookeeper_long.sql} | 0 ...per_empty_replicated_merge_tree_optimize_final_long.reference} | 0 ...zookeeper_empty_replicated_merge_tree_optimize_final_long.sql} | 0 ...er.reference => 00933_ttl_replicated_zookeeper_long.reference} | 0 ...ated_zookeeper.sql => 00933_ttl_replicated_zookeeper_long.sql} | 0 ...eference => 01062_alter_on_mutataion_zookeeper_long.reference} | 0 ..._zookeeper.sql => 01062_alter_on_mutataion_zookeeper_long.sql} | 0 ...e => 01213_alter_rename_with_default_zookeeper_long.reference} | 0 ...per.sql => 01213_alter_rename_with_default_zookeeper_long.sql} | 0 ...e => 01267_alter_default_key_columns_zookeeper_long.reference} | 0 ...per.sql => 01267_alter_default_key_columns_zookeeper_long.sql} | 0 ...as.reference => 01319_manual_write_to_replicas_long.reference} | 0 ...te_to_replicas.sql => 01319_manual_write_to_replicas_long.sql} | 0 ...e.reference => 01669_columns_declaration_serde_long.reference} | 0 ...aration_serde.sql => 01669_columns_declaration_serde_long.sql} | 0 32 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00226_zookeeper_deduplication_and_unexpected_parts.reference => 00226_zookeeper_deduplication_and_unexpected_parts_long.reference} (100%) rename tests/queries/0_stateless/{00226_zookeeper_deduplication_and_unexpected_parts.sql => 00226_zookeeper_deduplication_and_unexpected_parts_long.sql} (100%) rename tests/queries/0_stateless/{00236_replicated_drop_on_non_leader_zookeeper.reference => 00236_replicated_drop_on_non_leader_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00236_replicated_drop_on_non_leader_zookeeper.sql => 00236_replicated_drop_on_non_leader_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00446_clear_column_in_partition_zookeeper.reference => 00446_clear_column_in_partition_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00446_clear_column_in_partition_zookeeper.sql => 00446_clear_column_in_partition_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00623_replicated_truncate_table_zookeeper.reference => 00623_replicated_truncate_table_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00623_replicated_truncate_table_zookeeper.sql => 00623_replicated_truncate_table_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00721_force_by_identical_result_after_merge_zookeeper.reference => 00721_force_by_identical_result_after_merge_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00721_force_by_identical_result_after_merge_zookeeper.sql => 00721_force_by_identical_result_after_merge_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00725_comment_columns.reference => 00725_comment_columns_long.reference} (100%) rename tests/queries/0_stateless/{00725_comment_columns.sql => 00725_comment_columns_long.sql} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_have_data_before_quorum_zookeeper.reference => 00732_quorum_insert_have_data_before_quorum_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_have_data_before_quorum_zookeeper.sql => 00732_quorum_insert_have_data_before_quorum_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_lost_part_zookeeper.reference => 00732_quorum_insert_lost_part_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_lost_part_zookeeper.sql => 00732_quorum_insert_lost_part_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_simple_test_2_parts_zookeeper.reference => 00732_quorum_insert_simple_test_2_parts_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_simple_test_2_parts_zookeeper.sql => 00732_quorum_insert_simple_test_2_parts_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00925_zookeeper_empty_replicated_merge_tree_optimize_final.reference => 00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.reference} (100%) rename tests/queries/0_stateless/{00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql => 00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.sql} (100%) rename tests/queries/0_stateless/{00933_ttl_replicated_zookeeper.reference => 00933_ttl_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00933_ttl_replicated_zookeeper.sql => 00933_ttl_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01062_alter_on_mutataion_zookeeper.reference => 01062_alter_on_mutataion_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01062_alter_on_mutataion_zookeeper.sql => 01062_alter_on_mutataion_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01213_alter_rename_with_default_zookeeper.reference => 01213_alter_rename_with_default_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01213_alter_rename_with_default_zookeeper.sql => 01213_alter_rename_with_default_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01267_alter_default_key_columns_zookeeper.reference => 01267_alter_default_key_columns_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01267_alter_default_key_columns_zookeeper.sql => 01267_alter_default_key_columns_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01319_manual_write_to_replicas.reference => 01319_manual_write_to_replicas_long.reference} (100%) rename tests/queries/0_stateless/{01319_manual_write_to_replicas.sql => 01319_manual_write_to_replicas_long.sql} (100%) rename tests/queries/0_stateless/{01669_columns_declaration_serde.reference => 01669_columns_declaration_serde_long.reference} (100%) rename tests/queries/0_stateless/{01669_columns_declaration_serde.sql => 01669_columns_declaration_serde_long.sql} (100%) diff --git a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.reference b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts_long.reference similarity index 100% rename from tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.reference rename to tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts_long.reference diff --git a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts_long.sql similarity index 100% rename from tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql rename to tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts_long.sql diff --git a/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.reference b/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.reference rename to tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql b/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql rename to tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.reference b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.reference rename to tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql rename to tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.reference b/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.reference rename to tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql b/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql rename to tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.reference b/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.reference rename to tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql b/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql rename to tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00725_comment_columns.reference b/tests/queries/0_stateless/00725_comment_columns_long.reference similarity index 100% rename from tests/queries/0_stateless/00725_comment_columns.reference rename to tests/queries/0_stateless/00725_comment_columns_long.reference diff --git a/tests/queries/0_stateless/00725_comment_columns.sql b/tests/queries/0_stateless/00725_comment_columns_long.sql similarity index 100% rename from tests/queries/0_stateless/00725_comment_columns.sql rename to tests/queries/0_stateless/00725_comment_columns_long.sql diff --git a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.reference b/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.reference similarity index 100% rename from tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.reference rename to tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.reference diff --git a/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql b/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.sql similarity index 100% rename from tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql rename to tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.sql diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference rename to tests/queries/0_stateless/00933_ttl_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql rename to tests/queries/0_stateless/00933_ttl_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.reference b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.reference rename to tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql rename to tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference rename to tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql rename to tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.reference b/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.reference rename to tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql b/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql rename to tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01319_manual_write_to_replicas.reference b/tests/queries/0_stateless/01319_manual_write_to_replicas_long.reference similarity index 100% rename from tests/queries/0_stateless/01319_manual_write_to_replicas.reference rename to tests/queries/0_stateless/01319_manual_write_to_replicas_long.reference diff --git a/tests/queries/0_stateless/01319_manual_write_to_replicas.sql b/tests/queries/0_stateless/01319_manual_write_to_replicas_long.sql similarity index 100% rename from tests/queries/0_stateless/01319_manual_write_to_replicas.sql rename to tests/queries/0_stateless/01319_manual_write_to_replicas_long.sql diff --git a/tests/queries/0_stateless/01669_columns_declaration_serde.reference b/tests/queries/0_stateless/01669_columns_declaration_serde_long.reference similarity index 100% rename from tests/queries/0_stateless/01669_columns_declaration_serde.reference rename to tests/queries/0_stateless/01669_columns_declaration_serde_long.reference diff --git a/tests/queries/0_stateless/01669_columns_declaration_serde.sql b/tests/queries/0_stateless/01669_columns_declaration_serde_long.sql similarity index 100% rename from tests/queries/0_stateless/01669_columns_declaration_serde.sql rename to tests/queries/0_stateless/01669_columns_declaration_serde_long.sql From 55b3d216012b1a46ec6e0108cf3b28c28bedeca1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 00:33:13 +0300 Subject: [PATCH 075/147] Fix polling of /sys/block in case of block devices reopened on error Found on CI [1]. [1]: https://clickhouse-test-reports.s3.yandex.net/27248/4f3b80ff33c846465983aa2bc9ae9490e1118b15/fuzzer_ubsan/report.html#fail1 --- src/Interpreters/AsynchronousMetrics.cpp | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index d708ff4f9e0..81f6f224c7b 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -896,9 +896,9 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti if (block_devices_rescan_delay.elapsedSeconds() >= 300) openBlockDevices(); - for (auto & [name, device] : block_devs) + try { - try + for (auto & [name, device] : block_devs) { device->rewind(); @@ -947,20 +947,20 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti new_values["BlockQueueTimePerOp_" + name] = delta_values.time_in_queue * time_multiplier / delta_values.in_flight_ios; } } + } + catch (...) + { + /// Try to reopen block devices in case of error + /// (i.e. ENOENT means that some disk had been replaced, and it may apperas with a new name) + try + { + openBlockDevices(); + } catch (...) { - /// Try to reopen block devices in case of error - /// (i.e. ENOENT means that some disk had been replaced, and it may apperas with a new name) - try - { - openBlockDevices(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } tryLogCurrentException(__PRETTY_FUNCTION__); } + tryLogCurrentException(__PRETTY_FUNCTION__); } if (net_dev) From a822e2d45361c90259671e66b7e3f8f48d5d2ff8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Aug 2021 02:06:26 +0300 Subject: [PATCH 076/147] Fix error --- src/Common/clickhouse_malloc.cpp | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp index cb6419ae6da..bb6ba7643b1 100644 --- a/src/Common/clickhouse_malloc.cpp +++ b/src/Common/clickhouse_malloc.cpp @@ -23,25 +23,21 @@ extern "C" void * clickhouse_calloc(size_t number_of_members, size_t size) extern "C" void * clickhouse_realloc(void * ptr, size_t size) { + if (ptr) + Memory::untrackMemory(ptr); void * res = realloc(ptr, size); if (res) - { - if (ptr) - Memory::untrackMemory(ptr); Memory::trackMemory(size); - } return res; } extern "C" void * clickhouse_reallocarray(void * ptr, size_t number_of_members, size_t size) { + if (ptr) + Memory::untrackMemory(ptr); void * res = reallocarray(ptr, number_of_members, size); if (res) - { - if (ptr) - Memory::untrackMemory(ptr); Memory::trackMemory(number_of_members * size); - } return res; } From 35062b2c1224bc4b17dbcdab17d7370283763a1b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Aug 2021 02:10:55 +0300 Subject: [PATCH 077/147] Fix compatibility --- src/Common/clickhouse_malloc.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp index bb6ba7643b1..3f69ebdf58d 100644 --- a/src/Common/clickhouse_malloc.cpp +++ b/src/Common/clickhouse_malloc.cpp @@ -33,12 +33,11 @@ extern "C" void * clickhouse_realloc(void * ptr, size_t size) extern "C" void * clickhouse_reallocarray(void * ptr, size_t number_of_members, size_t size) { - if (ptr) - Memory::untrackMemory(ptr); - void * res = reallocarray(ptr, number_of_members, size); - if (res) - Memory::trackMemory(number_of_members * size); - return res; + size_t real_size = 0; + if (__builtin_mul_overflow(number_of_members, size, &real_size)) + return nullptr; + + return clickhouse_realloc(ptr, real_size); } extern "C" void clickhouse_free(void * ptr) From 6567aeda6c63a911a3bf2b3e515782ab9fbc9a60 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 07:56:23 +0300 Subject: [PATCH 078/147] Mark more tests as long --- ...754_alter_modify_order_by_replicated_zookeeper_long.reference} | 0 ... => 00754_alter_modify_order_by_replicated_zookeeper_long.sql} | 0 ...rence => 01049_zookeeper_synchronous_mutations_long.reference} | 0 ...tations.sql => 01049_zookeeper_synchronous_mutations_long.sql} | 0 ..._without_select_sequence_consistency_zookeeper_long.reference} | 0 ..._count_without_select_sequence_consistency_zookeeper_long.sql} | 0 .../{01666_blns.reference => 01666_blns_long.reference} | 0 tests/queries/0_stateless/{01666_blns.sql => 01666_blns_long.sql} | 0 ... => 01715_background_checker_blather_zookeeper_long.reference} | 0 ...er.sql => 01715_background_checker_blather_zookeeper_long.sql} | 0 10 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00754_alter_modify_order_by_replicated_zookeeper.reference => 00754_alter_modify_order_by_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00754_alter_modify_order_by_replicated_zookeeper.sql => 00754_alter_modify_order_by_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01049_zookeeper_synchronous_mutations.reference => 01049_zookeeper_synchronous_mutations_long.reference} (100%) rename tests/queries/0_stateless/{01049_zookeeper_synchronous_mutations.sql => 01049_zookeeper_synchronous_mutations_long.sql} (100%) rename tests/queries/0_stateless/{01513_count_without_select_sequence_consistency_zookeeper.reference => 01513_count_without_select_sequence_consistency_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01513_count_without_select_sequence_consistency_zookeeper.sql => 01513_count_without_select_sequence_consistency_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01666_blns.reference => 01666_blns_long.reference} (100%) rename tests/queries/0_stateless/{01666_blns.sql => 01666_blns_long.sql} (100%) rename tests/queries/0_stateless/{01715_background_checker_blather_zookeeper.reference => 01715_background_checker_blather_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01715_background_checker_blather_zookeeper.sql => 01715_background_checker_blather_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference rename to tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql rename to tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference b/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.reference similarity index 100% rename from tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference rename to tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.reference diff --git a/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql b/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.sql similarity index 100% rename from tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql rename to tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.sql diff --git a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.reference b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.reference rename to tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql rename to tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01666_blns.reference b/tests/queries/0_stateless/01666_blns_long.reference similarity index 100% rename from tests/queries/0_stateless/01666_blns.reference rename to tests/queries/0_stateless/01666_blns_long.reference diff --git a/tests/queries/0_stateless/01666_blns.sql b/tests/queries/0_stateless/01666_blns_long.sql similarity index 100% rename from tests/queries/0_stateless/01666_blns.sql rename to tests/queries/0_stateless/01666_blns_long.sql diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.reference b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01715_background_checker_blather_zookeeper.reference rename to tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql rename to tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql From c523904b746b70a941f802ff38031170880db489 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 14:00:07 +0300 Subject: [PATCH 079/147] Mark more tests as long --- ...nce => 00837_minmax_index_replicated_zookeeper_long.reference} | 0 ...eeper.sql => 00837_minmax_index_replicated_zookeeper_long.sql} | 0 ...01277_alter_rename_column_constraint_zookeeper_long.reference} | 0 ...ql => 01277_alter_rename_column_constraint_zookeeper_long.sql} | 0 ...ce => 01509_parallel_quorum_insert_no_replicas_long.reference} | 0 ...icas.sql => 01509_parallel_quorum_insert_no_replicas_long.sql} | 0 6 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00837_minmax_index_replicated_zookeeper.reference => 00837_minmax_index_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00837_minmax_index_replicated_zookeeper.sql => 00837_minmax_index_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01277_alter_rename_column_constraint_zookeeper.reference => 01277_alter_rename_column_constraint_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01277_alter_rename_column_constraint_zookeeper.sql => 01277_alter_rename_column_constraint_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01509_parallel_quorum_insert_no_replicas.reference => 01509_parallel_quorum_insert_no_replicas_long.reference} (100%) rename tests/queries/0_stateless/{01509_parallel_quorum_insert_no_replicas.sql => 01509_parallel_quorum_insert_no_replicas_long.sql} (100%) diff --git a/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.reference b/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.reference rename to tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql b/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql rename to tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference rename to tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql rename to tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.reference b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.reference similarity index 100% rename from tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.reference rename to tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.reference diff --git a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql similarity index 100% rename from tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql rename to tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql From 6747b8a03bfc98982cc742ff50af3162de6fd261 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 07:07:50 +0300 Subject: [PATCH 080/147] Mark more tests as long --- ...orum_insert_lost_part_and_alive_part_zookeeper_long.reference} | 0 ...732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql} | 0 ...nce => 00988_constraints_replication_zookeeper_long.reference} | 0 ...eeper.sql => 00988_constraints_replication_zookeeper_long.sql} | 0 ...per.reference => 01079_alter_default_zookeeper_long.reference} | 0 ...fault_zookeeper.sql => 01079_alter_default_zookeeper_long.sql} | 0 ..._drop_column_compact_part_replicated_zookeeper_long.reference} | 0 ... 01201_drop_column_compact_part_replicated_zookeeper_long.sql} | 0 8 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference => 00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql => 00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00988_constraints_replication_zookeeper.reference => 00988_constraints_replication_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00988_constraints_replication_zookeeper.sql => 00988_constraints_replication_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01079_alter_default_zookeeper.reference => 01079_alter_default_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01079_alter_default_zookeeper.sql => 01079_alter_default_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01201_drop_column_compact_part_replicated_zookeeper.reference => 01201_drop_column_compact_part_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01201_drop_column_compact_part_replicated_zookeeper.sql => 01201_drop_column_compact_part_replicated_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00988_constraints_replication_zookeeper.reference b/tests/queries/0_stateless/00988_constraints_replication_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00988_constraints_replication_zookeeper.reference rename to tests/queries/0_stateless/00988_constraints_replication_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql b/tests/queries/0_stateless/00988_constraints_replication_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql rename to tests/queries/0_stateless/00988_constraints_replication_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01079_alter_default_zookeeper.reference b/tests/queries/0_stateless/01079_alter_default_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01079_alter_default_zookeeper.reference rename to tests/queries/0_stateless/01079_alter_default_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01079_alter_default_zookeeper.sql b/tests/queries/0_stateless/01079_alter_default_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01079_alter_default_zookeeper.sql rename to tests/queries/0_stateless/01079_alter_default_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.reference b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.reference rename to tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql rename to tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql From e7c28767fcd09e168724fb38b8ff56ba9a835af1 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Fri, 6 Aug 2021 04:54:41 +0000 Subject: [PATCH 081/147] Fixes --- docs/en/sql-reference/functions/other-functions.md | 2 +- docs/ru/sql-reference/functions/other-functions.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 74eeb76577b..c182a4ea82a 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2140,7 +2140,7 @@ Result: ## currentProfiles {#current-profiles} -Returns list of the current [setting profiles](../../operations/access-rights.md#settings-profiles-management) for current user. The command [SET PROFILE](../../sql-reference/statements/set.md#query-set) could be used to change the current setting profile. If the command `SET PROFILE` hasn't been called the function returns the profiles specified at the current user's definition (see [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement), `SETTINGS` section). +Returns list of the current [setting profiles](../../operations/access-rights.md#settings-profiles-management) for current user. The command [SET PROFILE](../../sql-reference/statements/set.md#query-set) could be used to change the current setting profile. If the command `SET PROFILE` hasn't been called the function returns the profiles specified at the current user's definition (see [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)). **Syntax** diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 4e3e5a638dc..7e69392d979 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2124,7 +2124,7 @@ enabledProfiles() ## defaultProfiles {#default-profiles} -Возвращает все профили, указанные при объявлении текущего пользователя (см. [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement), секция `SETTINGS`) +Возвращает все профили, указанные при объявлении текущего пользователя (см. [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)) **Синтаксис** @@ -2136,4 +2136,4 @@ defaultProfiles() - Список профилей по умолчанию. -ТИп: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). \ No newline at end of file +Тип: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). \ No newline at end of file From d6b5b0922c63a15b41c946ac490ab3bc394df031 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Fri, 6 Aug 2021 05:11:13 +0000 Subject: [PATCH 082/147] More fixes --- docs/en/sql-reference/functions/other-functions.md | 2 -- docs/ru/sql-reference/functions/other-functions.md | 4 +--- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 5ebb00c8dcd..8200a3043c5 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2160,8 +2160,6 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere Returns settings profiles, assigned to the current user both explicitly and implicitly. Explicitly assigned profiles are the same as returned by the [currentProfiles](#current-profiles) function. Implicitly assigned profiles include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the `default_profile` section in the main server configuration file). - If some profile occurs multiple times (with some other settings in between), the latest occurrence overrides all the previous ones. - **Syntax** ``` sql diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index c902ecce225..0e23c2f743f 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2092,7 +2092,7 @@ SELECT tcpPort(); Возвращает список [профилей настроек](../../operations/access-rights.md#settings-profiles-management) для текущего пользователя. -Профиль настроек может быть указан при создании пользователя (см. [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)). Позднее для изменения настроек могут применяться другие профили с помощью выражения [SET PROFILE](../../sql-reference/statements/set.md#query-set). +Для изменения текущего профиля настроек может быть использована команда [SET PROFILE](../../sql-reference/statements/set.md#set-statement#query-set). Если команда `SET PROFILE` не применялась, функция возвращает профили, указанные при определении текущего пользователя (см. [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)). **Синтаксис** @@ -2110,8 +2110,6 @@ currentProfiles() Возвращает профили настроек, назначенные пользователю как явно, так и неявно. Явно назначенные профили — это те же профили, которые возвращает функция [currentProfiles](#current-profiles). Неявно назначенные профили включают родительские профили других назначенных профилей; профили, назначенные с помощью предоставленных ролей; профили, назначенные с помощью собственных настроек; основной профиль по умолчанию (см. секцию `default_profile` в основном конфигурационном файле сервера). - Если какой-либо профиль встречается несколько раз (с некоторыми другими настройками между ними), последнее событие перезаписывает все предыдущие. - **Синтаксис** ``` sql From 6ef73ef051f93cfb58aa9080e1ac7e56dde4a2e0 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Fri, 6 Aug 2021 05:15:55 +0000 Subject: [PATCH 083/147] More fixes --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 8200a3043c5..9b1a07a9faa 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2142,7 +2142,7 @@ Result: Returns a list of the current [settings profiles](../../operations/access-rights.md#settings-profiles-management) for the current user. -The settings profile can be specified at the user's definition (see [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)) and can be changed with the [SET PROFILE](../../sql-reference/statements/set.md#query-set) statements lately. +The command [SET PROFILE](../../sql-reference/statements/set.md#query-set) could be used to change the current setting profile. If the command `SET PROFILE` was not used the function returns the profiles specified at the current user's definition (see [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)). **Syntax** From b280e1418a06468e3ac81400eb65337f22c6b7fb Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Fri, 6 Aug 2021 11:37:07 +0300 Subject: [PATCH 084/147] Check if error_report is initialized in PocoHTTPClient.cpp --- src/IO/S3/PocoHTTPClient.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 2eb208e2587..78cb5300101 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -297,7 +297,7 @@ void PocoHTTPClient::makeRequestInternal( else if (status_code >= 300) { ProfileEvents::increment(select_metric(S3MetricType::Errors)); - if (status_code >= 500) + if (status_code >= 500 && error_report) error_report(request_configuration); } From fbcefaee5d7499d2c46bf8fb9f604f50497d385e Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 5 Aug 2021 11:36:35 +0300 Subject: [PATCH 085/147] Fill result vector only for empty input in multiSearch functions --- src/Functions/FunctionsMultiStringPosition.h | 2 +- src/Functions/MultiSearchFirstIndexImpl.h | 4 +++- src/Functions/MultiSearchFirstPositionImpl.h | 4 +++- src/Functions/MultiSearchImpl.h | 4 +++- 4 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsMultiStringPosition.h b/src/Functions/FunctionsMultiStringPosition.h index 74496484c92..f36f7639ccd 100644 --- a/src/Functions/FunctionsMultiStringPosition.h +++ b/src/Functions/FunctionsMultiStringPosition.h @@ -109,7 +109,7 @@ public: auto & vec_res = col_res->getData(); auto & offsets_res = col_offsets->getData(); - vec_res.resize_fill(column_haystack_size * refs.size()); + vec_res.resize(column_haystack_size * refs.size()); if (col_haystack_vector) Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res); diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index 146cb551bb3..4b29577d0eb 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -26,7 +26,7 @@ struct MultiSearchFirstIndexImpl { auto searcher = Impl::createMultiSearcherInBigHaystack(needles); const size_t haystack_string_size = haystack_offsets.size(); - res.resize_fill(haystack_string_size); + res.resize(haystack_string_size); size_t iteration = 0; while (searcher.hasMoreToSearch()) { @@ -42,6 +42,8 @@ struct MultiSearchFirstIndexImpl } ++iteration; } + if (iteration == 0) + std::fill(res.begin(), res.end(), 0); } }; diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index aee4da43958..bb1017c43ee 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -30,7 +30,7 @@ struct MultiSearchFirstPositionImpl }; auto searcher = Impl::createMultiSearcherInBigHaystack(needles); const size_t haystack_string_size = haystack_offsets.size(); - res.resize_fill(haystack_string_size); + res.resize(haystack_string_size); size_t iteration = 0; while (searcher.hasMoreToSearch()) { @@ -51,6 +51,8 @@ struct MultiSearchFirstPositionImpl } ++iteration; } + if (iteration == 0) + std::fill(res.begin(), res.end(), 0); } }; diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index 4580eda27e4..461af5c3295 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -26,7 +26,7 @@ struct MultiSearchImpl { auto searcher = Impl::createMultiSearcherInBigHaystack(needles); const size_t haystack_string_size = haystack_offsets.size(); - res.resize_fill(haystack_string_size); + res.resize(haystack_string_size); size_t iteration = 0; while (searcher.hasMoreToSearch()) { @@ -41,6 +41,8 @@ struct MultiSearchImpl } ++iteration; } + if (iteration == 0) + std::fill(res.begin(), res.end(), 0); } }; From 76ca40857a7072ef8303641365ee50829d5df7b9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 6 Aug 2021 12:15:09 +0300 Subject: [PATCH 086/147] Better error message for changed json_use_nulls in StorageView, check ast --- src/Storages/StorageView.cpp | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index a0ab532ece0..8749c3939a2 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -59,6 +59,16 @@ bool changedNullabilityOneWay(const Block & src_block, const Block & dst_block) return false; } +bool hasJoin(const ASTSelectWithUnionQuery & ast) +{ + for (const auto & child : ast.list_of_selects->children) + { + if (const auto * select = child->as(); select && select->join()) + return true; + } + return false; +} + } StorageView::StorageView( @@ -133,11 +143,16 @@ void StorageView::read( const auto & expected_header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); const auto & header = query_plan.getCurrentDataStream().header; - if (changedNullabilityOneWay(header, expected_header)) + const auto * select_with_union = current_inner_query->as(); + if (select_with_union && hasJoin(*select_with_union) && changedNullabilityOneWay(header, expected_header)) { throw DB::Exception(ErrorCodes::INCORRECT_QUERY, - "Joined columns is nullable, but setting `join_use_nulls` wans't set on CREATE VIEW"); + "Query from view {} returned Nullable column having not Nullable type in structure. " + "If query from view has JOIN, it may be cause by different values of 'json_use_nulls' setting. " + "You may explicitly specify 'json_use_nulls' in 'CREATE VIEW' query to avoid this error", + getStorageID().getFullTableName()); } + auto convert_actions_dag = ActionsDAG::makeConvertingActions( header.getColumnsWithTypeAndName(), expected_header.getColumnsWithTypeAndName(), From 084e3da13fa25b8729f3e96544877ea10034109f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 11:36:52 +0200 Subject: [PATCH 087/147] 01099_operators_date_and_timestamp: Use dates that work with all available timezones --- .../01099_operators_date_and_timestamp.reference | 10 +++++----- .../0_stateless/01099_operators_date_and_timestamp.sql | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01099_operators_date_and_timestamp.reference b/tests/queries/0_stateless/01099_operators_date_and_timestamp.reference index 103053a19f0..0d8a65c3869 100644 --- a/tests/queries/0_stateless/01099_operators_date_and_timestamp.reference +++ b/tests/queries/0_stateless/01099_operators_date_and_timestamp.reference @@ -1,13 +1,13 @@ 1 1 1 1 1 1 1 1 -2001-09-28 -2001-10-05 Date +2001-09-29 +2001-10-06 Date 2001-09-24 Date -2001-10-05 Date +2001-10-06 Date 2001-09-24 Date -2001-09-28 01:00:00 DateTime -2001-09-27 23:00:00 DateTime +2001-09-29 01:00:00 DateTime +2001-09-28 23:00:00 DateTime 3 Int32 2001-09-29 00:00:00 2001-09-28 00:00:00 diff --git a/tests/queries/0_stateless/01099_operators_date_and_timestamp.sql b/tests/queries/0_stateless/01099_operators_date_and_timestamp.sql index c630e19490d..8c3068cd36b 100644 --- a/tests/queries/0_stateless/01099_operators_date_and_timestamp.sql +++ b/tests/queries/0_stateless/01099_operators_date_and_timestamp.sql @@ -2,13 +2,13 @@ select interval 1 second, interval 1 minute, interval 1 hour; select interval 1 day, interval 1 week, interval 1 month; select interval 1 quarter, interval 1 year; -select date '2001-09-28'; -select (date '2001-09-28' + interval 7 day) x, toTypeName(x); +select date '2001-09-29'; +select (date '2001-09-29' + interval 7 day) x, toTypeName(x); select (date '2001-10-01' - interval 7 day) x, toTypeName(x); -select (date '2001-09-28' + 7) x, toTypeName(x); +select (date '2001-09-29' + 7) x, toTypeName(x); select (date '2001-10-01' - 7) x, toTypeName(x); -select (date '2001-09-28' + interval 1 hour) x, toTypeName(x); -select (date '2001-09-28' - interval 1 hour) x, toTypeName(x); +select (date '2001-09-29' + interval 1 hour) x, toTypeName(x); +select (date '2001-09-29' - interval 1 hour) x, toTypeName(x); select (date '2001-10-01' - date '2001-09-28') x, toTypeName(x); select timestamp '2001-09-28 01:00:00' + interval 23 hour; select timestamp '2001-09-28 23:00:00' - interval 23 hour; From 314f1d3b931539057b5cc550c3a3c4e6b79a9ce6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 6 Aug 2021 13:30:49 +0300 Subject: [PATCH 088/147] Update version_date.tsv after release 21.7.6.39 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index cd781e56973..5a1d2f4c098 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v21.7.6.39-stable 2021-08-06 v21.7.5.29-stable 2021-07-28 v21.7.4.18-stable 2021-07-17 v21.7.3.14-stable 2021-07-13 From ffab19caad56756109c1a355f9861d8244f8aa1b Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 6 Aug 2021 13:48:47 +0300 Subject: [PATCH 089/147] Fix hasJoin in StorageView.cpp --- src/Storages/StorageView.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 8749c3939a2..790b925f891 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -59,11 +59,21 @@ bool changedNullabilityOneWay(const Block & src_block, const Block & dst_block) return false; } +bool hasJoin(const ASTSelectQuery & select) +{ + const auto & tables = select.tables(); + if (!tables || tables->children.size() < 2) + return false; + + const auto & joined_table = tables->children[1]->as(); + return joined_table.table_join != nullptr; +} + bool hasJoin(const ASTSelectWithUnionQuery & ast) { for (const auto & child : ast.list_of_selects->children) { - if (const auto * select = child->as(); select && select->join()) + if (const auto * select = child->as(); select && hasJoin(*select)) return true; } return false; From 226db3f6bead3b6839894e10f80189c66b94fb60 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 13:38:30 +0200 Subject: [PATCH 090/147] Integration runner: Output longer errors --- tests/integration/runner | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/runner b/tests/integration/runner index 36cb4f22f9a..2143d7ebf29 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -277,7 +277,7 @@ if __name__ == "__main__": --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ {dockerd_internal_volume} -e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 \ - {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list}' {img} {command}".format( + {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list} -vvv' {img} {command}".format( net=net, tty=tty, bin=args.binary, From 4f1b02426a5db73714f3a09e8ab83574418d8672 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 13:38:56 +0200 Subject: [PATCH 091/147] test_storage_kafka: Remove duplicated code --- tests/integration/test_storage_kafka/test.py | 107 +------------------ 1 file changed, 2 insertions(+), 105 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index b9fc0b2272f..3c91f26fde8 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2,7 +2,6 @@ import json import os.path as p import random import socket -import subprocess import threading import time import logging @@ -181,32 +180,6 @@ def avro_confluent_message(schema_registry_client, value): }) return serializer.encode_record_with_schema('test_subject', schema, value) -# Since everything is async and shaky when receiving messages from Kafka, -# we may want to try and check results multiple times in a loop. -def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference'): - fpath = p.join(p.dirname(__file__), ref_file) - with open(fpath) as reference: - if check: - assert TSV(result) == TSV(reference) - else: - return TSV(result) == TSV(reference) - -def describe_consumer_group(kafka_cluster, name): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) - consumer_groups = admin_client.describe_consumer_groups([name]) - res = [] - for member in consumer_groups[0].members: - member_info = {} - member_info['member_id'] = member.member_id - member_info['client_id'] = member.client_id - member_info['client_host'] = member.client_host - member_topics_assignment = [] - for (topic, partitions) in member.member_assignment.assignment: - member_topics_assignment.append({'topic': topic, 'partitions': partitions}) - member_info['assignment'] = member_topics_assignment - res.append(member_info) - return res - # Fixtures @pytest.fixture(scope="module") @@ -262,7 +235,7 @@ kafka_topic_old old kafka_check_result(result, True) - members = describe_consumer_group('old') + members = describe_consumer_group(kafka_cluster, 'old') assert members[0]['client_id'] == 'ClickHouse-instance-test-kafka' # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:9092 --describe --members --group old --verbose")) @@ -302,7 +275,7 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_check_result(result, True) - members = describe_consumer_group('new') + members = describe_consumer_group(kafka_cluster, 'new') assert members[0]['client_id'] == 'instance test 1234' @@ -734,82 +707,6 @@ def kafka_setup_teardown(): # Tests - -def test_kafka_settings_old_syntax(kafka_cluster): - assert TSV(instance.query("SELECT * FROM system.macros WHERE macro like 'kafka%' ORDER BY macro", - ignore_error=True)) == TSV('''kafka_broker kafka1 -kafka_client_id instance -kafka_format_json_each_row JSONEachRow -kafka_group_name_new new -kafka_group_name_old old -kafka_topic_new new -kafka_topic_old old -''') - - instance.query(''' - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n'); - ''') - - # Don't insert malformed messages since old settings syntax - # doesn't support skipping of broken messages. - messages = [] - for i in range(50): - messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce(kafka_cluster, 'old', messages) - - result = '' - while True: - result += instance.query('SELECT * FROM test.kafka', ignore_error=True) - if kafka_check_result(result): - break - - kafka_check_result(result, True) - - members = describe_consumer_group(kafka_cluster, 'old') - assert members[0]['client_id'] == 'ClickHouse-instance-test-kafka' - # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:{} --describe --members --group old --verbose".format(cluster.kafka_port))) - - -def test_kafka_settings_new_syntax(kafka_cluster): - instance.query(''' - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = '{kafka_broker}:19092', - kafka_topic_list = '{kafka_topic_new}', - kafka_group_name = '{kafka_group_name_new}', - kafka_format = '{kafka_format_json_each_row}', - kafka_row_delimiter = '\\n', - kafka_client_id = '{kafka_client_id} test 1234', - kafka_skip_broken_messages = 1; - ''') - - messages = [] - for i in range(25): - messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce(kafka_cluster, 'new', messages) - - # Insert couple of malformed messages. - kafka_produce(kafka_cluster, 'new', ['}{very_broken_message,']) - kafka_produce(kafka_cluster, 'new', ['}another{very_broken_message,']) - - messages = [] - for i in range(25, 50): - messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce(kafka_cluster, 'new', messages) - - result = '' - while True: - result += instance.query('SELECT * FROM test.kafka', ignore_error=True) - if kafka_check_result(result): - break - - kafka_check_result(result, True) - - members = describe_consumer_group(kafka_cluster, 'new') - assert members[0]['client_id'] == 'instance test 1234' - - def test_kafka_issue11308(kafka_cluster): # Check that matview does respect Kafka SETTINGS kafka_produce(kafka_cluster, 'issue11308', ['{"t": 123, "e": {"x": "woof"} }', '{"t": 123, "e": {"x": "woof"} }', From b4e417e507e9d4b77eba1cc13691536645767457 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 13:39:33 +0200 Subject: [PATCH 092/147] test_kafka_virtual_columns_with_materialized_view: Limit iterations and show better error --- tests/integration/test_storage_kafka/test.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 3c91f26fde8..947b71b5f96 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1482,18 +1482,21 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster): messages.append(json.dumps({'key': i, 'value': i})) kafka_produce(kafka_cluster, 'virt2', messages, 0) - while True: - result = instance.query('SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view') - if kafka_check_result(result, False, 'test_kafka_virtual2.reference'): - break + sql = 'SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view ORDER BY kafka_key' + result = instance.query(sql) + iterations = 0 + while not kafka_check_result(result, False, 'test_kafka_virtual2.reference') and iterations < 10: + time.sleep(3) + iterations += 1 + result = instance.query(sql) + + kafka_check_result(result, True, 'test_kafka_virtual2.reference') instance.query(''' DROP TABLE test.consumer; DROP TABLE test.view; ''') - kafka_check_result(result, True, 'test_kafka_virtual2.reference') - def test_kafka_insert(kafka_cluster): instance.query(''' From d33a634014e85bb87a88cf1464febae908b211c6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 16:12:59 +0200 Subject: [PATCH 093/147] Improve 00738_lock_for_inner_table stability A sufficiently fast machine would finish processing the MV before the drop --- tests/queries/0_stateless/00738_lock_for_inner_table.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00738_lock_for_inner_table.sh b/tests/queries/0_stateless/00738_lock_for_inner_table.sh index d19288f65d8..6cbed42baea 100755 --- a/tests/queries/0_stateless/00738_lock_for_inner_table.sh +++ b/tests/queries/0_stateless/00738_lock_for_inner_table.sh @@ -14,7 +14,8 @@ DROP TABLE IF EXISTS mv SYNC; -- create table with fsync and 20 partitions for slower INSERT -- (since increasing number of records will make it significantly slower in debug build, but not in release) CREATE TABLE tab_00738(a Int) ENGINE = MergeTree() ORDER BY a PARTITION BY a%20 SETTINGS fsync_after_insert=1; -CREATE MATERIALIZED VIEW mv UUID '$uuid' ENGINE = Log AS SELECT a FROM tab_00738;" | ${CLICKHOUSE_CLIENT} -n +-- The matview will take at least 2 seconds to be finished (10000000 * 0.0000002) +CREATE MATERIALIZED VIEW mv UUID '$uuid' ENGINE = Log AS SELECT sleepEachRow(0.0000002) FROM tab_00738;" | ${CLICKHOUSE_CLIENT} -n ${CLICKHOUSE_CLIENT} --query_id insert_$CLICKHOUSE_DATABASE --query "INSERT INTO tab_00738 SELECT number FROM numbers(10000000)" & From 48eb9954613eae87316243e012848dba4ff2ded3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 17:23:50 +0300 Subject: [PATCH 094/147] Fix unknown column bug in sampling. --- .../QueryPlan/ReadFromMergeTree.cpp | 8 ++++---- ..._sampling_and_unknown_column_bug.reference | 2 ++ .../02002_sampling_and_unknown_column_bug.sql | 20 +++++++++++++++++++ 3 files changed, 26 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.reference create mode 100644 tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index f8c12449c7e..379f44cdb5d 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -995,21 +995,18 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build Block cur_header = result_projection ? result_projection->getResultColumns() : pipe.getHeader(); - auto append_actions = [&result_projection, &cur_header](ActionsDAGPtr actions) + auto append_actions = [&result_projection](ActionsDAGPtr actions) { if (!result_projection) result_projection = std::move(actions); else result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(*actions)); - - cur_header = result_projection->getResultColumns(); }; /// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values. if (sample_factor_column_queried) { ColumnWithTypeAndName column; - column.name = "_sample_factor"; column.type = std::make_shared(); column.column = column.type->createColumnConst(0, Field(result.sampling.used_sample_factor)); @@ -1017,6 +1014,9 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build append_actions(std::move(adding_column)); } + if (result_projection) + cur_header = result_projection->updateHeader(cur_header); + /// Extra columns may be returned (for example, if sampling is used). /// Convert pipe to step header structure. if (!isCompatibleHeader(cur_header, getOutputStream().header)) diff --git a/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.reference b/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.reference new file mode 100644 index 00000000000..9315e86b328 --- /dev/null +++ b/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.reference @@ -0,0 +1,2 @@ +1 +1 1 1 diff --git a/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.sql b/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.sql new file mode 100644 index 00000000000..838d7a5526b --- /dev/null +++ b/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.sql @@ -0,0 +1,20 @@ +drop table if exists sessions; +CREATE TABLE sessions +( + `user_id` UInt64 +) +ENGINE = MergeTree +ORDER BY user_id +SAMPLE BY user_id; + +insert into sessions values(1); + +SELECT + sum(user_id * _sample_factor) +FROM sessions +SAMPLE 10000000; + +SELECT + uniq(user_id) a, min(_sample_factor) x, a*x +FROM sessions +SAMPLE 10000000; From eb4d7415a285fa6d511879a0b3bc87105c37fba2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 17:28:56 +0300 Subject: [PATCH 095/147] Fix unknown column bug in sampling. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 379f44cdb5d..5fc32e2db69 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1007,6 +1007,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build if (sample_factor_column_queried) { ColumnWithTypeAndName column; + column.name = "_sample_factor"; column.type = std::make_shared(); column.column = column.type->createColumnConst(0, Field(result.sampling.used_sample_factor)); From af536b1b5ee509f968b7cb77bccb3a7971f7fbd1 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 5 Aug 2021 17:15:51 +0300 Subject: [PATCH 096/147] Using formatted string literals, extracted sort funcs in tester --- tests/clickhouse-test | 316 ++++++++++++++++++++++++++---------------- 1 file changed, 194 insertions(+), 122 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index d83b3f08c42..212ccd79f00 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -11,6 +11,7 @@ import copy import traceback from argparse import ArgumentParser +from typing import Tuple, Union, Optional, TextIO import shlex import subprocess from subprocess import Popen @@ -20,10 +21,12 @@ from subprocess import TimeoutExpired from datetime import datetime from time import time, sleep from errno import ESRCH + try: import termcolor except ImportError: termcolor = None + import random import string import multiprocessing @@ -81,7 +84,7 @@ def stop_tests(): def json_minify(string): """ Removes all js-style comments from json string. Allows to have comments in skip_list.json. - The code taken from https://github.com/getify/JSON.minify/tree/python under the MIT license. + The code was taken from https://github.com/getify/JSON.minify/tree/python under the MIT license. """ tokenizer = re.compile(r'"|(/\*)|(\*/)|(//)|\n|\r') @@ -148,13 +151,17 @@ def remove_control_characters(s): s = re.sub(r"[\x00-\x08\x0b\x0e-\x1f\x7f]", "", s) return s + def get_db_engine(args, database_name): if args.replicated_database: - return " ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', '{{shard}}', '{{replica}}')".format(database_name) + return f" ON CLUSTER test_cluster_database_replicated \ + ENGINE=Replicated('/test/clickhouse/db/{database_name}', \ + '{{shard}}', '{{replica}}')" if args.db_engine: return " ENGINE=" + args.db_engine return "" # Will use default engine + def configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file): testcase_args = copy.deepcopy(args) @@ -166,7 +173,6 @@ def configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file): database = testcase_args.database os.environ.setdefault("CLICKHOUSE_DATABASE", database) os.environ.setdefault("CLICKHOUSE_TMP", suite_tmp_dir) - else: # If --database is not specified, we will create temporary database with unique name # And we will recreate and drop it for each test @@ -176,8 +182,14 @@ def configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file): database = 'test_{suffix}'.format(suffix=random_str()) with open(stderr_file, 'w') as stderr: - client_cmd = testcase_args.testcase_client + " " + get_additional_client_options(args) - clickhouse_proc_create = Popen(shlex.split(client_cmd), stdin=PIPE, stdout=PIPE, stderr=stderr, universal_newlines=True) + client_cmd = testcase_args.testcase_client + " " \ + + get_additional_client_options(args) + + clickhouse_proc_create = open_client_process( + universal_newlines=True, + client_args=client_cmd, + stderr_file=stderr) + try: clickhouse_proc_create.communicate(("CREATE DATABASE " + database + get_db_engine(testcase_args, database)), timeout=testcase_args.timeout) except TimeoutExpired: @@ -237,8 +249,10 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std if need_drop_database: with open(stderr_file, 'a') as stderr: - clickhouse_proc_create = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=stderr, universal_newlines=True) + clickhouse_proc_create = open_client_process(client, universal_newlines=True, stderr_file=stderr) + seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 20) + try: drop_database_query = "DROP DATABASE " + database if args.replicated_database: @@ -254,7 +268,7 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std raise total_time = (datetime.now() - start_time).total_seconds() - return clickhouse_proc_create, "", "Timeout dropping database {} after test".format(database), total_time + return clickhouse_proc_create, "", f"Timeout dropping database {database} after test", total_time shutil.rmtree(args.test_tmp_dir) @@ -286,12 +300,16 @@ def need_retry(stdout, stderr): def get_processlist(args): try: query = b"SHOW PROCESSLIST FORMAT Vertical" + if args.replicated_database: query = b"SELECT materialize((hostName(), tcpPort())) as host, * " \ b"FROM clusterAllReplicas('test_cluster_database_replicated', system.processes) " \ b"WHERE query NOT LIKE '%system.processes%' FORMAT Vertical" - clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + + clickhouse_proc = open_client_process(args.client) + (stdout, _) = clickhouse_proc.communicate((query), timeout=20) + return False, stdout.decode('utf-8') except Exception as ex: print("Exception", ex) @@ -301,47 +319,90 @@ def get_processlist(args): # collect server stacktraces using gdb def get_stacktraces_from_gdb(server_pid): try: - cmd = "gdb -batch -ex 'thread apply all backtrace' -p {}".format(server_pid) + cmd = f"gdb -batch -ex 'thread apply all backtrace' -p {server_pid}" return subprocess.check_output(cmd, shell=True).decode('utf-8') - except Exception as ex: - print("Error occured while receiving stack traces from gdb: {}".format(str(ex))) + except Exception as e: + print(f"Error occurred while receiving stack traces from gdb: {e}") return None # collect server stacktraces from system.stack_trace table # it does not work in Sandbox def get_stacktraces_from_clickhouse(client, replicated_database=False): - try: - if replicated_database: - return subprocess.check_output("{} --allow_introspection_functions=1 --skip_unavailable_shards=1 --query " - "\"SELECT materialize((hostName(), tcpPort())) as host, thread_id, " - "arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), " - "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace " - "FROM clusterAllReplicas('test_cluster_database_replicated', 'system.stack_trace') " - "ORDER BY host, thread_id format Vertical\"".format(client), shell=True, stderr=subprocess.STDOUT).decode('utf-8') + replicated_msg = \ + "{} --allow_introspection_functions=1 --skip_unavailable_shards=1 --query \ + \"SELECT materialize((hostName(), tcpPort())) as host, thread_id, \ + arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), \ + arrayMap(x -> addressToLine(x), trace), \ + arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace \ + FROM clusterAllReplicas('test_cluster_database_replicated', 'system.stack_trace') \ + ORDER BY host, thread_id FORMAT Vertical\"".format(client) - return subprocess.check_output("{} --allow_introspection_functions=1 --query " - "\"SELECT arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), " - "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace " - "FROM system.stack_trace format Vertical\"".format(client), shell=True, stderr=subprocess.STDOUT).decode('utf-8') - except Exception as ex: - print("Error occured while receiving stack traces from client: {}".format(str(ex))) + msg = \ + "{} --allow_introspection_functions=1 --query \ + \"SELECT arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), \ + arrayMap(x -> addressToLine(x), trace), \ + arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace \ + FROM system.stack_trace FORMAT Vertical\"".format(client) + + try: + return subprocess.check_output( + replicated_msg if replicated_database else msg, + shell=True, stderr=subprocess.STDOUT).decode('utf-8') + except Exception as e: + print(f"Error occurred while receiving stack traces from client: {e}") return None -def get_server_pid(server_tcp_port): + +def print_stacktraces() -> None: + server_pid = get_server_pid() + + bt = None + + if server_pid and not args.replicated_database: + print("") + print(f"Located ClickHouse server process {server_pid} listening at TCP port {args.tcp_port}") + print("Collecting stacktraces from all running threads with gdb:") + + bt = get_stacktraces_from_gdb(server_pid) + + if len(bt) < 1000: + print("Got suspiciously small stacktraces: ", bt) + bt = None + + if bt is None: + print("\nCollecting stacktraces from system.stacktraces table:") + + bt = get_stacktraces_from_clickhouse( + args.client, args.replicated_database) + + if bt is not None: + print(bt) + return + + print(colored( + f"\nUnable to locate ClickHouse server process listening at TCP port {args.tcp_port}. " + "It must have crashed or exited prematurely!", + args, "red", attrs=["bold"])) + + +def get_server_pid(): # lsof does not work in stress tests for some reason - cmd_lsof = "lsof -i tcp:{port} -s tcp:LISTEN -Fp | awk '/^p[0-9]+$/{{print substr($0, 2)}}'".format(port=server_tcp_port) + cmd_lsof = f"lsof -i tcp:{args.tcp_port} -s tcp:LISTEN -Fp | awk '/^p[0-9]+$/{{print substr($0, 2)}}'" cmd_pidof = "pidof -s clickhouse-server" + commands = [cmd_lsof, cmd_pidof] output = None + for cmd in commands: try: output = subprocess.check_output(cmd, shell=True, stderr=subprocess.STDOUT, universal_newlines=True) if output: return int(output) except Exception as e: - print("Cannot get server pid with {}, got {}: {}".format(cmd, output, e)) - return None # most likely server dead + print(f"Cannot get server pid with {cmd}, got {output}: {e}") + + return None # most likely server is dead def colored(text, args, color=None, on_color=None, attrs=None): @@ -357,6 +418,14 @@ server_died = multiprocessing.Event() stop_tests_triggered_lock = multiprocessing.Lock() stop_tests_triggered = multiprocessing.Event() queue = multiprocessing.Queue(maxsize=1) + + +def print_test_time(test_time) -> str: + if args.print_time: + return " {0:.2f} sec.".format(test_time) + else: + return '' + restarted_tests = [] # (test, stderr) # def run_tests_array(all_tests, suite, suite_dir, suite_tmp_dir, run_total): @@ -385,15 +454,10 @@ def run_tests_array(all_tests_with_params): client_options = get_additional_client_options(args) - def print_test_time(test_time): - if args.print_time: - return " {0:.2f} sec.".format(test_time) - else: - return '' - if num_tests > 0: about = 'about ' if is_concurrent else '' - print(f"\nRunning {about}{num_tests} {suite} tests ({multiprocessing.current_process().name}).\n") + proc_name = multiprocessing.current_process().name + print(f"\nRunning {about}{num_tests} {suite} tests ({proc_name}).\n") while True: if is_concurrent: @@ -459,7 +523,6 @@ def run_tests_array(all_tests_with_params): message = open(disabled_file, 'r').read() status += MSG_SKIPPED + " - " + message + "\n" else: - if args.testname: clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) failed_to_check = False @@ -599,7 +662,12 @@ def run_tests_array(all_tests_with_params): except: exc_type, exc_value, tb = sys.exc_info() failures += 1 - print("{0} - Test internal error: {1}\n{2}\n{3}".format(MSG_FAIL, exc_type.__name__, exc_value, "\n".join(traceback.format_tb(tb, 10)))) + + exc_name = exc_type.__name__ + traceback_str = "\n".join(traceback.format_tb(tb, 10)) + + print(f"{MSG_FAIL} - Test internal error: {exc_name}") + print(f"{exc_value}\n{traceback_str}") if failures_chain >= 20: stop_tests() @@ -627,9 +695,11 @@ server_logs_level = "warning" def check_server_started(client, retry_count): print("Connecting to ClickHouse server...", end='') + sys.stdout.flush() + while retry_count > 0: - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT 1") if clickhouse_proc.returncode == 0 and stdout.startswith(b"1"): @@ -679,7 +749,7 @@ class BuildFlags(): def collect_build_flags(client): - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") result = [] @@ -695,7 +765,7 @@ def collect_build_flags(client): else: raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'") if clickhouse_proc.returncode == 0: @@ -706,7 +776,7 @@ def collect_build_flags(client): else: raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'") if clickhouse_proc.returncode == 0: @@ -715,7 +785,7 @@ def collect_build_flags(client): else: raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.settings WHERE name = 'default_database_engine'") if clickhouse_proc.returncode == 0: @@ -724,7 +794,7 @@ def collect_build_flags(client): else: raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'") if clickhouse_proc.returncode == 0: @@ -736,6 +806,56 @@ def collect_build_flags(client): return result +def suite_key_func(item: str) -> Union[int, Tuple[int, str]]: + if args.order == 'random': + return random.random() + + if -1 == item.find('_'): + return 99998, '' + + prefix, suffix = item.split('_', 1) + + try: + return int(prefix), suffix + except ValueError: + return 99997, '' + + +def tests_in_suite_key_func(item: str) -> int: + if args.order == 'random': + return random.random() + + reverse = 1 if args.order == 'asc' else -1 + + if -1 == item.find('_'): + return 99998 + + prefix, _ = item.split('_', 1) + + try: + return reverse * int(prefix) + except ValueError: + return 99997 + + +def extract_key(key: str) -> str: + return subprocess.getstatusoutput( + args.extract_from_config + + " --try --config " + + args.configserver + key)[1] + + +def open_client_process( + client_args: str, + universal_newlines: bool = False, + stderr_file: Optional[TextIO] = None): + return Popen( + shlex.split(client_args), stdin=PIPE, stdout=PIPE, + stderr=stderr_file if stderr_file is not None else PIPE, + universal_newlines=True if universal_newlines else None) + + + def do_run_tests(jobs, suite, suite_dir, suite_tmp_dir, all_tests, parallel_tests, sequential_tests, parallel): if jobs > 1 and len(parallel_tests) > 0: print("Found", len(parallel_tests), "parallel tests and", len(sequential_tests), "sequential tests") @@ -790,7 +910,7 @@ def removesuffix(text, *suffixes): Added in python 3.9 https://www.python.org/dev/peps/pep-0616/ - This version can work with severtal possible suffixes + This version can work with several possible suffixes """ for suffix in suffixes: if suffix and text.endswith(suffix): @@ -875,7 +995,7 @@ def main(args): global server_logs_level def is_data_present(): - clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(args.client) (stdout, stderr) = clickhouse_proc.communicate(b"EXISTS TABLE test.hits") if clickhouse_proc.returncode != 0: raise CalledProcessError(clickhouse_proc.returncode, args.client, stderr) @@ -885,9 +1005,10 @@ def main(args): if not check_server_started(args.client, args.server_check_retries): raise Exception( "Server is not responding. Cannot execute 'SELECT 1' query. \ - Note: if you are using split build, you may have to specify -c option.") + If you are using split build, you have to specify -c option.") build_flags = collect_build_flags(args.client) + if args.replicated_database: build_flags.append(BuildFlags.DATABASE_REPLICATED) @@ -911,6 +1032,7 @@ def main(args): os.environ.setdefault("CLICKHOUSE_BINARY", args.binary) #os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver) + if args.configclient: os.environ.setdefault("CLICKHOUSE_CONFIG_CLIENT", args.configclient) @@ -923,52 +1045,35 @@ def main(args): stop_time = time() + args.global_time_limit if args.zookeeper is None: - _, out = subprocess.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l') try: - if int(out) > 0: - args.zookeeper = True - else: - args.zookeeper = False + args.zookeeper = int(extract_key(" --key zookeeper | grep . | wc -l")) > 0 except ValueError: args.zookeeper = False if args.shard is None: - _, out = subprocess.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key listen_host | grep -E "127.0.0.2|::"') - if out: - args.shard = True - else: - args.shard = False + args.shard = bool(extract_key(' --key listen_host | grep -E "127.0.0.2|::"')) def create_common_database(args, db_name): create_database_retries = 0 while create_database_retries < MAX_RETRIES: client_cmd = args.client + " " + get_additional_client_options(args) - clickhouse_proc_create = Popen(shlex.split(client_cmd), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) + + clickhouse_proc_create = open_client_process(client_cmd, universal_newlines=True) + (stdout, stderr) = clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name))) + if not need_retry(stdout, stderr): break create_database_retries += 1 if args.database and args.database != "test": create_common_database(args, args.database) + create_common_database(args, "test") - def sute_key_func(item): - if args.order == 'random': - return random.random() - - if -1 == item.find('_'): - return 99998, '' - - prefix, suffix = item.split('_', 1) - - try: - return int(prefix), suffix - except ValueError: - return 99997, '' - total_tests_run = 0 - for suite in sorted(os.listdir(base_dir), key=sute_key_func): + + for suite in sorted(os.listdir(base_dir), key=suite_key_func): if server_died.is_set(): break @@ -982,8 +1087,8 @@ def main(args): os.makedirs(suite_tmp_dir) suite = suite_re_obj.group(1) - if os.path.isdir(suite_dir): + if os.path.isdir(suite_dir): if 'stateful' in suite and not args.no_stateful and not is_data_present(): print("Won't run stateful tests because test data wasn't loaded.") continue @@ -994,29 +1099,14 @@ def main(args): print("Won't run stateful tests because they were manually disabled.") continue - # Reverse sort order: we want run newest test first. - # And not reverse subtests - def key_func(item): - if args.order == 'random': - return random.random() - - reverse = 1 if args.order == 'asc' else -1 - - if -1 == item.find('_'): - return 99998 - - prefix, _ = item.split('_', 1) - - try: - return reverse * int(prefix) - except ValueError: - return 99997 - - all_tests = get_tests_list(suite_dir, args.test, args.test_runs, key_func) + all_tests = get_tests_list( + suite_dir, args.test, args.test_runs, tests_in_suite_key_func) jobs = args.jobs + parallel_tests = [] sequential_tests = [] + for test in all_tests: if any(s in test for s in args.sequential): sequential_tests.append(test) @@ -1042,44 +1132,21 @@ def main(args): else: print(colored("Seems like server hung and cannot respond to queries", args, "red", attrs=["bold"])) - clickhouse_tcp_port = os.getenv("CLICKHOUSE_PORT_TCP", '9000') - server_pid = get_server_pid(clickhouse_tcp_port) - bt = None - if server_pid and not args.replicated_database: - print("\nLocated ClickHouse server process {} listening at TCP port {}".format(server_pid, clickhouse_tcp_port)) - print("\nCollecting stacktraces from all running threads with gdb:") - bt = get_stacktraces_from_gdb(server_pid) - if len(bt) < 1000: - print("Got suspiciously small stacktraces: ", bt) - bt = None - if bt is None: - print("\nCollecting stacktraces from system.stacktraces table:") - bt = get_stacktraces_from_clickhouse(args.client, args.replicated_database) - if bt is None: - print( - colored( - "\nUnable to locate ClickHouse server process listening at TCP port {}. " - "It must have crashed or exited prematurely!".format(clickhouse_tcp_port), - args, "red", attrs=["bold"])) - else: - print(bt) + print_stacktraces() exit_code.value = 1 else: print(colored("\nNo queries hung.", args, "green", attrs=["bold"])) if len(restarted_tests) > 0: print("\nSome tests were restarted:\n") + for (test_case, stderr) in restarted_tests: - print(test_case) - print(stderr) - print("\n") + print(test_case + "\n" + stderr + "\n") if total_tests_run == 0: print("No tests were run.") sys.exit(1) - else: - print("All tests have finished.") sys.exit(exit_code.value) @@ -1196,9 +1263,11 @@ if __name__ == '__main__': parser.add_argument('--no-long', action='store_false', dest='no_long', help='Do not run long tests') parser.add_argument('--client-option', nargs='+', help='Specify additional client argument') parser.add_argument('--print-time', action='store_true', dest='print_time', help='Print test time') + group=parser.add_mutually_exclusive_group(required=False) group.add_argument('--zookeeper', action='store_true', default=None, dest='zookeeper', help='Run zookeeper related tests') group.add_argument('--no-zookeeper', action='store_false', default=None, dest='zookeeper', help='Do not run zookeeper related tests') + group=parser.add_mutually_exclusive_group(required=False) group.add_argument('--shard', action='store_true', default=None, dest='shard', help='Run sharding related tests (required to clickhouse-server listen 127.0.0.2 127.0.0.3)') group.add_argument('--no-shard', action='store_false', default=None, dest='shard', help='Do not run shard related tests') @@ -1206,7 +1275,7 @@ if __name__ == '__main__': args = parser.parse_args() if args.queries and not os.path.isdir(args.queries): - print("Cannot access the specified directory with queries (" + args.queries + ")", file=sys.stderr) + print(f"Cannot access the specified directory with queries ({args.queries})", file=sys.stderr) sys.exit(1) # Autodetect the directory with queries if not specified @@ -1257,10 +1326,13 @@ if __name__ == '__main__': if args.configclient: args.client += ' --config-file=' + args.configclient + if os.getenv("CLICKHOUSE_HOST"): args.client += ' --host=' + os.getenv("CLICKHOUSE_HOST") - if os.getenv("CLICKHOUSE_PORT_TCP"): - args.client += ' --port=' + os.getenv("CLICKHOUSE_PORT_TCP") + + args.tcp_port = int(os.getenv("CLICKHOUSE_PORT_TCP", 9000)) + args.client += f" --port={args.tcp_port}" + if os.getenv("CLICKHOUSE_DATABASE"): args.client += ' --database=' + os.getenv("CLICKHOUSE_DATABASE") From a308c8f8427b3f58c5ca2b879fc564a5c6bfbc4a Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Fri, 6 Aug 2021 17:38:28 +0300 Subject: [PATCH 097/147] Extracted test skipping functions --- tests/clickhouse-test | 357 ++++++++++++++++++++++++------------------ 1 file changed, 201 insertions(+), 156 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 212ccd79f00..b734af0bdea 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -54,7 +54,7 @@ MESSAGES_TO_RETRY = [ "DB::Exception: New table appeared in database being dropped or detached. Try again", "is already started to be removing by another replica right now", "Shutdown is called for table", # It happens in SYSTEM SYNC REPLICA query if session with ZooKeeper is being reinitialized. - DISTRIBUTED_DDL_TIMEOUT_MSG # FIXME + DISTRIBUTED_DDL_TIMEOUT_MSG # FIXME ] MAX_RETRIES = 3 @@ -426,6 +426,67 @@ def print_test_time(test_time) -> str: else: return '' + +def should_skip_test_by_name(name: str, test_ext: str) -> Tuple[bool, str]: + if args.skip and any(s in name for s in args.skip): + return True, "skip" + + if not args.zookeeper and ('zookeeper' in name or 'replica' in name): + return True, "no zookeeper" + + if not args.shard and \ + ('shard' in name or 'distributed' in name or 'global' in name): + return True, "no shard" + + # Tests for races and deadlocks usually are run in a loop for a significant + # amount of time + if args.no_long and \ + ('long' in name or 'deadlock' in name or 'race' in name): + return True, "no long" + + if not USE_JINJA and test_ext.endswith("j2"): + return True, "no jinja" + + return False, "" + + +def should_skip_disabled_test(name: str, suite_dir: str) -> Tuple[bool, str]: + disabled_file = os.path.join(suite_dir, name) + '.disabled' + + if os.path.exists(disabled_file) and not args.disabled: + return True, open(disabled_file, 'r').read() + + return False, "" + + +# should skip test, should increment skipped_total, skip reason +def should_skip_test(name: str, test_ext: str, suite_dir: str) -> Tuple[bool, bool, str]: + should_skip, skip_reason = should_skip_test_by_name(name, test_ext) + + if should_skip: + return True, True, skip_reason + + should_skip, skip_reason = should_skip_disabled_test(name, suite_dir) + + return should_skip, False, skip_reason + + +def send_test_name_failed(suite: str, case: str) -> bool: + clickhouse_proc = open_client_process(args.client, universal_newlines=True) + + failed_to_check = False + + pid = os.getpid() + query = f"SELECT 'Running test {suite}/{case} from pid={pid}';" + + try: + clickhouse_proc.communicate((query), timeout=20) + except: + failed_to_check = True + + return failed_to_check or clickhouse_proc.returncode != 0 + + restarted_tests = [] # (test, stderr) # def run_tests_array(all_tests, suite, suite_dir, suite_tmp_dir, run_total): @@ -494,161 +555,135 @@ def run_tests_array(all_tests_with_params): else: status = "{0:72}".format(removesuffix(name, ".gen", ".sql") + ": ") - if args.skip and any(s in name for s in args.skip): - status += MSG_SKIPPED + " - skip\n" - skipped_total += 1 - elif not args.zookeeper and ('zookeeper' in name - or 'replica' in name): - status += MSG_SKIPPED + " - no zookeeper\n" - skipped_total += 1 - elif not args.shard and ('shard' in name - or 'distributed' in name - or 'global' in name): - status += MSG_SKIPPED + " - no shard\n" - skipped_total += 1 - elif not args.no_long and ('long' in name - # Tests for races and deadlocks usually are runned in loop - # for significant amount of time - or 'deadlock' in name - or 'race' in name): - status += MSG_SKIPPED + " - no long\n" - skipped_total += 1 - elif not USE_JINJA and ext.endswith("j2"): - status += MSG_SKIPPED + " - no jinja\n" - skipped_total += 1 + skip_test, increment_skip_count, skip_reason = \ + should_skip_test(name, ext, suite_dir) + + if skip_test: + status += MSG_SKIPPED + f" - {skip_reason}\n" + + if increment_skip_count: + skipped_total += 1 else: - disabled_file = os.path.join(suite_dir, name) + '.disabled' + if args.testname and send_test_name_failed(suite, case): + failures += 1 + print("Server does not respond to health check") + server_died.set() + stop_tests() + break - if os.path.exists(disabled_file) and not args.disabled: - message = open(disabled_file, 'r').read() - status += MSG_SKIPPED + " - " + message + "\n" + file_suffix = ('.' + str(os.getpid())) if is_concurrent and args.test_runs > 1 else '' + reference_file = get_reference_file(suite_dir, name) + stdout_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stdout' + stderr_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stderr' + + testcase_args = configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file) + proc, stdout, stderr, total_time = run_single_test(testcase_args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file) + + if proc.returncode is None: + try: + proc.kill() + except OSError as e: + if e.errno != ESRCH: + raise + + failures += 1 + status += MSG_FAIL + status += print_test_time(total_time) + status += " - Timeout!\n" + if stderr: + status += stderr + status += 'Database: ' + testcase_args.testcase_database else: - if args.testname: - clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) - failed_to_check = False - try: - clickhouse_proc.communicate(("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)), timeout=20) - except: - failed_to_check = True - - if failed_to_check or clickhouse_proc.returncode != 0: - failures += 1 - print("Server does not respond to health check") - server_died.set() - stop_tests() + counter = 1 + while need_retry(stdout, stderr): + restarted_tests.append((case_file, stderr)) + testcase_args = configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file) + proc, stdout, stderr, total_time = run_single_test(testcase_args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file) + sleep(2**counter) + counter += 1 + if MAX_RETRIES < counter: + if args.replicated_database: + if DISTRIBUTED_DDL_TIMEOUT_MSG in stderr: + server_died.set() break - file_suffix = ('.' + str(os.getpid())) if is_concurrent and args.test_runs > 1 else '' - reference_file = get_reference_file(suite_dir, name) - stdout_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stdout' - stderr_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stderr' - - testcase_args = configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file) - proc, stdout, stderr, total_time = run_single_test(testcase_args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file) - - if proc.returncode is None: - try: - proc.kill() - except OSError as e: - if e.errno != ESRCH: - raise - + if proc.returncode != 0: failures += 1 + failures_chain += 1 status += MSG_FAIL status += print_test_time(total_time) - status += " - Timeout!\n" + status += ' - return code {}\n'.format(proc.returncode) + if stderr: status += stderr + + # Stop on fatal errors like segmentation fault. They are sent to client via logs. + if ' ' in stderr: + server_died.set() + + if testcase_args.stop \ + and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) \ + and 'Received exception from server' not in stderr: + server_died.set() + + if os.path.isfile(stdout_file): + status += ", result:\n\n" + status += '\n'.join( + open(stdout_file).read().split('\n')[:100]) + status += '\n' + + status += 'Database: ' + testcase_args.testcase_database + + elif stderr: + failures += 1 + failures_chain += 1 + status += MSG_FAIL + status += print_test_time(total_time) + status += " - having stderror:\n{}\n".format( + '\n'.join(stderr.split('\n')[:100])) + status += 'Database: ' + testcase_args.testcase_database + elif 'Exception' in stdout: + failures += 1 + failures_chain += 1 + status += MSG_FAIL + status += print_test_time(total_time) + status += " - having exception:\n{}\n".format( + '\n'.join(stdout.split('\n')[:100])) + status += 'Database: ' + testcase_args.testcase_database + elif reference_file is None: + status += MSG_UNKNOWN + status += print_test_time(total_time) + status += " - no reference file\n" status += 'Database: ' + testcase_args.testcase_database else: - counter = 1 - while need_retry(stdout, stderr): - restarted_tests.append((case_file, stderr)) - testcase_args = configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file) - proc, stdout, stderr, total_time = run_single_test(testcase_args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file) - sleep(2**counter) - counter += 1 - if MAX_RETRIES < counter: - if args.replicated_database: - if DISTRIBUTED_DDL_TIMEOUT_MSG in stderr: - server_died.set() - break + result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout=PIPE) - if proc.returncode != 0: + if result_is_different: + diff = Popen(['diff', '-U', str(testcase_args.unified), reference_file, stdout_file], stdout=PIPE, universal_newlines=True).communicate()[0] failures += 1 - failures_chain += 1 status += MSG_FAIL status += print_test_time(total_time) - status += ' - return code {}\n'.format(proc.returncode) - - if stderr: - status += stderr - - # Stop on fatal errors like segmentation fault. They are sent to client via logs. - if ' ' in stderr: - server_died.set() - - if testcase_args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr: - server_died.set() - - if os.path.isfile(stdout_file): - status += ", result:\n\n" - status += '\n'.join( - open(stdout_file).read().split('\n')[:100]) - status += '\n' - - status += 'Database: ' + testcase_args.testcase_database - - elif stderr: - failures += 1 - failures_chain += 1 - status += MSG_FAIL - status += print_test_time(total_time) - status += " - having stderror:\n{}\n".format( - '\n'.join(stderr.split('\n')[:100])) - status += 'Database: ' + testcase_args.testcase_database - elif 'Exception' in stdout: - failures += 1 - failures_chain += 1 - status += MSG_FAIL - status += print_test_time(total_time) - status += " - having exception:\n{}\n".format( - '\n'.join(stdout.split('\n')[:100])) - status += 'Database: ' + testcase_args.testcase_database - elif reference_file is None: - status += MSG_UNKNOWN - status += print_test_time(total_time) - status += " - no reference file\n" + status += " - result differs with reference:\n{}\n".format(diff) status += 'Database: ' + testcase_args.testcase_database else: - result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout=PIPE) - - if result_is_different: - diff = Popen(['diff', '-U', str(testcase_args.unified), reference_file, stdout_file], stdout=PIPE, universal_newlines=True).communicate()[0] + if testcase_args.test_runs > 1 and total_time > 60 and 'long' not in name: + # We're in Flaky Check mode, check the run time as well while we're at it. failures += 1 + failures_chain += 1 status += MSG_FAIL status += print_test_time(total_time) - status += " - result differs with reference:\n{}\n".format(diff) + status += " - Test runs too long (> 60s). Make it faster.\n" status += 'Database: ' + testcase_args.testcase_database else: - if testcase_args.test_runs > 1 and total_time > 60 and 'long' not in name: - # We're in Flaky Check mode, check the run time as well while we're at it. - failures += 1 - failures_chain += 1 - status += MSG_FAIL - status += print_test_time(total_time) - status += " - Test runs too long (> 60s). Make it faster.\n" - status += 'Database: ' + testcase_args.testcase_database - else: - passed_total += 1 - failures_chain = 0 - status += MSG_OK - status += print_test_time(total_time) - status += "\n" - if os.path.exists(stdout_file): - os.remove(stdout_file) - if os.path.exists(stderr_file): - os.remove(stderr_file) + passed_total += 1 + failures_chain = 0 + status += MSG_OK + status += print_test_time(total_time) + status += "\n" + if os.path.exists(stdout_file): + os.remove(stdout_file) + if os.path.exists(stderr_file): + os.remove(stderr_file) if status and not status.endswith('\n'): status += '\n' @@ -709,27 +744,30 @@ def check_server_started(client, retry_count): if clickhouse_proc.returncode == 210: # Connection refused, retry - print('.', end = '') + print('.', end='') sys.stdout.flush() retry_count -= 1 sleep(0.5) continue - # Other kind of error, fail. - print('') - print("Client invocation failed with code ", clickhouse_proc.returncode, ": ") + # FIXME Some old comment, maybe now CH supports Python3 ? # We can't print this, because for some reason this is python 2, # and args appeared in 3.3. To hell with it. # print(''.join(clickhouse_proc.args)) - print("stdout: ") - print(stdout) - print("stderr: ") - print(stderr) + + # Other kind of error, fail. + + code: int = clickhouse_proc.returncode + + print(f"\nClient invocation failed with code {code}:\n\ + stdout: {stdout}\n\ + stderr: {stderr}") + sys.stdout.flush() + return False - print('') - print('All connection tries failed') + print('\nAll connection tries failed') sys.stdout.flush() return False @@ -928,7 +966,7 @@ def render_test_template(j2env, suite_dir, test_name): test_base_name = removesuffix(test_name, ".sql.j2", ".sql") - reference_file_name = test_base_name + ".reference.j2" + reference_file_name = test_base_name + ".reference.j2" reference_file_path = os.path.join(suite_dir, reference_file_name) if os.path.isfile(reference_file_path): tpl = j2env.get_template(reference_file_name) @@ -1015,7 +1053,7 @@ def main(args): if args.use_skip_list: tests_to_skip_from_list = collect_tests_to_skip(args.skip_list_path, build_flags) else: - tests_to_skip_from_list = set([]) + tests_to_skip_from_list = set() if args.skip: args.skip = set(args.skip) | tests_to_skip_from_list @@ -1030,7 +1068,7 @@ def main(args): # Keep same default values as in queries/shell_config.sh os.environ.setdefault("CLICKHOUSE_BINARY", args.binary) - #os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) + # os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver) if args.configclient: @@ -1079,7 +1117,7 @@ def main(args): suite_dir = os.path.join(base_dir, suite) suite_re_obj = re.search('^[0-9]+_(.*)$', suite) - if not suite_re_obj: #skip .gitignore and so on + if not suite_re_obj: # skip .gitignore and so on continue suite_tmp_dir = os.path.join(tmp_dir, suite) @@ -1147,6 +1185,8 @@ def main(args): if total_tests_run == 0: print("No tests were run.") sys.exit(1) + else: + print("All tests have finished.") sys.exit(exit_code.value) @@ -1183,18 +1223,23 @@ def get_additional_client_options_url(args): def collect_tests_to_skip(skip_list_path, build_flags): result = set([]) + if not os.path.exists(skip_list_path): return result with open(skip_list_path, 'r') as skip_list_file: content = skip_list_file.read() + # allows to have comments in skip_list.json skip_dict = json.loads(json_minify(content)) + for build_flag in build_flags: result |= set(skip_dict[build_flag]) - if len(result) > 0: - print("Found file with skip-list {}, {} test will be skipped".format(skip_list_path, len(result))) + count = len(result) + + if count > 0: + print(f"Found file with skip-list {skip_list_path}, {count} test will be skipped") return result @@ -1221,7 +1266,7 @@ if __name__ == '__main__': signal.signal(signal.SIGINT, signal_handler) signal.signal(signal.SIGHUP, signal_handler) - parser=ArgumentParser(description='ClickHouse functional tests') + parser = ArgumentParser(description='ClickHouse functional tests') parser.add_argument('-q', '--queries', help='Path to queries dir') parser.add_argument('--tmp', help='Path to tmp dir') @@ -1233,7 +1278,7 @@ if __name__ == '__main__': parser.add_argument('--extract_from_config', help='extract-from-config program') parser.add_argument('--configclient', help='Client config (if you use not default ports)') - parser.add_argument('--configserver', default= '/etc/clickhouse-server/config.xml', help='Preprocessed server config') + parser.add_argument('--configserver', default='/etc/clickhouse-server/config.xml', help='Preprocessed server config') parser.add_argument('-o', '--output', help='Output xUnit compliant test report directory') parser.add_argument('-t', '--timeout', type=int, default=600, help='Timeout for each test case in seconds') parser.add_argument('--global_time_limit', type=int, help='Stop if executing more than specified time (after current test finished)') @@ -1260,15 +1305,15 @@ if __name__ == '__main__': parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests') parser.add_argument('--skip', nargs='+', help="Skip these tests") parser.add_argument('--sequential', nargs='+', help="Run these tests sequentially even if --parallel specified") - parser.add_argument('--no-long', action='store_false', dest='no_long', help='Do not run long tests') + parser.add_argument('--no-long', action='store_true', dest='no_long', help='Do not run long tests') parser.add_argument('--client-option', nargs='+', help='Specify additional client argument') parser.add_argument('--print-time', action='store_true', dest='print_time', help='Print test time') - group=parser.add_mutually_exclusive_group(required=False) + group = parser.add_mutually_exclusive_group(required=False) group.add_argument('--zookeeper', action='store_true', default=None, dest='zookeeper', help='Run zookeeper related tests') group.add_argument('--no-zookeeper', action='store_false', default=None, dest='zookeeper', help='Do not run zookeeper related tests') - group=parser.add_mutually_exclusive_group(required=False) + group = parser.add_mutually_exclusive_group(required=False) group.add_argument('--shard', action='store_true', default=None, dest='shard', help='Run sharding related tests (required to clickhouse-server listen 127.0.0.2 127.0.0.3)') group.add_argument('--no-shard', action='store_false', default=None, dest='shard', help='Do not run shard related tests') @@ -1284,7 +1329,7 @@ if __name__ == '__main__': if not os.path.isdir(args.queries): # If we're running from the repo - args.queries = os.path.join(os.path.dirname(os.path.abspath( __file__ )), 'queries') + args.queries = os.path.join(os.path.dirname(os.path.abspath(__file__)), 'queries') if not os.path.isdir(args.queries): # Next we're going to try some system directories, don't write 'stdout' files into them. @@ -1330,7 +1375,7 @@ if __name__ == '__main__': if os.getenv("CLICKHOUSE_HOST"): args.client += ' --host=' + os.getenv("CLICKHOUSE_HOST") - args.tcp_port = int(os.getenv("CLICKHOUSE_PORT_TCP", 9000)) + args.tcp_port = int(os.getenv("CLICKHOUSE_PORT_TCP", "9000")) args.client += f" --port={args.tcp_port}" if os.getenv("CLICKHOUSE_DATABASE"): From 1f60cd735daf2f79f5bd7b949a2776cda631dae3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 17:52:23 +0300 Subject: [PATCH 098/147] Revert changes in 01710_projection_fetch --- .../queries/0_stateless/01710_projection_fetch.reference | 8 ++++---- tests/queries/0_stateless/01710_projection_fetch.sql | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01710_projection_fetch.reference b/tests/queries/0_stateless/01710_projection_fetch.reference index 6fc199d6ba2..abce5410b26 100644 --- a/tests/queries/0_stateless/01710_projection_fetch.reference +++ b/tests/queries/0_stateless/01710_projection_fetch.reference @@ -10,8 +10,8 @@ 3 3 4 4 0 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 2 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 diff --git a/tests/queries/0_stateless/01710_projection_fetch.sql b/tests/queries/0_stateless/01710_projection_fetch.sql index 5781d63788f..7e4f6cc1d9a 100644 --- a/tests/queries/0_stateless/01710_projection_fetch.sql +++ b/tests/queries/0_stateless/01710_projection_fetch.sql @@ -1,9 +1,9 @@ drop table if exists tp_1; drop table if exists tp_2; -create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/{shard}/01710_projection_fetch_' || currentDatabase(), '1_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; +create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '1_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; -create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/{shard}/01710_projection_fetch_' || currentDatabase(), '2_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; +create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '2_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; insert into tp_1 select number, number from numbers(3); From 27421284297ad34bf3b114ecbf19cadc0082c709 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 6 Aug 2021 18:03:05 +0300 Subject: [PATCH 099/147] Try update contrib/zlib-ng --- contrib/zlib-ng | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/zlib-ng b/contrib/zlib-ng index db232d30b4c..6a5e93b9007 160000 --- a/contrib/zlib-ng +++ b/contrib/zlib-ng @@ -1 +1 @@ -Subproject commit db232d30b4c72fd58e6d7eae2d12cebf9c3d90db +Subproject commit 6a5e93b9007782115f7f7e5235dedc81c4f1facb From 3a453636e8771c42009c0ed4e41dd2aed75f72db Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 6 Aug 2021 18:12:36 +0300 Subject: [PATCH 100/147] Update Dockerfile --- docker/test/pvs/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/pvs/Dockerfile b/docker/test/pvs/Dockerfile index 2983be2305f..7bd45ba4018 100644 --- a/docker/test/pvs/Dockerfile +++ b/docker/test/pvs/Dockerfile @@ -28,7 +28,7 @@ RUN apt-get update --yes \ ENV PKG_VERSION="pvs-studio-latest" RUN set -x \ - && export PUBKEY_HASHSUM="486a0694c7f92e96190bbfac01c3b5ac2cb7823981db510a28f744c99eabbbf17a7bcee53ca42dc6d84d4323c2742761" \ + && export PUBKEY_HASHSUM="686e5eb8b3c543a5c54442c39ec876b6c2d912fe8a729099e600017ae53c877dda3368fe38ed7a66024fe26df6b5892a" \ && wget -nv https://files.viva64.com/etc/pubkey.txt -O /tmp/pubkey.txt \ && echo "${PUBKEY_HASHSUM} /tmp/pubkey.txt" | sha384sum -c \ && apt-key add /tmp/pubkey.txt \ From 28901ef55f7c4a71b346329a27aae0549816be3e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 17:25:24 +0200 Subject: [PATCH 101/147] Fix 01300_client_save_history_when_terminated_long It was broken because of colored output --- .../01300_client_save_history_when_terminated_long.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect index 7fed383dc38..0c53f5658d1 100755 --- a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect +++ b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect @@ -27,7 +27,7 @@ close spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT" expect ":) " send -- "\[A" -expect "SELECT 'for the history'" +expect "for the history" # Will check that Ctrl+C clears current line. send -- "\3" From 31a9b6d5cae0799f6cadf092810b4cae1ba2569f Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 6 Aug 2021 17:38:55 +0200 Subject: [PATCH 102/147] Less clock_gettime calls --- src/DataStreams/ExecutionSpeedLimits.cpp | 13 +++++++++---- src/DataStreams/ExecutionSpeedLimits.h | 3 ++- src/DataStreams/IBlockInputStream.cpp | 2 +- src/Processors/Sources/SourceWithProgress.cpp | 2 +- .../Transforms/LimitsCheckingTransform.cpp | 2 +- 5 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/DataStreams/ExecutionSpeedLimits.cpp b/src/DataStreams/ExecutionSpeedLimits.cpp index 6cc1b9006bf..81afd4fe984 100644 --- a/src/DataStreams/ExecutionSpeedLimits.cpp +++ b/src/DataStreams/ExecutionSpeedLimits.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace ProfileEvents @@ -104,14 +105,18 @@ static bool handleOverflowMode(OverflowMode mode, const String & message, int co } } -bool ExecutionSpeedLimits::checkTimeLimit(UInt64 elapsed_ns, OverflowMode overflow_mode) const +bool ExecutionSpeedLimits::checkTimeLimit(const Stopwatch & stopwatch, OverflowMode overflow_mode) const { - if (max_execution_time != 0 - && elapsed_ns > static_cast(max_execution_time.totalMicroseconds()) * 1000) - return handleOverflowMode(overflow_mode, + if (max_execution_time != 0) + { + auto elapsed_ns = stopwatch.elapsed(); + + if (elapsed_ns > static_cast(max_execution_time.totalMicroseconds()) * 1000) + return handleOverflowMode(overflow_mode, "Timeout exceeded: elapsed " + toString(static_cast(elapsed_ns) / 1000000000ULL) + " seconds, maximum: " + toString(max_execution_time.totalMicroseconds() / 1000000.0), ErrorCodes::TIMEOUT_EXCEEDED); + } return true; } diff --git a/src/DataStreams/ExecutionSpeedLimits.h b/src/DataStreams/ExecutionSpeedLimits.h index 9ab58e12cf4..d52dc713c1a 100644 --- a/src/DataStreams/ExecutionSpeedLimits.h +++ b/src/DataStreams/ExecutionSpeedLimits.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -25,7 +26,7 @@ public: /// Pause execution in case if speed limits were exceeded. void throttle(size_t read_rows, size_t read_bytes, size_t total_rows_to_read, UInt64 total_elapsed_microseconds) const; - bool checkTimeLimit(UInt64 elapsed_ns, OverflowMode overflow_mode) const; + bool checkTimeLimit(const Stopwatch & stopwatch, OverflowMode overflow_mode) const; }; } diff --git a/src/DataStreams/IBlockInputStream.cpp b/src/DataStreams/IBlockInputStream.cpp index c3071cdcf20..e57d6903673 100644 --- a/src/DataStreams/IBlockInputStream.cpp +++ b/src/DataStreams/IBlockInputStream.cpp @@ -201,7 +201,7 @@ void IBlockInputStream::updateExtremes(Block & block) bool IBlockInputStream::checkTimeLimit() const { - return limits.speed_limits.checkTimeLimit(info.total_stopwatch.elapsed(), limits.timeout_overflow_mode); + return limits.speed_limits.checkTimeLimit(info.total_stopwatch, limits.timeout_overflow_mode); } diff --git a/src/Processors/Sources/SourceWithProgress.cpp b/src/Processors/Sources/SourceWithProgress.cpp index 66168e7d73a..647ad0f205f 100644 --- a/src/Processors/Sources/SourceWithProgress.cpp +++ b/src/Processors/Sources/SourceWithProgress.cpp @@ -49,7 +49,7 @@ void SourceWithProgress::setProcessListElement(QueryStatus * elem) void SourceWithProgress::work() { - if (!limits.speed_limits.checkTimeLimit(total_stopwatch.elapsed(), limits.timeout_overflow_mode)) + if (!limits.speed_limits.checkTimeLimit(total_stopwatch, limits.timeout_overflow_mode)) { cancel(); } diff --git a/src/Processors/Transforms/LimitsCheckingTransform.cpp b/src/Processors/Transforms/LimitsCheckingTransform.cpp index 9e021e8e59d..64b6b64ccd9 100644 --- a/src/Processors/Transforms/LimitsCheckingTransform.cpp +++ b/src/Processors/Transforms/LimitsCheckingTransform.cpp @@ -32,7 +32,7 @@ void LimitsCheckingTransform::transform(Chunk & chunk) info.started = true; } - if (!limits.speed_limits.checkTimeLimit(info.total_stopwatch.elapsed(), limits.timeout_overflow_mode)) + if (!limits.speed_limits.checkTimeLimit(info.total_stopwatch, limits.timeout_overflow_mode)) { stopReading(); return; From 8439135960b1dba64d15dc3e95be42530847ff4d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 18:28:44 +0200 Subject: [PATCH 103/147] Try to make the test faster --- tests/queries/0_stateless/00738_lock_for_inner_table.sh | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00738_lock_for_inner_table.sh b/tests/queries/0_stateless/00738_lock_for_inner_table.sh index 6cbed42baea..37d5755fb0b 100755 --- a/tests/queries/0_stateless/00738_lock_for_inner_table.sh +++ b/tests/queries/0_stateless/00738_lock_for_inner_table.sh @@ -11,9 +11,7 @@ uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase()) echo "DROP TABLE IF EXISTS tab_00738 SYNC; DROP TABLE IF EXISTS mv SYNC; --- create table with fsync and 20 partitions for slower INSERT --- (since increasing number of records will make it significantly slower in debug build, but not in release) -CREATE TABLE tab_00738(a Int) ENGINE = MergeTree() ORDER BY a PARTITION BY a%20 SETTINGS fsync_after_insert=1; +CREATE TABLE tab_00738(a Int) ENGINE = MergeTree() ORDER BY a; -- The matview will take at least 2 seconds to be finished (10000000 * 0.0000002) CREATE MATERIALIZED VIEW mv UUID '$uuid' ENGINE = Log AS SELECT sleepEachRow(0.0000002) FROM tab_00738;" | ${CLICKHOUSE_CLIENT} -n From 74dd3a6af69aaa2daf88e7da4238bae19916b56d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 22:10:54 +0300 Subject: [PATCH 104/147] More fixes. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 5fc32e2db69..4276160f514 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -992,8 +992,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build }); } - Block cur_header = result_projection ? result_projection->getResultColumns() - : pipe.getHeader(); + Block cur_header = pipe.getHeader(); auto append_actions = [&result_projection](ActionsDAGPtr actions) { From e2cee2576d797b55d74e82a50f96b6d2d5f44727 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Fri, 6 Aug 2021 22:33:21 +0300 Subject: [PATCH 105/147] Fixing reading of /proc/meminfo --- src/Interpreters/AsynchronousMetrics.cpp | 83 ++++++++++++++---------- 1 file changed, 50 insertions(+), 33 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index d708ff4f9e0..fe34d5c5bc6 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -779,43 +779,60 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti uint64_t kb = 0; readText(kb, *meminfo); - if (kb) + + if (!kb) { - skipWhitespaceIfAny(*meminfo, true); - assertString("kB", *meminfo); + skipToNextLineOrEOF(*meminfo); + continue; + } - uint64_t bytes = kb * 1024; + skipWhitespaceIfAny(*meminfo, true); - if (name == "MemTotal:") - { - new_values["OSMemoryTotal"] = bytes; - } - else if (name == "MemFree:") - { - /// We cannot simply name this metric "Free", because it confuses users. - /// See https://www.linuxatemyram.com/ - /// For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable. + /** + * Not all entries in /proc/meminfo contain the kB suffix, e.g. + * HugePages_Total: 0 + * HugePages_Free: 0 + * We simply skip such entries as they're not needed + */ + if (*meminfo->position() == '\n') + { + skipToNextLineOrEOF(*meminfo); + continue; + } - free_plus_cached_bytes += bytes; - new_values["OSMemoryFreeWithoutCached"] = bytes; - } - else if (name == "MemAvailable:") - { - new_values["OSMemoryAvailable"] = bytes; - } - else if (name == "Buffers:") - { - new_values["OSMemoryBuffers"] = bytes; - } - else if (name == "Cached:") - { - free_plus_cached_bytes += bytes; - new_values["OSMemoryCached"] = bytes; - } - else if (name == "SwapCached:") - { - new_values["OSMemorySwapCached"] = bytes; - } + assertString("kB", *meminfo); + + uint64_t bytes = kb * 1024; + + if (name == "MemTotal:") + { + new_values["OSMemoryTotal"] = bytes; + } + else if (name == "MemFree:") + { + /// We cannot simply name this metric "Free", because it confuses users. + /// See https://www.linuxatemyram.com/ + /// For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable. + + free_plus_cached_bytes += bytes; + new_values["OSMemoryFreeWithoutCached"] = bytes; + } + else if (name == "MemAvailable:") + { + new_values["OSMemoryAvailable"] = bytes; + } + else if (name == "Buffers:") + { + new_values["OSMemoryBuffers"] = bytes; + } + else if (name == "Cached:") + { + free_plus_cached_bytes += bytes; + new_values["OSMemoryCached"] = bytes; + } + else if (name == "SwapCached:") + { + new_values["OSMemorySwapCached"] = bytes; } skipToNextLineOrEOF(*meminfo); From 1fcbd1e196403ae43a16d5e75e1b7b740930aa5a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 23:46:29 +0300 Subject: [PATCH 106/147] Mark more tests as long --- ...ence => 00563_insert_into_remote_and_zookeeper_long.reference} | 0 ...keeper.sql => 00563_insert_into_remote_and_zookeeper_long.sql} | 0 ...32_quorum_insert_simple_test_1_parts_zookeeper_long.reference} | 0 ...=> 00732_quorum_insert_simple_test_1_parts_zookeeper_long.sql} | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00563_insert_into_remote_and_zookeeper.reference => 00563_insert_into_remote_and_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00563_insert_into_remote_and_zookeeper.sql => 00563_insert_into_remote_and_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_simple_test_1_parts_zookeeper.reference => 00732_quorum_insert_simple_test_1_parts_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_simple_test_1_parts_zookeeper.sql => 00732_quorum_insert_simple_test_1_parts_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.reference b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.reference rename to tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql rename to tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper_long.sql From 5e2bfd5ba1c3331401b5f75a8bbdb25a948c9cb4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 6 Aug 2021 21:03:38 +0300 Subject: [PATCH 107/147] fix partition id validation --- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++--- src/Storages/MergeTree/MergeTreePartInfo.cpp | 32 ++++++------------- src/Storages/MergeTree/MergeTreePartInfo.h | 2 +- .../01925_broken_partition_id_zookeeper.sql | 10 ++++++ 4 files changed, 23 insertions(+), 29 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 60ff3d094b7..4730bf9f47c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3213,8 +3213,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc if (!partition_ast.value) { - if (!MergeTreePartInfo::validatePartitionID(partition_ast.id, format_version)) - throw Exception("Invalid partition format: " + partition_ast.id, ErrorCodes::INVALID_PARTITION_VALUE); + MergeTreePartInfo::validatePartitionID(partition_ast.id, format_version); return partition_ast.id; } @@ -3225,10 +3224,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc if (partition_lit && partition_lit->value.getType() == Field::Types::String) { String partition_id = partition_lit->value.get(); - if (partition_id.size() != 6 || !std::all_of(partition_id.begin(), partition_id.end(), isNumericASCII)) - throw Exception( - "Invalid partition format: " + partition_id + ". Partition should consist of 6 digits: YYYYMM", - ErrorCodes::INVALID_PARTITION_VALUE); + MergeTreePartInfo::validatePartitionID(partition_id, format_version); return partition_id; } } diff --git a/src/Storages/MergeTree/MergeTreePartInfo.cpp b/src/Storages/MergeTree/MergeTreePartInfo.cpp index ccb26a0999e..6a98e666c34 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -9,6 +9,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_DATA_PART_NAME; + extern const int INVALID_PARTITION_VALUE; } @@ -21,38 +22,25 @@ MergeTreePartInfo MergeTreePartInfo::fromPartName(const String & part_name, Merg } -bool MergeTreePartInfo::validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version) +void MergeTreePartInfo::validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version) { if (partition_id.empty()) - return false; - - ReadBufferFromString in(partition_id); + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Partition id is empty"); if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - UInt32 min_yyyymmdd = 0; - UInt32 max_yyyymmdd = 0; - if (!tryReadIntText(min_yyyymmdd, in) - || !checkChar('_', in) - || !tryReadIntText(max_yyyymmdd, in) - || !checkChar('_', in)) - { - return false; - } + if (partition_id.size() != 6 || !std::all_of(partition_id.begin(), partition_id.end(), isNumericASCII)) + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, + "Invalid partition format: {}. Partition should consist of 6 digits: YYYYMM", + partition_id); } else { - while (!in.eof()) - { - char c; - readChar(c, in); - - if (c == '_') - break; - } + auto is_valid_char = [](char c) { return c == '-' || isAlphaNumericASCII(c); }; + if (!std::all_of(partition_id.begin(), partition_id.end(), is_valid_char)) + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Invalid partition format: {}", partition_id); } - return in.eof(); } bool MergeTreePartInfo::tryParsePartName(const String & part_name, MergeTreePartInfo * part_info, MergeTreeDataFormatVersion format_version) diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index 87f96ed5038..be856c1f157 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -88,7 +88,7 @@ struct MergeTreePartInfo } /// Simple sanity check for partition ID. Checking that it's not too long or too short, doesn't contain a lot of '_'. - static bool validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version); + static void validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version); static MergeTreePartInfo fromPartName(const String & part_name, MergeTreeDataFormatVersion format_version); // -V1071 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 baf6c1fbf8f..07e490d0ce0 100644 --- a/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql +++ b/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql @@ -14,3 +14,13 @@ ALTER TABLE broken_partition DROP PARTITION ID '20210325_0_13241_6_12747'; --{se ALTER TABLE broken_partition DROP PARTITION ID '20210325_0_13241_6_12747'; --{serverError 248} DROP TABLE IF EXISTS broken_partition; + +DROP TABLE IF EXISTS old_partition_key; + +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} + +ALTER TABLE old_partition_key DROP PARTITION ID '202103'; + +DROP TABLE old_partition_key; From 9a45458faf1b2d567f408739afe9b46ddffa35be Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 6 Aug 2021 23:55:04 +0000 Subject: [PATCH 108/147] Add bzip2 file compression --- .gitmodules | 3 + CMakeLists.txt | 1 + cmake/find/bzip2.cmake | 19 +++ contrib/CMakeLists.txt | 4 + contrib/bzip2-cmake/CMakeLists.txt | 23 +++ src/CMakeLists.txt | 5 + src/Common/ErrorCodes.cpp | 2 + src/Common/config.h.in | 1 + src/IO/Bzip2ReadBuffer.cpp | 97 ++++++++++++ src/IO/Bzip2ReadBuffer.h | 33 +++++ src/IO/Bzip2WriteBuffer.cpp | 138 ++++++++++++++++++ src/IO/Bzip2WriteBuffer.h | 37 +++++ src/IO/CompressionMethod.cpp | 18 ++- src/IO/CompressionMethod.h | 3 +- src/IO/ya.make | 2 + ...StorageSystemBuildOptions.generated.cpp.in | 1 + .../0_stateless/02003_compress_bz2.reference | 1 + .../queries/0_stateless/02003_compress_bz2.sh | 13 ++ 18 files changed, 397 insertions(+), 4 deletions(-) create mode 100644 cmake/find/bzip2.cmake create mode 100644 contrib/bzip2-cmake/CMakeLists.txt create mode 100644 src/IO/Bzip2ReadBuffer.cpp create mode 100644 src/IO/Bzip2ReadBuffer.h create mode 100644 src/IO/Bzip2WriteBuffer.cpp create mode 100644 src/IO/Bzip2WriteBuffer.h create mode 100644 tests/queries/0_stateless/02003_compress_bz2.reference create mode 100755 tests/queries/0_stateless/02003_compress_bz2.sh diff --git a/.gitmodules b/.gitmodules index 43c878427ec..6bcb2f18c61 100644 --- a/.gitmodules +++ b/.gitmodules @@ -243,3 +243,6 @@ [submodule "contrib/s2geometry"] path = contrib/s2geometry url = https://github.com/ClickHouse-Extras/s2geometry.git +[submodule "contrib/bzip2"] + path = contrib/bzip2 + url = https://gitlab.com/federicomenaquintero/bzip2.git diff --git a/CMakeLists.txt b/CMakeLists.txt index 24022c256ec..d3cb5f70c83 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -543,6 +543,7 @@ include (cmake/find/nuraft.cmake) include (cmake/find/yaml-cpp.cmake) include (cmake/find/s2geometry.cmake) include (cmake/find/nlp.cmake) +include (cmake/find/bzip2.cmake) if(NOT USE_INTERNAL_PARQUET_LIBRARY) set (ENABLE_ORC OFF CACHE INTERNAL "") diff --git a/cmake/find/bzip2.cmake b/cmake/find/bzip2.cmake new file mode 100644 index 00000000000..15532a67c00 --- /dev/null +++ b/cmake/find/bzip2.cmake @@ -0,0 +1,19 @@ +option(ENABLE_BZIP2 "Enable bzip2 compression support" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_BZIP2) + message (STATUS "bzip2 compression disabled") + return() +endif() + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/bzip2/bzlib.h") + message (WARNING "submodule contrib/bzip2 is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal bzip2 library") + set (USE_NLP 0) + return() +endif () + +set (USE_BZIP2 1) +set (BZIP2_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/bzip2") +set (BZIP2_LIBRARY bzip2) + +message (STATUS "Using bzip2=${USE_BZIP2}: ${BZIP2_INCLUDE_DIR} : ${BZIP2_LIBRARY}") diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 82cddb0ace0..e6e098a05b3 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -334,6 +334,10 @@ if (USE_NLP) add_subdirectory(lemmagen-c-cmake) endif() +if (USE_BZIP2) + add_subdirectory(bzip2-cmake) +endif() + if (USE_SQLITE) add_subdirectory(sqlite-cmake) endif() diff --git a/contrib/bzip2-cmake/CMakeLists.txt b/contrib/bzip2-cmake/CMakeLists.txt new file mode 100644 index 00000000000..a9d2efa43c1 --- /dev/null +++ b/contrib/bzip2-cmake/CMakeLists.txt @@ -0,0 +1,23 @@ +set(BZIP2_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/bzip2") +set(BZIP2_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/bzip2") + +set(SRCS + "${BZIP2_SOURCE_DIR}/blocksort.c" + "${BZIP2_SOURCE_DIR}/huffman.c" + "${BZIP2_SOURCE_DIR}/crctable.c" + "${BZIP2_SOURCE_DIR}/randtable.c" + "${BZIP2_SOURCE_DIR}/compress.c" + "${BZIP2_SOURCE_DIR}/decompress.c" + "${BZIP2_SOURCE_DIR}/bzlib.c" +) + +# From bzip2/CMakeLists.txt +set(BZ_VERSION "1.0.7") +configure_file ( + "${BZIP2_SOURCE_DIR}/bz_version.h.in" + "${BZIP2_BINARY_DIR}/bz_version.h" +) + +add_library(bzip2 ${SRCS}) + +target_include_directories(bzip2 PUBLIC "${BZIP2_SOURCE_DIR}" "${BZIP2_BINARY_DIR}") diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index a99201e4aaa..2853a2e1a81 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -479,6 +479,11 @@ if (USE_NLP) dbms_target_link_libraries (PUBLIC lemmagen) endif() +if (USE_BZIP2) + target_link_libraries (clickhouse_common_io PRIVATE ${BZIP2_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${BZIP2_INCLUDE_DIR}) +endif() + include ("${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake") if (ENABLE_TESTS AND USE_GTEST) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 7904d0ac61d..2bb378f13b5 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -567,6 +567,8 @@ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ M(1002, UNKNOWN_EXCEPTION) \ + M(1003, BZIP2_STREAM_DECODER_FAILED) \ + M(1004, BZIP2_STREAM_ENCODER_FAILED) \ /* See END */ diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 0665b1717ed..bf118e2507f 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -19,3 +19,4 @@ #cmakedefine01 USE_DATASKETCHES #cmakedefine01 USE_YAML_CPP #cmakedefine01 CLICKHOUSE_SPLIT_BINARY +#cmakedefine01 USE_BZIP2 diff --git a/src/IO/Bzip2ReadBuffer.cpp b/src/IO/Bzip2ReadBuffer.cpp new file mode 100644 index 00000000000..e264ce75444 --- /dev/null +++ b/src/IO/Bzip2ReadBuffer.cpp @@ -0,0 +1,97 @@ +#if !defined(ARCADIA_BUILD) +# include +#endif + +#if USE_BZIP2 +# include +# include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BZIP2_STREAM_DECODER_FAILED; +} + + +class Bzip2ReadBuffer::Bzip2StateWrapper +{ +public: + Bzip2StateWrapper() + { + memset(&stream, 0, sizeof(stream)); + + int ret = BZ2_bzDecompressInit(&stream, 0, 0); + + if (ret != BZ_OK) + throw Exception( + ErrorCodes::BZIP2_STREAM_DECODER_FAILED, + "bzip2 stream encoder init failed: error code: {}", + ret); + } + + ~Bzip2StateWrapper() + { + BZ2_bzDecompressEnd(&stream); + } + + bz_stream stream; +}; + +Bzip2ReadBuffer::Bzip2ReadBuffer(std::unique_ptr in_, size_t buf_size, char *existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment) + , in(std::move(in_)) + , bz(std::make_unique()) + , eof(false) +{ +} + +Bzip2ReadBuffer::~Bzip2ReadBuffer() = default; + +bool Bzip2ReadBuffer::nextImpl() +{ + if (eof) + return false; + + if (!bz->stream.avail_in) + { + in->nextIfAtEnd(); + bz->stream.avail_in = in->buffer().end() - in->position(); + bz->stream.next_in = in->position(); + } + + bz->stream.avail_out = internal_buffer.size(); + bz->stream.next_out = internal_buffer.begin(); + + int ret = BZ2_bzDecompress(&bz->stream); + + in->position() = in->buffer().end() - bz->stream.avail_in; + working_buffer.resize(internal_buffer.size() - bz->stream.avail_out); + + if (ret == BZ_STREAM_END) + { + if (in->eof()) + { + eof = true; + return !working_buffer.empty(); + } + else + { + throw Exception( + ErrorCodes::BZIP2_STREAM_DECODER_FAILED, + "bzip2 decoder finished, but input stream has not exceeded: error code: {}", ret); + } + } + + if (ret != BZ_OK) + throw Exception( + ErrorCodes::BZIP2_STREAM_DECODER_FAILED, + "bzip2 stream decoder failed: error code: {}", + ret); + + return true; +} +} + +#endif diff --git a/src/IO/Bzip2ReadBuffer.h b/src/IO/Bzip2ReadBuffer.h new file mode 100644 index 00000000000..dc113800683 --- /dev/null +++ b/src/IO/Bzip2ReadBuffer.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Bzip2ReadBuffer : public BufferWithOwnMemory +{ +public: + Bzip2ReadBuffer( + std::unique_ptr in_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + + ~Bzip2ReadBuffer() override; + +private: + bool nextImpl() override; + + std::unique_ptr in; + + class Bzip2StateWrapper; + std::unique_ptr bz; + + bool eof; +}; + +} + diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp new file mode 100644 index 00000000000..7bf20a964cd --- /dev/null +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -0,0 +1,138 @@ +#if !defined(ARCADIA_BUILD) +# include +#endif + +#if USE_BROTLI +# include +# include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BZIP2_STREAM_ENCODER_FAILED; +} + + +class Bzip2WriteBuffer::Bzip2StateWrapper +{ +public: + Bzip2StateWrapper(int compression_level) + { + memset(&stream, 0, sizeof(stream)); + + int ret = BZ2_bzCompressInit(&stream, compression_level, 0, 0); + + if (ret != BZ_OK) + throw Exception( + ErrorCodes::BZIP2_STREAM_ENCODER_FAILED, + "bzip2 stream encoder init failed: error code: {}", + ret); + } + + ~Bzip2StateWrapper() + { + BZ2_bzCompressEnd(&stream); + } + + bz_stream stream; +}; + +Bzip2WriteBuffer::Bzip2WriteBuffer(std::unique_ptr out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment) + , bz(std::make_unique(compression_level)) + , out(std::move(out_)) +{ +} + +Bzip2WriteBuffer::~Bzip2WriteBuffer() +{ + /// FIXME move final flush into the caller + MemoryTracker::LockExceptionInThread lock(VariableContext::Global); + finish(); +} + +void Bzip2WriteBuffer::nextImpl() +{ + if (!offset()) + { + return; + } + + bz->stream.next_in = working_buffer.begin(); + bz->stream.avail_in = offset(); + + try + { + do + { + out->nextIfAtEnd(); + bz->stream.next_out = out->position(); + bz->stream.avail_out = out->buffer().end() - out->position(); + + int ret = BZ2_bzCompress(&bz->stream, BZ_RUN); + + out->position() = out->buffer().end() - bz->stream.avail_out; + + if (ret != BZ_RUN_OK) + throw Exception( + ErrorCodes::BZIP2_STREAM_ENCODER_FAILED, + "bzip2 stream encoder failed: error code: {}", + ret); + + } + while (bz->stream.avail_in > 0); + } + catch (...) + { + /// Do not try to write next time after exception. + out->position() = out->buffer().begin(); + throw; + } +} + +void Bzip2WriteBuffer::finish() +{ + if (finished) + return; + + try + { + finishImpl(); + out->finalize(); + finished = true; + } + catch (...) + { + /// Do not try to flush next time after exception. + out->position() = out->buffer().begin(); + finished = true; + throw; + } +} + +void Bzip2WriteBuffer::finishImpl() +{ + next(); + + out->nextIfAtEnd(); + bz->stream.next_out = out->position(); + bz->stream.avail_out = out->buffer().end() - out->position(); + + int ret = BZ2_bzCompress(&bz->stream, BZ_FINISH); + + out->position() = out->buffer().end() - bz->stream.avail_out; + + if (ret != BZ_STREAM_END && ret != BZ_FINISH_OK) + throw Exception( + ErrorCodes::BZIP2_STREAM_ENCODER_FAILED, + "bzip2 stream encoder failed: error code: {}", + ret); +} + +} + +#endif diff --git a/src/IO/Bzip2WriteBuffer.h b/src/IO/Bzip2WriteBuffer.h new file mode 100644 index 00000000000..72f14a344e3 --- /dev/null +++ b/src/IO/Bzip2WriteBuffer.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class Bzip2WriteBuffer : public BufferWithOwnMemory +{ +public: + Bzip2WriteBuffer( + std::unique_ptr out_, + int compression_level, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + + ~Bzip2WriteBuffer() override; + + void finalize() override { finish(); } + +private: + void nextImpl() override; + + void finish(); + void finishImpl(); + + class Bzip2StateWrapper; + std::unique_ptr bz; + + std::unique_ptr out; + + bool finished = false; +}; + +} diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index c6bb5232c0c..e02475c1d83 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -10,6 +10,8 @@ #include #include #include +#include +#include #if !defined(ARCADIA_BUILD) # include @@ -40,6 +42,8 @@ std::string toContentEncodingName(CompressionMethod method) return "xz"; case CompressionMethod::Zstd: return "zstd"; + case CompressionMethod::Bzip2: + return "bz2"; case CompressionMethod::None: return ""; } @@ -69,11 +73,13 @@ CompressionMethod chooseCompressionMethod(const std::string & path, const std::s return CompressionMethod::Xz; if (method_str == "zstd" || method_str == "zst") return CompressionMethod::Zstd; + if (method_str == "bz2") + return CompressionMethod::Bzip2; if (hint.empty() || hint == "auto" || hint == "none") return CompressionMethod::None; throw Exception( - "Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'deflate', 'br', 'xz', 'zstd' are supported as compression methods", + "Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'deflate', 'br', 'xz', 'zstd', 'bz2' are supported as compression methods", ErrorCodes::NOT_IMPLEMENTED); } @@ -91,7 +97,10 @@ std::unique_ptr wrapReadBufferWithCompressionMethod( return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); if (method == CompressionMethod::Zstd) return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); - +#if USE_BZIP2 + if (method == CompressionMethod::Bzip2) + return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); +#endif if (method == CompressionMethod::None) return nested; @@ -114,7 +123,10 @@ std::unique_ptr wrapWriteBufferWithCompressionMethod( if (method == CompressionMethod::Zstd) return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); - +#if USE_BZIP2 + if (method == CompressionMethod::Bzip2) + return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); +#endif if (method == CompressionMethod::None) return nested; diff --git a/src/IO/CompressionMethod.h b/src/IO/CompressionMethod.h index 6f2d87b45cf..cf034229f77 100644 --- a/src/IO/CompressionMethod.h +++ b/src/IO/CompressionMethod.h @@ -31,7 +31,8 @@ enum class CompressionMethod /// Zstd compressor /// This option corresponds to HTTP Content-Encoding: zstd Zstd, - Brotli + Brotli, + Bzip2 }; /// How the compression method is named in HTTP. diff --git a/src/IO/ya.make b/src/IO/ya.make index 9e35a062a96..7723464be6f 100644 --- a/src/IO/ya.make +++ b/src/IO/ya.make @@ -23,6 +23,8 @@ SRCS( AIOContextPool.cpp BrotliReadBuffer.cpp BrotliWriteBuffer.cpp + Bzip2ReadBuffer.cpp + Bzip2WriteBuffer.cpp CascadeWriteBuffer.cpp CompressionMethod.cpp DoubleConverter.cpp diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 8fe574da643..6bb97355151 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -50,6 +50,7 @@ const char * auto_config_build[] "USE_LDAP", "@USE_LDAP@", "TZDATA_VERSION", "@TZDATA_VERSION@", "USE_KRB5", "@USE_KRB5@", + "USE_BZIP2", "@USE_BZIP2@", nullptr, nullptr }; diff --git a/tests/queries/0_stateless/02003_compress_bz2.reference b/tests/queries/0_stateless/02003_compress_bz2.reference new file mode 100644 index 00000000000..8ab686eafeb --- /dev/null +++ b/tests/queries/0_stateless/02003_compress_bz2.reference @@ -0,0 +1 @@ +Hello, World! diff --git a/tests/queries/0_stateless/02003_compress_bz2.sh b/tests/queries/0_stateless/02003_compress_bz2.sh new file mode 100755 index 00000000000..e3b670d8da7 --- /dev/null +++ b/tests/queries/0_stateless/02003_compress_bz2.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +${CLICKHOUSE_CLIENT} --query "SELECT * FROM (SELECT 'Hello, World!' as String) INTO OUTFILE '${USER_FILES_PATH}/bz2_compression.bz2'" +bzip2 -t ${USER_FILES_PATH}/bz2_compression.bz2 +${CLICKHOUSE_CLIENT} --query "SELECT * FROM file('${USER_FILES_PATH}/bz2_compression.bz2', 'TabSeparated', 'col String')" + +rm -f "${USER_FILES_PATH}/bz2_compression.bz2" From aa33a7add163c6668949a1f1056ec529975ae8ef Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 7 Aug 2021 13:07:41 +0800 Subject: [PATCH 109/147] Add settings to check hyperscan regexp length. --- src/Core/Settings.h | 2 ++ .../FunctionsMultiStringFuzzySearch.h | 16 +++++++++- src/Functions/FunctionsMultiStringSearch.h | 16 +++++++++- src/Functions/hyperscanRegexpChecker.cpp | 29 +++++++++++++++++++ src/Functions/hyperscanRegexpChecker.h | 10 +++++++ ...02004_max_hyperscan_regex_length.reference | 6 ++++ .../02004_max_hyperscan_regex_length.sql | 26 +++++++++++++++++ 7 files changed, 103 insertions(+), 2 deletions(-) create mode 100644 src/Functions/hyperscanRegexpChecker.cpp create mode 100644 src/Functions/hyperscanRegexpChecker.h create mode 100644 tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference create mode 100644 tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 20404089210..d3493677af5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -377,6 +377,8 @@ class IColumn; M(Bool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql', 'postgresql' and 'odbc' table functions.", 0) \ \ M(Bool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \ + M(UInt64, max_hyperscan_regexp_length, 0, "Max length of regexp than can be used in hyperscan multi-match functions. Zero means unlimited.", 0) \ + M(UInt64, max_hyperscan_regexp_total_length, 0, "Max total length of all regexps than can be used in hyperscan multi-match functions. Zero means unlimited.", 0) \ M(Bool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \ M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \ \ diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 209efb0fc2f..a2d0c972abb 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -40,7 +41,13 @@ public: throw Exception( "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); - return std::make_shared(); + return std::make_shared( + context->getSettingsRef().max_hyperscan_regexp_length, context->getSettingsRef().max_hyperscan_regexp_total_length); + } + + FunctionsMultiStringFuzzySearch(size_t max_hyperscan_regexp_length_, size_t max_hyperscan_regexp_total_length_) + : max_hyperscan_regexp_length(max_hyperscan_regexp_length_), max_hyperscan_regexp_total_length(max_hyperscan_regexp_total_length_) + { } String getName() const override { return name; } @@ -113,6 +120,9 @@ public: for (const auto & el : src_arr) refs.emplace_back(el.get()); + if (Impl::is_using_hyperscan) + checkRegexp(refs, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + auto col_res = ColumnVector::create(); auto col_offsets = ColumnArray::ColumnOffsets::create(); @@ -131,6 +141,10 @@ public: else return col_res; } + +private: + size_t max_hyperscan_regexp_length; + size_t max_hyperscan_regexp_total_length; }; } diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 08b4668940e..3dd2e8bfd09 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -53,7 +54,13 @@ public: throw Exception( "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); - return std::make_shared(); + return std::make_shared( + context->getSettingsRef().max_hyperscan_regexp_length, context->getSettingsRef().max_hyperscan_regexp_total_length); + } + + FunctionsMultiStringSearch(size_t max_hyperscan_regexp_length_, size_t max_hyperscan_regexp_total_length_) + : max_hyperscan_regexp_length(max_hyperscan_regexp_length_), max_hyperscan_regexp_total_length(max_hyperscan_regexp_total_length_) + { } String getName() const override { return name; } @@ -105,6 +112,9 @@ public: for (const auto & el : src_arr) refs.emplace_back(el.get()); + if (Impl::is_using_hyperscan) + checkRegexp(refs, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + auto col_res = ColumnVector::create(); auto col_offsets = ColumnArray::ColumnOffsets::create(); @@ -122,6 +132,10 @@ public: else return col_res; } + +private: + size_t max_hyperscan_regexp_length; + size_t max_hyperscan_regexp_total_length; }; } diff --git a/src/Functions/hyperscanRegexpChecker.cpp b/src/Functions/hyperscanRegexpChecker.cpp new file mode 100644 index 00000000000..b3c46e34daa --- /dev/null +++ b/src/Functions/hyperscanRegexpChecker.cpp @@ -0,0 +1,29 @@ +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +void checkRegexp(const std::vector & refs, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length) +{ + if (max_hyperscan_regexp_length > 0 || max_hyperscan_regexp_total_length > 0) + { + size_t total_regexp_length = 0; + for (const auto & pattern : refs) + { + if (max_hyperscan_regexp_length > 0 && pattern.size > max_hyperscan_regexp_length) + throw Exception("Regexp length too large", ErrorCodes::BAD_ARGUMENTS); + total_regexp_length += pattern.size; + } + + if (max_hyperscan_regexp_total_length > 0 && total_regexp_length > max_hyperscan_regexp_total_length) + throw Exception("Total regexp lengths too large", ErrorCodes::BAD_ARGUMENTS); + } +} + +} diff --git a/src/Functions/hyperscanRegexpChecker.h b/src/Functions/hyperscanRegexpChecker.h new file mode 100644 index 00000000000..f2988120899 --- /dev/null +++ b/src/Functions/hyperscanRegexpChecker.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ + +void checkRegexp(const std::vector & refs, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length); + +} diff --git a/tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference new file mode 100644 index 00000000000..87e68c152c3 --- /dev/null +++ b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference @@ -0,0 +1,6 @@ +1 +1 +[1] +1 +1 +[1] diff --git a/tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql new file mode 100644 index 00000000000..6058ef2f430 --- /dev/null +++ b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql @@ -0,0 +1,26 @@ +set max_hyperscan_regexp_length = 1; +set max_hyperscan_regexp_total_length = 1; + +select multiMatchAny('123', ['1']); +select multiMatchAny('123', ['12']); -- { serverError 36 } +select multiMatchAny('123', ['1', '2']); -- { serverError 36 } + +select multiMatchAnyIndex('123', ['1']); +select multiMatchAnyIndex('123', ['12']); -- { serverError 36 } +select multiMatchAnyIndex('123', ['1', '2']); -- { serverError 36 } + +select multiMatchAllIndices('123', ['1']); +select multiMatchAllIndices('123', ['12']); -- { serverError 36 } +select multiMatchAllIndices('123', ['1', '2']); -- { serverError 36 } + +select multiFuzzyMatchAny('123', 0, ['1']); +select multiFuzzyMatchAny('123', 0, ['12']); -- { serverError 36 } +select multiFuzzyMatchAny('123', 0, ['1', '2']); -- { serverError 36 } + +select multiFuzzyMatchAnyIndex('123', 0, ['1']); +select multiFuzzyMatchAnyIndex('123', 0, ['12']); -- { serverError 36 } +select multiFuzzyMatchAnyIndex('123', 0, ['1', '2']); -- { serverError 36 } + +select multiFuzzyMatchAllIndices('123', 0, ['1']); +select multiFuzzyMatchAllIndices('123', 0, ['12']); -- { serverError 36 } +select multiFuzzyMatchAllIndices('123', 0, ['1', '2']); -- { serverError 36 } From c6a6bc8b1a3db25e891bf70e3b8e342465d3deca Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 7 Aug 2021 06:18:14 +0000 Subject: [PATCH 110/147] Add submodule bzip2 --- contrib/bzip2 | 1 + 1 file changed, 1 insertion(+) create mode 160000 contrib/bzip2 diff --git a/contrib/bzip2 b/contrib/bzip2 new file mode 160000 index 00000000000..bf905ea2251 --- /dev/null +++ b/contrib/bzip2 @@ -0,0 +1 @@ +Subproject commit bf905ea2251191ff9911ae7ec0cfc35d41f9f7f6 From 00301cf36af223cd989682dcf00bd99f1c4facd7 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 7 Aug 2021 06:20:59 +0000 Subject: [PATCH 111/147] Ignore test in fasttest --- docker/test/fasttest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 6419ea3659c..a0dcf07c41d 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -312,6 +312,7 @@ function run_tests 01798_uniq_theta_sketch 01799_long_uniq_theta_sketch 01890_stem # depends on libstemmer_c + 02003_compress_bz2 # depends on bzip2 collate collation _orc_ From 5ef59d48287b1642e9cea139731fa647d7a731f8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 7 Aug 2021 14:26:08 +0800 Subject: [PATCH 112/147] Add setting to log formatted query into system.query_log --- src/Core/Settings.h | 1 + src/Interpreters/QueryLog.cpp | 2 ++ src/Interpreters/QueryLog.h | 1 + src/Interpreters/executeQuery.cpp | 6 ++++++ .../0_stateless/02005_log_formatted_queries.reference | 3 +++ tests/queries/0_stateless/02005_log_formatted_queries.sql | 5 +++++ 6 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02005_log_formatted_queries.reference create mode 100644 tests/queries/0_stateless/02005_log_formatted_queries.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 20404089210..2ee7ad283b4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -169,6 +169,7 @@ class IColumn; M(Int64, os_thread_priority, 0, "If non zero - set corresponding 'nice' value for query processing threads. Can be used to adjust query priority for OS scheduler.", 0) \ \ M(Bool, log_queries, 1, "Log requests and write the log to the system table.", 0) \ + M(Bool, log_formatted_queries, 0, "Log formatted queries and write the log to the system table.", 0) \ M(LogQueriesType, log_queries_min_type, QueryLogElementType::QUERY_START, "Minimal type in query_log to log, possible values (from low to high): QUERY_START, QUERY_FINISH, EXCEPTION_BEFORE_START, EXCEPTION_WHILE_PROCESSING.", 0) \ M(Milliseconds, log_queries_min_query_duration_ms, 0, "Minimal time for the query to run, to get to the query_log/query_thread_log.", 0) \ M(UInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \ diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 3f668e5e0ab..0f7ff579f5d 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -57,6 +57,7 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"current_database", std::make_shared()}, {"query", std::make_shared()}, + {"formatted_query", std::make_shared()}, {"normalized_query_hash", std::make_shared()}, {"query_kind", std::make_shared(std::make_shared())}, {"databases", std::make_shared( @@ -151,6 +152,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(current_database.data(), current_database.size()); columns[i++]->insertData(query.data(), query.size()); + columns[i++]->insertData(formatted_query.data(), formatted_query.size()); columns[i++]->insert(normalized_query_hash); columns[i++]->insertData(query_kind.data(), query_kind.size()); diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 0aa02104306..aad3e56190b 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -51,6 +51,7 @@ struct QueryLogElement String current_database; String query; + String formatted_query; UInt64 normalized_query_hash{}; String query_kind; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 3756f1b2765..1b59f3bc7df 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -265,7 +265,11 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr // Try log query_kind if ast is valid if (ast) + { elem.query_kind = ast->getQueryKindString(); + if (settings.log_formatted_queries) + elem.formatted_query = queryToString(ast); + } // We don't calculate databases, tables and columns when the query isn't able to start @@ -641,6 +645,8 @@ static std::tuple executeQueryImpl( elem.current_database = context->getCurrentDatabase(); elem.query = query_for_logging; + if (settings.log_formatted_queries) + elem.formatted_query = queryToString(ast); elem.normalized_query_hash = normalizedQueryHash(query_for_logging); elem.client_info = client_info; diff --git a/tests/queries/0_stateless/02005_log_formatted_queries.reference b/tests/queries/0_stateless/02005_log_formatted_queries.reference new file mode 100644 index 00000000000..3ddd8b0d64f --- /dev/null +++ b/tests/queries/0_stateless/02005_log_formatted_queries.reference @@ -0,0 +1,3 @@ +02005_log_formatted_queries.sql +select \'02005_log_formatted_queries.sql\' from system.one; SELECT \'02005_log_formatted_queries.sql\' FROM system.one +select \'02005_log_formatted_queries.sql\' from system.one; SELECT \'02005_log_formatted_queries.sql\' FROM system.one diff --git a/tests/queries/0_stateless/02005_log_formatted_queries.sql b/tests/queries/0_stateless/02005_log_formatted_queries.sql new file mode 100644 index 00000000000..62f839af0f0 --- /dev/null +++ b/tests/queries/0_stateless/02005_log_formatted_queries.sql @@ -0,0 +1,5 @@ +set log_formatted_queries = 1; + +select '02005_log_formatted_queries.sql' from system.one; +system flush logs; +select query, formatted_query from system.query_log where current_database = currentDatabase() and query = 'select \'02005_log_formatted_queries.sql\' from system.one;' and event_date >= yesterday() and event_time > now() - interval 5 minute; From 0febbd4a1def319cef1f4da457eed59c1c2679f6 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 7 Aug 2021 08:42:54 +0000 Subject: [PATCH 113/147] Fix build --- src/IO/Bzip2WriteBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index 7bf20a964cd..41cb972966c 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes class Bzip2WriteBuffer::Bzip2StateWrapper { public: - Bzip2StateWrapper(int compression_level) + explicit Bzip2StateWrapper(int compression_level) { memset(&stream, 0, sizeof(stream)); From bc7eaad5caffc2feb23feef7e04ad4cb6ce1b52a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Aug 2021 12:35:59 +0300 Subject: [PATCH 114/147] Fix --- tests/integration/test_library_bridge/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_library_bridge/test.py b/tests/integration/test_library_bridge/test.py index 607afb6db5f..97b2ccfbdbe 100644 --- a/tests/integration/test_library_bridge/test.py +++ b/tests/integration/test_library_bridge/test.py @@ -100,6 +100,7 @@ def test_load_ids(ch_cluster): if instance.is_built_with_memory_sanitizer(): pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") + instance.query('DROP DICTIONARY IF EXISTS lib_dict_c') instance.query(''' CREATE DICTIONARY lib_dict_c (key UInt64, value1 UInt64, value2 UInt64, value3 UInt64) PRIMARY KEY key SOURCE(library(PATH '/etc/clickhouse-server/config.d/dictionaries_lib/dict_lib.so')) @@ -263,6 +264,7 @@ def test_bridge_dies_with_parent(ch_cluster): assert clickhouse_pid is None assert bridge_pid is None instance.start_clickhouse(20) + instance.query('DROP DICTIONARY lib_dict_c') if __name__ == '__main__': From 8f5ecb5ed6fdacd11dacb97b436382992e28c2f1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 7 Aug 2021 12:50:42 +0300 Subject: [PATCH 115/147] Mark more tests as long --- ...=> 01650_drop_part_and_deduplication_zookeeper_long.reference} | 0 ...r.sql => 01650_drop_part_and_deduplication_zookeeper_long.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01650_drop_part_and_deduplication_zookeeper.reference => 01650_drop_part_and_deduplication_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01650_drop_part_and_deduplication_zookeeper.sql => 01650_drop_part_and_deduplication_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.reference b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.reference rename to tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql rename to tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql From bf0297ccab4c3298213807a6d2ff596b8f07bef5 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 7 Aug 2021 09:51:35 +0000 Subject: [PATCH 116/147] Fix stateless test --- tests/queries/0_stateless/02003_compress_bz2.sh | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02003_compress_bz2.sh b/tests/queries/0_stateless/02003_compress_bz2.sh index e3b670d8da7..790a520ef24 100755 --- a/tests/queries/0_stateless/02003_compress_bz2.sh +++ b/tests/queries/0_stateless/02003_compress_bz2.sh @@ -5,9 +5,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +WORKING_FOLDER_02003="${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM (SELECT 'Hello, World!' as String) INTO OUTFILE '${USER_FILES_PATH}/bz2_compression.bz2'" -bzip2 -t ${USER_FILES_PATH}/bz2_compression.bz2 -${CLICKHOUSE_CLIENT} --query "SELECT * FROM file('${USER_FILES_PATH}/bz2_compression.bz2', 'TabSeparated', 'col String')" +rm -rf "${WORKING_FOLDER_02003}" +mkdir ${WORKING_FOLDER_02003} -rm -f "${USER_FILES_PATH}/bz2_compression.bz2" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM (SELECT 'Hello, World!' as String) INTO OUTFILE '${WORKING_FOLDER_02003}/bz2_compression.bz2'" +bzip2 -t ${WORKING_FOLDER_02003}/bz2_compression.bz2 +${CLICKHOUSE_CLIENT} --query "SELECT * FROM file('${WORKING_FOLDER_02003}/bz2_compression.bz2', 'TabSeparated', 'col String')" + +rm -rf "${WORKING_FOLDER_02003}" From 08beb313e88f24ff72facd09d048f8f47b4c5827 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 7 Aug 2021 15:06:01 +0300 Subject: [PATCH 117/147] Mark more tests as long --- ...ce => 01213_alter_rename_primary_key_zookeeper_long.reference} | 0 ...eper.sql => 01213_alter_rename_primary_key_zookeeper_long.sql} | 0 ...nce => 01451_replicated_detach_drop_and_quorum_long.reference} | 0 ...uorum.sql => 01451_replicated_detach_drop_and_quorum_long.sql} | 0 ...per.reference => 01761_alter_decimal_zookeeper_long.reference} | 0 ...cimal_zookeeper.sql => 01761_alter_decimal_zookeeper_long.sql} | 0 6 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01213_alter_rename_primary_key_zookeeper.reference => 01213_alter_rename_primary_key_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01213_alter_rename_primary_key_zookeeper.sql => 01213_alter_rename_primary_key_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01451_replicated_detach_drop_and_quorum.reference => 01451_replicated_detach_drop_and_quorum_long.reference} (100%) rename tests/queries/0_stateless/{01451_replicated_detach_drop_and_quorum.sql => 01451_replicated_detach_drop_and_quorum_long.sql} (100%) rename tests/queries/0_stateless/{01761_alter_decimal_zookeeper.reference => 01761_alter_decimal_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01761_alter_decimal_zookeeper.sql => 01761_alter_decimal_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.reference b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.reference rename to tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql rename to tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.reference b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.reference similarity index 100% rename from tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.reference rename to tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.reference diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql similarity index 100% rename from tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql rename to tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql diff --git a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference b/tests/queries/0_stateless/01761_alter_decimal_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference rename to tests/queries/0_stateless/01761_alter_decimal_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql b/tests/queries/0_stateless/01761_alter_decimal_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql rename to tests/queries/0_stateless/01761_alter_decimal_zookeeper_long.sql From 07e1047b4f1c979aee35f47355164c554bb7358c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 7 Aug 2021 16:30:12 +0300 Subject: [PATCH 118/147] Mark more tests as long --- ...ce => 01493_alter_remove_no_property_zookeeper_long.reference} | 0 ...eper.sql => 01493_alter_remove_no_property_zookeeper_long.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01493_alter_remove_no_property_zookeeper.reference => 01493_alter_remove_no_property_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01493_alter_remove_no_property_zookeeper.sql => 01493_alter_remove_no_property_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference rename to tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql rename to tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper_long.sql From ee53d9448c28f8e31ebd4a258720c86086e0dfef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 7 Aug 2021 16:58:14 +0300 Subject: [PATCH 119/147] Fix documentation #27175 --- docs/ru/sql-reference/functions/other-functions.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 0e23c2f743f..921e1a9e73b 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2090,9 +2090,9 @@ SELECT tcpPort(); ## currentProfiles {#current-profiles} -Возвращает список [профилей настроек](../../operations/access-rights.md#settings-profiles-management) для текущего пользователя. +Возвращает список [профилей настроек](../../operations/access-rights.md#settings-profiles-management) для текущего пользователя. -Для изменения текущего профиля настроек может быть использована команда [SET PROFILE](../../sql-reference/statements/set.md#set-statement#query-set). Если команда `SET PROFILE` не применялась, функция возвращает профили, указанные при определении текущего пользователя (см. [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)). +Для изменения текущего профиля настроек может быть использована команда SET PROFILE. Если команда `SET PROFILE` не применялась, функция возвращает профили, указанные при определении текущего пользователя (см. [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement)). **Синтаксис** @@ -2102,7 +2102,7 @@ currentProfiles() **Возвращаемое значение** -- Список профилей настроек для текущего пользователя. +- Список профилей настроек для текущего пользователя. Тип: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). @@ -2118,7 +2118,7 @@ enabledProfiles() **Возвращаемое значение** -- Список доступных профилей для текущего пользователя. +- Список доступных профилей для текущего пользователя. Тип: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). @@ -2134,6 +2134,6 @@ defaultProfiles() **Возвращаемое значение** -- Список профилей по умолчанию. +- Список профилей по умолчанию. -Тип: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). \ No newline at end of file +Тип: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). From 86ceaaadb9a40bb00eb74e93a2cd05a7664db7e6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 7 Aug 2021 17:40:09 +0300 Subject: [PATCH 120/147] Mark more tests as long --- ...ce => 01747_alter_partition_key_enum_zookeeper_long.reference} | 0 ...eper.sql => 01747_alter_partition_key_enum_zookeeper_long.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01747_alter_partition_key_enum_zookeeper.reference => 01747_alter_partition_key_enum_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01747_alter_partition_key_enum_zookeeper.sql => 01747_alter_partition_key_enum_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.reference b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.reference rename to tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql rename to tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper_long.sql From b5a6e18511f7df9c86151a14d7827e4ab630f46c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 7 Aug 2021 15:15:49 +0000 Subject: [PATCH 121/147] Small fixes --- .gitmodules | 2 +- src/Common/ErrorCodes.cpp | 4 ++-- tests/queries/0_stateless/02003_compress_bz2.sh | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.gitmodules b/.gitmodules index 6bcb2f18c61..37b22527eb4 100644 --- a/.gitmodules +++ b/.gitmodules @@ -245,4 +245,4 @@ url = https://github.com/ClickHouse-Extras/s2geometry.git [submodule "contrib/bzip2"] path = contrib/bzip2 - url = https://gitlab.com/federicomenaquintero/bzip2.git + url = https://github.com/ClickHouse-Extras/bzip2.git diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 2bb378f13b5..04f10fb536a 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -561,14 +561,14 @@ M(591, SQLITE_ENGINE_ERROR) \ M(592, DATA_ENCRYPTION_ERROR) \ M(593, ZERO_COPY_REPLICATION_ERROR) \ + M(594, BZIP2_STREAM_DECODER_FAILED) \ + M(595, BZIP2_STREAM_ENCODER_FAILED) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ M(1002, UNKNOWN_EXCEPTION) \ - M(1003, BZIP2_STREAM_DECODER_FAILED) \ - M(1004, BZIP2_STREAM_ENCODER_FAILED) \ /* See END */ diff --git a/tests/queries/0_stateless/02003_compress_bz2.sh b/tests/queries/0_stateless/02003_compress_bz2.sh index 790a520ef24..b65ce2f233b 100755 --- a/tests/queries/0_stateless/02003_compress_bz2.sh +++ b/tests/queries/0_stateless/02003_compress_bz2.sh @@ -8,7 +8,7 @@ USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonex WORKING_FOLDER_02003="${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}" rm -rf "${WORKING_FOLDER_02003}" -mkdir ${WORKING_FOLDER_02003} +mkdir "${WORKING_FOLDER_02003}" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM (SELECT 'Hello, World!' as String) INTO OUTFILE '${WORKING_FOLDER_02003}/bz2_compression.bz2'" bzip2 -t ${WORKING_FOLDER_02003}/bz2_compression.bz2 From 5523e5e1533a1d69689eba4ab8308af2e8264779 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 7 Aug 2021 18:48:52 +0300 Subject: [PATCH 122/147] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d3493677af5..b1e9cc66a56 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -378,7 +378,7 @@ class IColumn; \ M(Bool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \ M(UInt64, max_hyperscan_regexp_length, 0, "Max length of regexp than can be used in hyperscan multi-match functions. Zero means unlimited.", 0) \ - M(UInt64, max_hyperscan_regexp_total_length, 0, "Max total length of all regexps than can be used in hyperscan multi-match functions. Zero means unlimited.", 0) \ + M(UInt64, max_hyperscan_regexp_total_length, 0, "Max total length of all regexps than can be used in hyperscan multi-match functions (per every function). Zero means unlimited.", 0) \ M(Bool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \ M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \ \ From 161aa4020693410d4b799575a86317073a06cb7f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 7 Aug 2021 19:30:59 +0300 Subject: [PATCH 123/147] Fix build; add comments --- contrib/croaring-cmake/CMakeLists.txt | 22 +++++++++++++++------- src/Common/memory.cpp | 25 +++++++++++++++++++++++++ src/Common/memory.h | 23 ----------------------- 3 files changed, 40 insertions(+), 30 deletions(-) create mode 100644 src/Common/memory.cpp diff --git a/contrib/croaring-cmake/CMakeLists.txt b/contrib/croaring-cmake/CMakeLists.txt index 1c61eb39cc8..f0cb378864b 100644 --- a/contrib/croaring-cmake/CMakeLists.txt +++ b/contrib/croaring-cmake/CMakeLists.txt @@ -25,10 +25,18 @@ target_include_directories(roaring PRIVATE "${LIBRARY_DIR}/include/roaring") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/include") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/cpp") -target_compile_definitions(roaring PRIVATE - -Dmalloc=clickhouse_malloc - -Dcalloc=clickhouse_calloc - -Drealloc=clickhouse_realloc - -Dreallocarray=clickhouse_reallocarray - -Dfree=clickhouse_free - -Dposix_memalign=clickhouse_posix_memalign) +# We redirect malloc/free family of functions to different functions that will track memory in ClickHouse. +# It will make this library depend on linking to 'clickhouse_common_io' library that is not done explicitly via 'target_link_libraries'. +# And we check that all libraries dependencies are satisfied and all symbols are resolved if we do build with shared libraries. +# That's why we enable it only in static build. +# Also note that we exploit implicit function declarations. + +if (USE_STATIC_LIBRARIES) + target_compile_definitions(roaring PRIVATE + -Dmalloc=clickhouse_malloc + -Dcalloc=clickhouse_calloc + -Drealloc=clickhouse_realloc + -Dreallocarray=clickhouse_reallocarray + -Dfree=clickhouse_free + -Dposix_memalign=clickhouse_posix_memalign) +endif () diff --git a/src/Common/memory.cpp b/src/Common/memory.cpp new file mode 100644 index 00000000000..a79d3572071 --- /dev/null +++ b/src/Common/memory.cpp @@ -0,0 +1,25 @@ +#if defined(OS_DARWIN) && defined(BUNDLED_STATIC_JEMALLOC) + +extern "C" +{ + extern void zone_register(); +} + +struct InitializeJemallocZoneAllocatorForOSX +{ + InitializeJemallocZoneAllocatorForOSX() + { + /// In case of OSX jemalloc register itself as a default zone allocator. + /// + /// But when you link statically then zone_register() will not be called, + /// and even will be optimized out: + /// + /// It is ok to call it twice (i.e. in case of shared libraries) + /// Since zone_register() is a no-op if the default zone is already replaced with something. + /// + /// https://github.com/jemalloc/jemalloc/issues/708 + zone_register(); + } +} initializeJemallocZoneAllocatorForOSX; + +#endif diff --git a/src/Common/memory.h b/src/Common/memory.h index 0aa99cdf70d..84c6af9a1be 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -62,29 +62,6 @@ inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unuse # include #endif -#if defined(OS_DARWIN) && defined(BUNDLED_STATIC_JEMALLOC) -extern "C" -{ -extern void zone_register(); -} - -struct InitializeJemallocZoneAllocatorForOSX -{ - InitializeJemallocZoneAllocatorForOSX() - { - /// In case of OSX jemalloc register itself as a default zone allocator. - /// - /// But when you link statically then zone_register() will not be called, - /// and even will be optimized out: - /// - /// It is ok to call it twice (i.e. in case of shared libraries) - /// Since zone_register() is a no-op if the default zone is already replaced with something. - /// - /// https://github.com/jemalloc/jemalloc/issues/708 - zone_register(); - } -} initializeJemallocZoneAllocatorForOSX; -#endif inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size) { From ac3abb8fd28a54bc871f53562ee0e6d9cd724442 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 7 Aug 2021 19:46:19 +0300 Subject: [PATCH 124/147] Update test.py --- tests/integration/test_merge_tree_s3_failover/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index d19236f4425..56d9441aba6 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -68,7 +68,10 @@ def drop_table(cluster): # S3 request will be failed for an appropriate part file write. FILES_PER_PART_BASE = 5 # partition.dat, default_compression_codec.txt, count.txt, columns.txt, checksums.txt FILES_PER_PART_WIDE = FILES_PER_PART_BASE + 1 + 1 + 3 * 2 # Primary index, MinMax, Mark and data file for column(s) + +# In debug build there are additional requests (from MergeTreeDataPartWriterWide.cpp:554 due to additional validation). FILES_PER_PART_WIDE_DEBUG = 2 # Additional requests to S3 in debug build + FILES_PER_PART_COMPACT = FILES_PER_PART_BASE + 1 + 1 + 2 FILES_PER_PART_COMPACT_DEBUG = 0 From c62de36e4c790d82d59e5531c15a4f789bc4505c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 00:55:40 +0300 Subject: [PATCH 125/147] Mark more tests as long --- ... => 01650_fetch_patition_with_macro_in_zk_path_long.reference} | 0 ...th.sql => 01650_fetch_patition_with_macro_in_zk_path_long.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01650_fetch_patition_with_macro_in_zk_path.reference => 01650_fetch_patition_with_macro_in_zk_path_long.reference} (100%) rename tests/queries/0_stateless/{01650_fetch_patition_with_macro_in_zk_path.sql => 01650_fetch_patition_with_macro_in_zk_path_long.sql} (100%) diff --git a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.reference b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.reference similarity index 100% rename from tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.reference rename to tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.reference diff --git a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql similarity index 100% rename from tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql rename to tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql From 8ac1f3b73d3be60e7ee620ba846339474b86ad19 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 8 Aug 2021 04:24:23 +0300 Subject: [PATCH 126/147] Update projection.md --- docs/en/sql-reference/statements/alter/projection.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 9d762a07234..07a13fc23c4 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -11,7 +11,7 @@ The following operations are available: - `ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. -- `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). +- `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). - `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. @@ -20,4 +20,4 @@ The commands ADD, DROP and CLEAR are lightweight in a sense that they only chang Also, they are replicated, syncing projections metadata via ZooKeeper. !!! note "Note" - Projection manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). + Projection manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). From 3f8c1a1433552d306ee8d70942b7a046ccceb47d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 04:36:34 +0300 Subject: [PATCH 127/147] Add a test --- .../queries/0_stateless/01961_roaring_memory_tracking.reference | 0 tests/queries/0_stateless/01961_roaring_memory_tracking.sql | 2 ++ 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/01961_roaring_memory_tracking.reference create mode 100644 tests/queries/0_stateless/01961_roaring_memory_tracking.sql diff --git a/tests/queries/0_stateless/01961_roaring_memory_tracking.reference b/tests/queries/0_stateless/01961_roaring_memory_tracking.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01961_roaring_memory_tracking.sql b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql new file mode 100644 index 00000000000..57d71cdc91c --- /dev/null +++ b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql @@ -0,0 +1,2 @@ +SET max_memory_usage = '100M'; +SELECT cityHash64(rand() % 1000) as n, groupBitmapState(number) FROM numbers_mt(2000000000) GROUP BY n; -- { serverError 241 } From ba3fbbaab0de908886920decfb720492d03ce1f4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 8 Aug 2021 04:43:59 +0300 Subject: [PATCH 128/147] Update ReplicatedMergeTreeMergeStrategyPicker.cpp --- .../MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp index 2d611dc0c97..4d8261da2ee 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp @@ -146,7 +146,7 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() { if (execute_merges_on_single_replica_time_threshold > 0) { - LOG_WARNING(storage.log, "Can't find current replica in the active replicas list, or too few active replicas to use execute_merges_on_single_replica_time_threshold!"); + LOG_WARNING(storage.log, "Can't find current replica in the active replicas list, or too few active replicas to use 'execute_merges_on_single_replica_time_threshold'"); /// we can reset the settings w/o lock (it's atomic) execute_merges_on_single_replica_time_threshold = 0; } From b8a679158c5cc115a79a0a7ce87ec771e8641fb6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 09:36:36 +0300 Subject: [PATCH 129/147] Fix 01951_distributed_push_down_limit reference --- .../0_stateless/01951_distributed_push_down_limit.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference index 9e803a171c4..ca3bbc4cb33 100644 --- a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference @@ -1,7 +1,7 @@ -- { echo } explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union @@ -16,11 +16,11 @@ Expression (Projection) ReadFromRemote (Read from remote replica) explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (with OFFSET)) MergingSorted (Merge sorted streams for ORDER BY) MergeSorting (Merge sorted blocks for ORDER BY) PartialSorting (Sort each block for ORDER BY) From 3be3c503aa03ae664254661f5775ee16fb95605e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 09:58:07 +0300 Subject: [PATCH 130/147] Fix some comments --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Storages/StorageDistributed.cpp | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 9e0185b96cd..49ebd3d48b0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1378,7 +1378,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu /// Limit is no longer needed if there is prelimit. /// - /// NOTE: that LIMIT cannot be applied of OFFSET should not be applied, + /// NOTE: that LIMIT cannot be applied if OFFSET should not be applied, /// since LIMIT will apply OFFSET too. /// This is the case for various optimizations for distributed queries, /// and when LIMIT cannot be applied it will be applied on the initiator anyway. diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index ff734d02462..bad2abf3cdc 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -553,7 +553,6 @@ std::optional StorageDistributed::getOptimizedQueryP if (const ASTPtr order_by = select.orderBy()) return default_stage; - // LIMIT BY // LIMIT // OFFSET if (select.limitLength() || select.limitOffset()) From f19ff3d61c4e36622a262977337c92a7bfeee016 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 10:10:36 +0300 Subject: [PATCH 131/147] Fix 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET --- ...onAndLimit_LIMIT_BY_LIMIT_OFFSET.reference | 14 +++++++----- ...regationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql | 22 ++++++++++++++----- 2 files changed, 24 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference index 08fe4251b69..6cdb80167d1 100644 --- a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference +++ b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference @@ -1,16 +1,17 @@ -- { echo } SELECT * FROM remote('127.{1,2}', view( - SELECT * - FROM numbers(10) + SELECT number%20 number + FROM numbers(40) + WHERE (number % 2) = (shardNum() - 1) ), number) GROUP BY number ORDER BY number ASC LIMIT 1 BY number LIMIT 5, 5 SETTINGS - optimize_skip_unused_shards = 1, - optimize_distributed_group_by_sharding_key = 1, + optimize_skip_unused_shards=1, + optimize_distributed_group_by_sharding_key=1, distributed_push_down_limit=1; 5 6 @@ -19,8 +20,9 @@ SETTINGS 9 SELECT * FROM remote('127.{1,2}', view( - SELECT * - FROM numbers(10) + SELECT number%20 number + FROM numbers(40) + WHERE (number % 2) = (shardNum() - 1) ), number) GROUP BY number ORDER BY number ASC diff --git a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql index 75c06704ab9..1a446a80603 100644 --- a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql +++ b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql @@ -1,21 +1,31 @@ +-- Here we use a trick with shardNum() to generate unique data on each shard. +-- Since distributed_group_by_no_merge=2 will use WithMergeableStateAfterAggregationAndLimit, +-- which assume that the data on shards is unique +-- (LIMIT BY will be applied only on shards, not on the initiator). + +-- To distinguish echoing from the comments above we use SELECT FORMAT Null. +SELECT '' FORMAT Null; + -- { echo } SELECT * FROM remote('127.{1,2}', view( - SELECT * - FROM numbers(10) + SELECT number%20 number + FROM numbers(40) + WHERE (number % 2) = (shardNum() - 1) ), number) GROUP BY number ORDER BY number ASC LIMIT 1 BY number LIMIT 5, 5 SETTINGS - optimize_skip_unused_shards = 1, - optimize_distributed_group_by_sharding_key = 1, + optimize_skip_unused_shards=1, + optimize_distributed_group_by_sharding_key=1, distributed_push_down_limit=1; SELECT * FROM remote('127.{1,2}', view( - SELECT * - FROM numbers(10) + SELECT number%20 number + FROM numbers(40) + WHERE (number % 2) = (shardNum() - 1) ), number) GROUP BY number ORDER BY number ASC From a59ce96242c1815377da0cc8d81831f2e71a29be Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 8 Aug 2021 22:29:37 +0800 Subject: [PATCH 132/147] Skip test in fasttest --- docker/test/fasttest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 6419ea3659c..0b4d476eaee 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -279,6 +279,7 @@ function run_tests 00926_multimatch 00929_multi_match_edit_distance 01681_hyperscan_debug_assertion + 02004_max_hyperscan_regex_length 01176_mysql_client_interactive # requires mysql client 01031_mutations_interpreter_and_context From 702d9955c0e04b15203b250f95ba4eae5347df9f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 12:38:24 +0300 Subject: [PATCH 133/147] Fix distributed queries with zero shards and aggregation --- src/Storages/StorageDistributed.cpp | 16 ++++++++++++++-- ...mize_skip_unused_shards_zero_shards.reference | 4 ++++ ...9_optimize_skip_unused_shards_zero_shards.sql | 6 +++--- 3 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index bad2abf3cdc..fcd0e255e5c 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -415,17 +415,22 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( ClusterPtr cluster = getCluster(); query_info.cluster = cluster; + size_t nodes = getClusterQueriedNodes(settings, cluster); + /// Always calculate optimized cluster here, to avoid conditions during read() /// (Anyway it will be calculated in the read()) - if (getClusterQueriedNodes(settings, cluster) > 1 && settings.optimize_skip_unused_shards) + if (nodes > 1 && settings.optimize_skip_unused_shards) { ClusterPtr optimized_cluster = getOptimizedCluster(local_context, metadata_snapshot, query_info.query); if (optimized_cluster) { LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", makeFormattedListOfShards(optimized_cluster)); + cluster = optimized_cluster; query_info.optimized_cluster = cluster; + + nodes = getClusterQueriedNodes(settings, cluster); } else { @@ -460,7 +465,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( /// If there is only one node, the query can be fully processed by the /// shard, initiator will work as a proxy only. - if (getClusterQueriedNodes(settings, cluster) == 1) + if (nodes == 1) { /// In case the query was processed to /// WithMergeableStateAfterAggregation/WithMergeableStateAfterAggregationAndLimit @@ -469,6 +474,13 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( /// relevant for Distributed over Distributed return std::max(to_stage, QueryProcessingStage::Complete); } + else if (nodes == 0) + { + /// In case of 0 shards, the query should be processed fully on the initiator, + /// since we need to apply aggregations. + /// That's why we need to return FetchColumns. + return QueryProcessingStage::FetchColumns; + } auto optimized_stage = getOptimizedQueryProcessingStage(query_info, settings); if (optimized_stage) diff --git a/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.reference b/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.reference index e69de29bb2d..109c1835a6e 100644 --- a/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.reference +++ b/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.reference @@ -0,0 +1,4 @@ +-- { echo } +select * from remote('127.{1,2}', system, one, dummy) where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1; +select count() from remote('127.{1,2}', system, one, dummy) where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1; +0 diff --git a/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.sql b/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.sql index 2ddf318313f..08b9581ace2 100644 --- a/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.sql +++ b/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.sql @@ -1,3 +1,3 @@ -create table dist_01756 (dummy UInt8) ENGINE = Distributed('test_cluster_two_shards', 'system', 'one', dummy); -select ignore(1), * from dist_01756 where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1; -drop table dist_01756; +-- { echo } +select * from remote('127.{1,2}', system, one, dummy) where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1; +select count() from remote('127.{1,2}', system, one, dummy) where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1; From 83d871b701ec8391e2a7025e6636ef6ef977a3f7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 11:29:39 +0300 Subject: [PATCH 134/147] Do not miss exceptions from the ThreadPool --- src/Common/ThreadPool.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index e6ccf405e9f..8ef85d82a1d 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -192,6 +192,9 @@ template ThreadPoolImpl::~ThreadPoolImpl() { finalize(); + /// wait() hadn't been called, log exception at least. + if (first_exception) + DB::tryLogException(first_exception, __PRETTY_FUNCTION__); } template @@ -270,11 +273,21 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ } catch (...) { + ALLOW_ALLOCATIONS_IN_SCOPE; + /// job should be reset before decrementing scheduled_jobs to /// ensure that the Job destroyed before wait() returns. job = {}; { + /// In case thread pool will not be terminated on exception + /// (this is the case for GlobalThreadPool), + /// than first_exception may be overwritten and got lost, + /// and this usually is an error, since this will finish the thread, + /// and for this the caller may not be ready. + if (!shutdown_on_exception) + DB::tryLogException(std::current_exception(), __PRETTY_FUNCTION__); + std::unique_lock lock(mutex); if (!first_exception) first_exception = std::current_exception(); // NOLINT From 0677a6c6cb463aec8ece376e8d53aa02d1442466 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 2 Aug 2021 21:07:07 +0300 Subject: [PATCH 135/147] Enable distributed_push_down_limit by default --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2ee7ad283b4..74efcea41db 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -123,7 +123,7 @@ class IColumn; \ M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard, if 1 SELECT is executed on each shard, if 2 SELECT and INSERT is executed on each shard", 0) \ M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed queries (shards will process query up to the Complete stage, initiator just proxies the data from the shards). If 2 the initiator will apply ORDER BY and LIMIT stages (it is not in case when shard process query up to the Complete stage)", 0) \ - M(UInt64, distributed_push_down_limit, 0, "If 1, LIMIT will be applied on each shard separatelly. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT.", 0) \ + M(UInt64, distributed_push_down_limit, 1, "If 1, LIMIT will be applied on each shard separatelly. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT.", 0) \ M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).", 0) \ M(UInt64, optimize_skip_unused_shards_limit, 1000, "Limit for number of sharding key values, turns off optimize_skip_unused_shards if the limit is reached", 0) \ M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ From e45a09e2f542b9aa7ec512493febe805e4b36449 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 21:09:39 +0300 Subject: [PATCH 136/147] Fix wrong PR #27184 --- src/Core/Settings.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2ee7ad283b4..0a3c50d8612 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -499,6 +499,7 @@ class IColumn; M(Bool, enable_debug_queries, false, "Obsolete setting, does nothing.", 0) \ M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing.", 0) \ M(Bool, allow_experimental_bigint_types, true, "Obsolete setting, does nothing.", 0) \ + M(Bool, allow_experimental_window_functions, true, "Obsolete setting, does nothing.", 0) \ M(HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT, "Obsolete setting, does nothing.", 0) \ M(Bool, database_replicated_ddl_output, true, "Obsolete setting, does nothing.", 0) \ /** The section above is for obsolete settings. Do not add anything there. */ From 7631242a3bf66a33a73d923cc12c596619dee78d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 22:14:01 +0300 Subject: [PATCH 137/147] Add UK Property Paid data --- .../getting-started/example-datasets/index.md | 1 + .../example-datasets/uk-price-paid.md | 325 ++++++++++++++++++ 2 files changed, 326 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/uk-price-paid.md diff --git a/docs/en/getting-started/example-datasets/index.md b/docs/en/getting-started/example-datasets/index.md index 53007c33306..669bdf31a61 100644 --- a/docs/en/getting-started/example-datasets/index.md +++ b/docs/en/getting-started/example-datasets/index.md @@ -15,6 +15,7 @@ The list of documented datasets: - [Recipes](../../getting-started/example-datasets/recipes.md) - [OnTime](../../getting-started/example-datasets/ontime.md) - [New York Taxi Data](../../getting-started/example-datasets/nyc-taxi.md) +- [UK Property Price Paid](../../getting-started/example-datasets/uk-price-paid.md) - [Star Schema Benchmark](../../getting-started/example-datasets/star-schema.md) - [WikiStat](../../getting-started/example-datasets/wikistat.md) - [Terabyte of Click Logs from Criteo](../../getting-started/example-datasets/criteo.md) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md new file mode 100644 index 00000000000..cb4baa55553 --- /dev/null +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -0,0 +1,325 @@ +--- +toc_priority: 20 +toc_title: UK Property Price Paid +--- + +# UK Property Price Paid + +The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. +The size of the dataset in uncompressed form is about 4 GiB and it will take about 272 MiB in ClickHouse. + +Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads +Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data + +Contains HM Land Registry data © Crown copyright and database right 2021. This data is licensed under the Open Government Licence v3.0. + +## Download the Dataset + +``` +wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv +``` + +Download will took about 2 minutes with good internet connection. + +## Create the Table + +``` +CREATE TABLE uk_price_paid +( + price UInt32, + date Date, + postcode1 LowCardinality(String), + postcode2 LowCardinality(String), + type Enum8('terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4, 'other' = 0), + is_new UInt8, + duration Enum8('freehold' = 1, 'leasehold' = 2, 'unknown' = 0), + addr1 String, + addr2 String, + street LowCardinality(String), + locality LowCardinality(String), + town LowCardinality(String), + district LowCardinality(String), + county LowCardinality(String), + category UInt8 +) ENGINE = MergeTree ORDER BY (postcode1, postcode2, addr1, addr2); +``` + +## Preprocess and Import Data + +We will use `clickhouse-local` tool for data preprocessing and `clickhouse-client` to upload it. + +In this example, we define the structure of source data from the CSV file and specidy a query to preprocess the data with `clickhouse-local`. + +The preprocessing is: +- splitting the postcode to two different columns `postcode1` and `postcode2` that is better for storage and queries; +- coverting the `time` field to date as it only contains 00:00 time; +- ignoring the `uuid` field because we don't need it for analysis; +- transforming `type` and `duration` to more readable Enum fields with function `transform`; +- transforming `is_new` and `category` fields from single-character string (`Y`/`N` and `A`/`B`) to UInt8 field with 0 and 1. + +Preprocessed data is piped directly to `clickhouse-client` to be inserted into ClickHouse table in streaming fashion. + +``` +clickhouse-local --input-format CSV --structure ' + uuid String, + price UInt32, + time DateTime, + postcode String, + a String, + b String, + c String, + addr1 String, + addr2 String, + street String, + locality String, + town String, + district String, + county String, + d String, + e String +' --query " + WITH splitByChar(' ', postcode) AS p + SELECT + price, + toDate(time) AS date, + p[1] AS postcode1, + p[2] AS postcode2, + transform(a, ['T', 'S', 'D', 'F', 'O'], ['terraced', 'semi-detached', 'detached', 'flat', 'other']) AS type, + b = 'Y' AS is_new, + transform(c, ['F', 'L', 'U'], ['freehold', 'leasehold', 'unknown']) AS duration, + addr1, + addr2, + street, + locality, + town, + district, + county, + d = 'B' AS category + FROM table" --date_time_input_format best_effort < pp-complete.csv | clickhouse-client --query "INSERT INTO uk_price_paid FORMAT TSV" +``` + +It will took about 40 seconds. + +## Validate the Data + +``` +SELECT count() FROM uk_price_paid +26248711 +``` + +The size of dataset in ClickHouse is just 226 MiB: + +``` +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid' +226.40 MiB +``` + +## Run Some Queries + +### Average price per year: + +``` +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year + +┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ +│ 1995 │ 67932 │ █████▍ │ +│ 1996 │ 71505 │ █████▋ │ +│ 1997 │ 78532 │ ██████▎ │ +│ 1998 │ 85435 │ ██████▋ │ +│ 1999 │ 96036 │ ███████▋ │ +│ 2000 │ 107478 │ ████████▌ │ +│ 2001 │ 118886 │ █████████▌ │ +│ 2002 │ 137940 │ ███████████ │ +│ 2003 │ 155888 │ ████████████▍ │ +│ 2004 │ 178885 │ ██████████████▎ │ +│ 2005 │ 189350 │ ███████████████▏ │ +│ 2006 │ 203528 │ ████████████████▎ │ +│ 2007 │ 219377 │ █████████████████▌ │ +│ 2008 │ 217056 │ █████████████████▎ │ +│ 2009 │ 213419 │ █████████████████ │ +│ 2010 │ 236110 │ ██████████████████▊ │ +│ 2011 │ 232804 │ ██████████████████▌ │ +│ 2012 │ 238366 │ ███████████████████ │ +│ 2013 │ 256931 │ ████████████████████▌ │ +│ 2014 │ 279917 │ ██████████████████████▍ │ +│ 2015 │ 297264 │ ███████████████████████▋ │ +│ 2016 │ 313197 │ █████████████████████████ │ +│ 2017 │ 346070 │ ███████████████████████████▋ │ +│ 2018 │ 350117 │ ████████████████████████████ │ +│ 2019 │ 351010 │ ████████████████████████████ │ +│ 2020 │ 368974 │ █████████████████████████████▌ │ +│ 2021 │ 384351 │ ██████████████████████████████▋ │ +└──────┴────────┴────────────────────────────────────────┘ + +27 rows in set. Elapsed: 0.027 sec. Processed 26.25 million rows, 157.49 MB (955.96 million rows/s., 5.74 GB/s.) +``` + +### Average price per year in London: + +``` +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year + +┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ +│ 1995 │ 109112 │ █████▍ │ +│ 1996 │ 118667 │ █████▊ │ +│ 1997 │ 136518 │ ██████▋ │ +│ 1998 │ 152983 │ ███████▋ │ +│ 1999 │ 180633 │ █████████ │ +│ 2000 │ 215830 │ ██████████▋ │ +│ 2001 │ 232996 │ ███████████▋ │ +│ 2002 │ 263672 │ █████████████▏ │ +│ 2003 │ 278394 │ █████████████▊ │ +│ 2004 │ 304665 │ ███████████████▏ │ +│ 2005 │ 322875 │ ████████████████▏ │ +│ 2006 │ 356192 │ █████████████████▋ │ +│ 2007 │ 404055 │ ████████████████████▏ │ +│ 2008 │ 420741 │ █████████████████████ │ +│ 2009 │ 427754 │ █████████████████████▍ │ +│ 2010 │ 480306 │ ████████████████████████ │ +│ 2011 │ 496274 │ ████████████████████████▋ │ +│ 2012 │ 519441 │ █████████████████████████▊ │ +│ 2013 │ 616209 │ ██████████████████████████████▋ │ +│ 2014 │ 724144 │ ████████████████████████████████████▏ │ +│ 2015 │ 792112 │ ███████████████████████████████████████▌ │ +│ 2016 │ 843568 │ ██████████████████████████████████████████▏ │ +│ 2017 │ 982566 │ █████████████████████████████████████████████████▏ │ +│ 2018 │ 1016845 │ ██████████████████████████████████████████████████▋ │ +│ 2019 │ 1043277 │ ████████████████████████████████████████████████████▏ │ +│ 2020 │ 1003963 │ ██████████████████████████████████████████████████▏ │ +│ 2021 │ 940794 │ ███████████████████████████████████████████████ │ +└──────┴─────────┴───────────────────────────────────────────────────────┘ + +27 rows in set. Elapsed: 0.024 sec. Processed 26.25 million rows, 76.88 MB (1.08 billion rows/s., 3.15 GB/s.) +``` + +Something happened in 2013. I don't have a clue. Maybe you have a clue what happened in 2020? + +### The most expensive neighborhoods: + +``` +SELECT + town, + district, + count() AS c, + round(avg(price)) AS price, + bar(price, 0, 5000000, 100) +FROM uk_price_paid +WHERE date >= '2020-01-01' +GROUP BY + town, + district +HAVING c >= 100 +ORDER BY price DESC +LIMIT 100 + +Query id: df8c0a98-4713-4f0e-9690-5f73b52f7206 + +┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ +│ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ +│ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ +│ LONDON │ KENSINGTON AND CHELSEA │ 2367 │ 2342422 │ ██████████████████████████████████████████████▋ │ +│ LEATHERHEAD │ ELMBRIDGE │ 108 │ 1927143 │ ██████████████████████████████████████▌ │ +│ VIRGINIA WATER │ RUNNYMEDE │ 142 │ 1868819 │ █████████████████████████████████████▍ │ +│ LONDON │ CAMDEN │ 2815 │ 1736788 │ ██████████████████████████████████▋ │ +│ THORNTON HEATH │ CROYDON │ 521 │ 1733051 │ ██████████████████████████████████▋ │ +│ WINDLESHAM │ SURREY HEATH │ 103 │ 1717255 │ ██████████████████████████████████▎ │ +│ BARNET │ ENFIELD │ 115 │ 1503458 │ ██████████████████████████████ │ +│ OXFORD │ SOUTH OXFORDSHIRE │ 298 │ 1275200 │ █████████████████████████▌ │ +│ LONDON │ ISLINGTON │ 2458 │ 1274308 │ █████████████████████████▍ │ +│ COBHAM │ ELMBRIDGE │ 364 │ 1260005 │ █████████████████████████▏ │ +│ LONDON │ HOUNSLOW │ 618 │ 1215682 │ ████████████████████████▎ │ +│ ASCOT │ WINDSOR AND MAIDENHEAD │ 379 │ 1215146 │ ████████████████████████▎ │ +│ LONDON │ RICHMOND UPON THAMES │ 654 │ 1207551 │ ████████████████████████▏ │ +│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 307 │ 1186220 │ ███████████████████████▋ │ +│ RICHMOND │ RICHMOND UPON THAMES │ 805 │ 1100420 │ ██████████████████████ │ +│ LONDON │ HAMMERSMITH AND FULHAM │ 2888 │ 1062959 │ █████████████████████▎ │ +│ WEYBRIDGE │ ELMBRIDGE │ 607 │ 1027161 │ ████████████████████▌ │ +│ RADLETT │ HERTSMERE │ 265 │ 1015896 │ ████████████████████▎ │ +│ SALCOMBE │ SOUTH HAMS │ 124 │ 1014393 │ ████████████████████▎ │ +│ BURFORD │ WEST OXFORDSHIRE │ 102 │ 993100 │ ███████████████████▋ │ +│ ESHER │ ELMBRIDGE │ 454 │ 969770 │ ███████████████████▍ │ +│ HINDHEAD │ WAVERLEY │ 128 │ 967786 │ ███████████████████▎ │ +│ BROCKENHURST │ NEW FOREST │ 121 │ 967046 │ ███████████████████▎ │ +│ LEATHERHEAD │ GUILDFORD │ 191 │ 964489 │ ███████████████████▎ │ +│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 376 │ 958555 │ ███████████████████▏ │ +│ EAST MOLESEY │ ELMBRIDGE │ 181 │ 943457 │ ██████████████████▋ │ +│ OLNEY │ MILTON KEYNES │ 220 │ 942892 │ ██████████████████▋ │ +│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 135 │ 926950 │ ██████████████████▌ │ +│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 509 │ 905732 │ ██████████████████ │ +│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 889 │ 899689 │ █████████████████▊ │ +│ BELVEDERE │ BEXLEY │ 313 │ 895336 │ █████████████████▊ │ +│ CRANBROOK │ TUNBRIDGE WELLS │ 404 │ 888190 │ █████████████████▋ │ +│ LONDON │ EALING │ 2460 │ 865893 │ █████████████████▎ │ +│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 114 │ 863814 │ █████████████████▎ │ +│ LONDON │ MERTON │ 1958 │ 857192 │ █████████████████▏ │ +│ GUILDFORD │ WAVERLEY │ 131 │ 854447 │ █████████████████ │ +│ LONDON │ HACKNEY │ 3088 │ 846571 │ ████████████████▊ │ +│ LYMM │ WARRINGTON │ 285 │ 839920 │ ████████████████▋ │ +│ HARPENDEN │ ST ALBANS │ 606 │ 836994 │ ████████████████▋ │ +│ LONDON │ WANDSWORTH │ 6113 │ 832292 │ ████████████████▋ │ +│ LONDON │ SOUTHWARK │ 3612 │ 831319 │ ████████████████▋ │ +│ BERKHAMSTED │ DACORUM │ 502 │ 830356 │ ████████████████▌ │ +│ KINGS LANGLEY │ DACORUM │ 137 │ 821358 │ ████████████████▍ │ +│ TONBRIDGE │ TUNBRIDGE WELLS │ 339 │ 806736 │ ████████████████▏ │ +│ EPSOM │ REIGATE AND BANSTEAD │ 157 │ 805903 │ ████████████████ │ +│ WOKING │ GUILDFORD │ 161 │ 803283 │ ████████████████ │ +│ STOCKBRIDGE │ TEST VALLEY │ 168 │ 801973 │ ████████████████ │ +│ TEDDINGTON │ RICHMOND UPON THAMES │ 539 │ 798591 │ ███████████████▊ │ +│ OXFORD │ VALE OF WHITE HORSE │ 329 │ 792907 │ ███████████████▋ │ +│ LONDON │ BARNET │ 3624 │ 789583 │ ███████████████▋ │ +│ TWICKENHAM │ RICHMOND UPON THAMES │ 1090 │ 787760 │ ███████████████▋ │ +│ LUTON │ CENTRAL BEDFORDSHIRE │ 196 │ 786051 │ ███████████████▋ │ +│ TONBRIDGE │ MAIDSTONE │ 277 │ 785746 │ ███████████████▋ │ +│ TOWCESTER │ WEST NORTHAMPTONSHIRE │ 186 │ 783532 │ ███████████████▋ │ +│ LONDON │ LAMBETH │ 4832 │ 783422 │ ███████████████▋ │ +│ LUTTERWORTH │ HARBOROUGH │ 515 │ 781775 │ ███████████████▋ │ +│ WOODSTOCK │ WEST OXFORDSHIRE │ 135 │ 777499 │ ███████████████▌ │ +│ ALRESFORD │ WINCHESTER │ 196 │ 775577 │ ███████████████▌ │ +│ LONDON │ NEWHAM │ 2942 │ 768551 │ ███████████████▎ │ +│ ALDERLEY EDGE │ CHESHIRE EAST │ 168 │ 768280 │ ███████████████▎ │ +│ MARLOW │ BUCKINGHAMSHIRE │ 301 │ 762784 │ ███████████████▎ │ +│ BILLINGSHURST │ CHICHESTER │ 134 │ 760920 │ ███████████████▏ │ +│ LONDON │ TOWER HAMLETS │ 4183 │ 759635 │ ███████████████▏ │ +│ MIDHURST │ CHICHESTER │ 245 │ 759101 │ ███████████████▏ │ +│ THAMES DITTON │ ELMBRIDGE │ 227 │ 753347 │ ███████████████ │ +│ POTTERS BAR │ WELWYN HATFIELD │ 163 │ 752926 │ ███████████████ │ +│ REIGATE │ REIGATE AND BANSTEAD │ 555 │ 740961 │ ██████████████▋ │ +│ TADWORTH │ REIGATE AND BANSTEAD │ 477 │ 738997 │ ██████████████▋ │ +│ SEVENOAKS │ SEVENOAKS │ 1074 │ 734658 │ ██████████████▋ │ +│ PETWORTH │ CHICHESTER │ 138 │ 732432 │ ██████████████▋ │ +│ BOURNE END │ BUCKINGHAMSHIRE │ 127 │ 730742 │ ██████████████▌ │ +│ PURLEY │ CROYDON │ 540 │ 727721 │ ██████████████▌ │ +│ OXTED │ TANDRIDGE │ 320 │ 726078 │ ██████████████▌ │ +│ LONDON │ HARINGEY │ 2988 │ 724573 │ ██████████████▍ │ +│ BANSTEAD │ REIGATE AND BANSTEAD │ 373 │ 713834 │ ██████████████▎ │ +│ PINNER │ HARROW │ 480 │ 712166 │ ██████████████▏ │ +│ MALMESBURY │ WILTSHIRE │ 293 │ 707747 │ ██████████████▏ │ +│ RICKMANSWORTH │ THREE RIVERS │ 732 │ 705400 │ ██████████████ │ +│ SLOUGH │ BUCKINGHAMSHIRE │ 359 │ 705002 │ ██████████████ │ +│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 214 │ 704904 │ ██████████████ │ +│ READING │ SOUTH OXFORDSHIRE │ 295 │ 701697 │ ██████████████ │ +│ HYTHE │ FOLKESTONE AND HYTHE │ 457 │ 700334 │ ██████████████ │ +│ WELWYN │ WELWYN HATFIELD │ 217 │ 699649 │ █████████████▊ │ +│ CHIGWELL │ EPPING FOREST │ 242 │ 697869 │ █████████████▊ │ +│ BARNET │ BARNET │ 906 │ 695680 │ █████████████▊ │ +│ HASLEMERE │ CHICHESTER │ 120 │ 694028 │ █████████████▊ │ +│ LEATHERHEAD │ MOLE VALLEY │ 748 │ 692026 │ █████████████▋ │ +│ LONDON │ BRENT │ 1945 │ 690799 │ █████████████▋ │ +│ HASLEMERE │ WAVERLEY │ 258 │ 690765 │ █████████████▋ │ +│ NORTHWOOD │ HILLINGDON │ 252 │ 690753 │ █████████████▋ │ +│ WALTON-ON-THAMES │ ELMBRIDGE │ 871 │ 689431 │ █████████████▋ │ +│ INGATESTONE │ BRENTWOOD │ 150 │ 688345 │ █████████████▋ │ +│ OXFORD │ OXFORD │ 1761 │ 686114 │ █████████████▋ │ +│ CHISLEHURST │ BROMLEY │ 410 │ 682892 │ █████████████▋ │ +│ KINGS LANGLEY │ THREE RIVERS │ 109 │ 682320 │ █████████████▋ │ +│ ASHTEAD │ MOLE VALLEY │ 280 │ 680483 │ █████████████▌ │ +│ WOKING │ SURREY HEATH │ 269 │ 679035 │ █████████████▌ │ +│ ASCOT │ BRACKNELL FOREST │ 160 │ 678632 │ █████████████▌ │ +└──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ + +100 rows in set. Elapsed: 0.039 sec. Processed 26.25 million rows, 278.03 MB (674.32 million rows/s., 7.14 GB/s.) +``` + +### Test it in Playground + +The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). From b368c12400a6fd7688d6623eb3d6c592d6d148f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 22:15:18 +0300 Subject: [PATCH 138/147] Add UK Property Paid data --- docs/en/getting-started/example-datasets/uk-price-paid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index cb4baa55553..3ff77aa5700 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -6,7 +6,7 @@ toc_title: UK Property Price Paid # UK Property Price Paid The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. -The size of the dataset in uncompressed form is about 4 GiB and it will take about 272 MiB in ClickHouse. +The size of the dataset in uncompressed form is about 4 GiB and it will take about 226 MiB in ClickHouse. Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data From 0f5d3963bd8a76dedc4d47ce157c1e04c5e97b7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 22:16:11 +0300 Subject: [PATCH 139/147] Add UK Property Paid data --- docs/en/getting-started/example-datasets/uk-price-paid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 3ff77aa5700..23d2db69af6 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -48,7 +48,7 @@ CREATE TABLE uk_price_paid We will use `clickhouse-local` tool for data preprocessing and `clickhouse-client` to upload it. -In this example, we define the structure of source data from the CSV file and specidy a query to preprocess the data with `clickhouse-local`. +In this example, we define the structure of source data from the CSV file and specify a query to preprocess the data with `clickhouse-local`. The preprocessing is: - splitting the postcode to two different columns `postcode1` and `postcode2` that is better for storage and queries; From 844c0cf0ab6cfa52e6d8afa10e01aa4213616f0d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 22:16:53 +0300 Subject: [PATCH 140/147] Add UK Property Paid data --- docs/en/getting-started/example-datasets/uk-price-paid.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 23d2db69af6..f328ea51a24 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -19,7 +19,7 @@ Contains HM Land Registry data © Crown copyright and database right 2021. This wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv ``` -Download will took about 2 minutes with good internet connection. +Download will take about 2 minutes with good internet connection. ## Create the Table @@ -98,7 +98,7 @@ clickhouse-local --input-format CSV --structure ' FROM table" --date_time_input_format best_effort < pp-complete.csv | clickhouse-client --query "INSERT INTO uk_price_paid FORMAT TSV" ``` -It will took about 40 seconds. +It will take about 40 seconds. ## Validate the Data From 8a320967e636cf8924921fe39b37ceca633871d1 Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Sun, 8 Aug 2021 23:26:35 +0300 Subject: [PATCH 141/147] fix description of changing url when cloning project --- docs/ru/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 391d28d5a89..c3bf81b58c7 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -43,7 +43,7 @@ ClickHouse не работает и не собирается на 32-битны git clone --recursive git@github.com:ClickHouse/ClickHouse.git cd ClickHouse -Замените *yandex* на имя вашего аккаунта на GitHub. +Замените первое вхождение слова `ClickHouse` в команде для git на имя вашего аккаунта на GitHub. Эта команда создаст директорию ClickHouse, содержащую рабочую копию проекта. From f55e113755406a02239981fe7c0c810a25163d95 Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Sun, 8 Aug 2021 23:36:02 +0300 Subject: [PATCH 142/147] remove extra english text already translated to russian --- docs/ru/development/developer-instruction.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 391d28d5a89..097b2532132 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -92,7 +92,6 @@ ClickHouse не работает и не собирается на 32-битны # Две последние команды могут быть объединены вместе: git submodule update --init -The next commands would help you to reset all submodules to the initial state (!WARING! - any changes inside will be deleted): Следующие команды помогут сбросить все сабмодули в изначальное состояние (!ВНИМАНИЕ! - все изменения в сабмодулях будут утеряны): # Synchronizes submodules' remote URL with .gitmodules From 63f624c0b986a84d5e0e0778e0f44dc9790c247e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 8 Aug 2021 23:38:06 +0300 Subject: [PATCH 143/147] Update coordinates.md --- docs/en/sql-reference/functions/geo/coordinates.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/en/sql-reference/functions/geo/coordinates.md b/docs/en/sql-reference/functions/geo/coordinates.md index fd1d46e48ff..537a78869d6 100644 --- a/docs/en/sql-reference/functions/geo/coordinates.md +++ b/docs/en/sql-reference/functions/geo/coordinates.md @@ -41,6 +41,13 @@ SELECT greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673) └───────────────────────────────────────────────────────────────────┘ ``` +## geoDistance + +Similar to `greatCircleDistance` but calculates the distance on WGS-84 ellipsoid instead of sphere. This is more precise approximation of the Earth Geoid. +The performance is the same as for `greatCircleDistance` (no performance drawback). It is recommended to use `geoDistance` to calculate the distances on Earth. + +Technical note: for close enough points we calculate the distance using planar approximation with the metric on the tangent plane on the midpoint of the coordinates. + ## greatCircleAngle {#greatcircleangle} Calculates the central angle between two points on the Earth’s surface using [the great-circle formula](https://en.wikipedia.org/wiki/Great-circle_distance). From f18e1aa2de5e9f111291cef1f718c8d86466eeb8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 8 Aug 2021 23:38:51 +0300 Subject: [PATCH 144/147] Update coordinates.md --- docs/en/sql-reference/functions/geo/coordinates.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/geo/coordinates.md b/docs/en/sql-reference/functions/geo/coordinates.md index 537a78869d6..2d9bb41fc27 100644 --- a/docs/en/sql-reference/functions/geo/coordinates.md +++ b/docs/en/sql-reference/functions/geo/coordinates.md @@ -46,7 +46,7 @@ SELECT greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673) Similar to `greatCircleDistance` but calculates the distance on WGS-84 ellipsoid instead of sphere. This is more precise approximation of the Earth Geoid. The performance is the same as for `greatCircleDistance` (no performance drawback). It is recommended to use `geoDistance` to calculate the distances on Earth. -Technical note: for close enough points we calculate the distance using planar approximation with the metric on the tangent plane on the midpoint of the coordinates. +Technical note: for close enough points we calculate the distance using planar approximation with the metric on the tangent plane at the midpoint of the coordinates. ## greatCircleAngle {#greatcircleangle} From 8b0cfdda686d593254490be6be4ee4336398c870 Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Sun, 8 Aug 2021 23:40:45 +0300 Subject: [PATCH 145/147] translate some text from english to russian in developer-instruction.md --- docs/ru/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 391d28d5a89..f23c2589c82 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -140,7 +140,7 @@ ClickHouse использует для сборки некоторое коли Впрочем, наша среда continuous integration проверяет около десятка вариантов сборки, включая gcc, но сборка с помощью gcc непригодна для использования в продакшене. -On Ubuntu/Debian you can use the automatic installation script (check [official webpage](https://apt.llvm.org/)) +На Ubuntu и Debian вы можете использовать скрипт для автоматической установки (см. [официальный сайт](https://apt.llvm.org/)) ```bash sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" From d159c83fb84081eb4e8ca15854fd5c8bb8aab5dd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 9 Aug 2021 00:02:25 +0300 Subject: [PATCH 146/147] Add OpenSky dataset --- .../getting-started/example-datasets/index.md | 1 + .../example-datasets/opensky.md | 384 ++++++++++++++++++ 2 files changed, 385 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/opensky.md diff --git a/docs/en/getting-started/example-datasets/index.md b/docs/en/getting-started/example-datasets/index.md index 669bdf31a61..9d4aec6eca2 100644 --- a/docs/en/getting-started/example-datasets/index.md +++ b/docs/en/getting-started/example-datasets/index.md @@ -14,6 +14,7 @@ The list of documented datasets: - [Anonymized Yandex.Metrica Dataset](../../getting-started/example-datasets/metrica.md) - [Recipes](../../getting-started/example-datasets/recipes.md) - [OnTime](../../getting-started/example-datasets/ontime.md) +- [OpenSky](../../getting-started/example-datasets/opensky.md) - [New York Taxi Data](../../getting-started/example-datasets/nyc-taxi.md) - [UK Property Price Paid](../../getting-started/example-datasets/uk-price-paid.md) - [Star Schema Benchmark](../../getting-started/example-datasets/star-schema.md) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md new file mode 100644 index 00000000000..c9f6729cdd7 --- /dev/null +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -0,0 +1,384 @@ +--- +toc_priority: 20 +toc_title: OpenSky +--- + +# Crowdsourced air traffic data from The OpenSky Network 2020 + +"The data in this dataset is derived and cleaned from the full OpenSky dataset to illustrate the development of air traffic during the COVID-19 pandemic. It spans all flights seen by the network's more than 2500 members since 1 January 2019. More data will be periodically included in the dataset until the end of the COVID-19 pandemic". + +Source: https://zenodo.org/record/5092942#.YRBCyTpRXYd + +Martin Strohmeier, Xavier Olive, Jannis Lübbe, Matthias Schäfer, and Vincent Lenders +"Crowdsourced air traffic data from the OpenSky Network 2019–2020" +Earth System Science Data 13(2), 2021 +https://doi.org/10.5194/essd-13-357-2021 + +## Download the Dataset + +``` +wget -O- https://zenodo.org/record/5092942 | grep -oP 'https://zenodo.org/record/5092942/files/flightlist_\d+_\d+\.csv\.gz' | xargs wget +``` + +Download will take about 2 minutes with good internet connection. There are 30 files with total size of 4.3 GB. + +## Create the Table + +``` +CREATE TABLE opensky +( + callsign String, + number String, + icao24 String, + registration String, + typecode String, + origin String, + destination String, + firstseen DateTime, + lastseen DateTime, + day DateTime, + latitude_1 Float64, + longitude_1 Float64, + altitude_1 Float64, + latitude_2 Float64, + longitude_2 Float64, + altitude_2 Float64 +) ENGINE = MergeTree ORDER BY (origin, destination, callsign); +``` + +## Import Data + +Upload data into ClickHouse in parallel: + +``` +ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c ' + gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' +``` + +Here we pass the list of files (`ls -1 flightlist_*.csv.gz`) to `xargs` for parallel processing. +`xargs -P100` specifies to use up to 100 parallel workers but as we only have 30 files, the number of workers will be only 30. + +For every file, `xargs` will run a script with `bash -c`. The script has substitution in form of `{}` and the `xargs` command will substitute the filename to it (we have asked it for xargs with `-I{}`). + +The script will decompress the file (`gzip -c -d "{}"`) to standard output (`-c` parameter) and the output is redirected to `clickhouse-client`. + +Finally, `clickhouse-client` will do insertion. It will read input data in `CSVWithNames` format. We also asked to parse DateTime fields with extended parser (`--date_time_input_format best_effort`) to recognize ISO-8601 format with timezone offsets. + +Parallel upload takes 24 seconds. + +If you don't like parallel upload, here is sequential variant: +``` +for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done +``` + +## Validate the Data + +``` +SELECT count() FROM opensky +66010819 +``` + +The size of dataset in ClickHouse is just 2.64 GiB: + +``` +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' +2.64 GiB +``` + +## Run Some Queries + +Total distance travelled is 68 billion kilometers: + +``` +SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) / 1000) FROM opensky + +┌─formatReadableQuantity(divide(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)), 1000))─┐ +│ 68.72 billion │ +└──────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +Average flight distance is around 1000 km. +``` +SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky + +┌─avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2))─┐ +│ 1041090.6465708319 │ +└────────────────────────────────────────────────────────────────────┘ +``` + +### Most busy origin airports and the average distance seen: + +``` +SELECT + origin, + count(), + round(avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2))) AS distance, + bar(distance, 0, 10000000, 100) AS bar +FROM opensky +WHERE origin != '' +GROUP BY origin +ORDER BY count() DESC +LIMIT 100 + +Query id: f9010ea5-97d0-45a3-a5bd-9657906cd105 + + ┌─origin─┬─count()─┬─distance─┬─bar────────────────────────────────────┐ + 1. │ KORD │ 745007 │ 1546108 │ ███████████████▍ │ + 2. │ KDFW │ 696702 │ 1358721 │ █████████████▌ │ + 3. │ KATL │ 667286 │ 1169661 │ ███████████▋ │ + 4. │ KDEN │ 582709 │ 1287742 │ ████████████▊ │ + 5. │ KLAX │ 581952 │ 2628393 │ ██████████████████████████▎ │ + 6. │ KLAS │ 447789 │ 1336967 │ █████████████▎ │ + 7. │ KPHX │ 428558 │ 1345635 │ █████████████▍ │ + 8. │ KSEA │ 412592 │ 1757317 │ █████████████████▌ │ + 9. │ KCLT │ 404612 │ 880355 │ ████████▋ │ + 10. │ VIDP │ 363074 │ 1445052 │ ██████████████▍ │ + 11. │ EDDF │ 362643 │ 2263960 │ ██████████████████████▋ │ + 12. │ KSFO │ 361869 │ 2445732 │ ████████████████████████▍ │ + 13. │ KJFK │ 349232 │ 2996550 │ █████████████████████████████▊ │ + 14. │ KMSP │ 346010 │ 1287328 │ ████████████▋ │ + 15. │ LFPG │ 344748 │ 2206203 │ ██████████████████████ │ + 16. │ EGLL │ 341370 │ 3216593 │ ████████████████████████████████▏ │ + 17. │ EHAM │ 340272 │ 2116425 │ █████████████████████▏ │ + 18. │ KEWR │ 337696 │ 1826545 │ ██████████████████▎ │ + 19. │ KPHL │ 320762 │ 1291761 │ ████████████▊ │ + 20. │ OMDB │ 308855 │ 2855706 │ ████████████████████████████▌ │ + 21. │ UUEE │ 307098 │ 1555122 │ ███████████████▌ │ + 22. │ KBOS │ 304416 │ 1621675 │ ████████████████▏ │ + 23. │ LEMD │ 291787 │ 1695097 │ ████████████████▊ │ + 24. │ YSSY │ 272979 │ 1875298 │ ██████████████████▋ │ + 25. │ KMIA │ 265121 │ 1923542 │ ███████████████████▏ │ + 26. │ ZGSZ │ 263497 │ 745086 │ ███████▍ │ + 27. │ EDDM │ 256691 │ 1361453 │ █████████████▌ │ + 28. │ WMKK │ 254264 │ 1626688 │ ████████████████▎ │ + 29. │ CYYZ │ 251192 │ 2175026 │ █████████████████████▋ │ + 30. │ KLGA │ 248699 │ 1106935 │ ███████████ │ + 31. │ VHHH │ 248473 │ 3457658 │ ██████████████████████████████████▌ │ + 32. │ RJTT │ 243477 │ 1272744 │ ████████████▋ │ + 33. │ KBWI │ 241440 │ 1187060 │ ███████████▋ │ + 34. │ KIAD │ 239558 │ 1683485 │ ████████████████▋ │ + 35. │ KIAH │ 234202 │ 1538335 │ ███████████████▍ │ + 36. │ KFLL │ 223447 │ 1464410 │ ██████████████▋ │ + 37. │ KDAL │ 212055 │ 1082339 │ ██████████▋ │ + 38. │ KDCA │ 207883 │ 1013359 │ ██████████▏ │ + 39. │ LIRF │ 207047 │ 1427965 │ ██████████████▎ │ + 40. │ PANC │ 206007 │ 2525359 │ █████████████████████████▎ │ + 41. │ LTFJ │ 205415 │ 860470 │ ████████▌ │ + 42. │ KDTW │ 204020 │ 1106716 │ ███████████ │ + 43. │ VABB │ 201679 │ 1300865 │ █████████████ │ + 44. │ OTHH │ 200797 │ 3759544 │ █████████████████████████████████████▌ │ + 45. │ KMDW │ 200796 │ 1232551 │ ████████████▎ │ + 46. │ KSAN │ 198003 │ 1495195 │ ██████████████▊ │ + 47. │ KPDX │ 197760 │ 1269230 │ ████████████▋ │ + 48. │ SBGR │ 197624 │ 2041697 │ ████████████████████▍ │ + 49. │ VOBL │ 189011 │ 1040180 │ ██████████▍ │ + 50. │ LEBL │ 188956 │ 1283190 │ ████████████▋ │ + 51. │ YBBN │ 188011 │ 1253405 │ ████████████▌ │ + 52. │ LSZH │ 187934 │ 1572029 │ ███████████████▋ │ + 53. │ YMML │ 187643 │ 1870076 │ ██████████████████▋ │ + 54. │ RCTP │ 184466 │ 2773976 │ ███████████████████████████▋ │ + 55. │ KSNA │ 180045 │ 778484 │ ███████▋ │ + 56. │ EGKK │ 176420 │ 1694770 │ ████████████████▊ │ + 57. │ LOWW │ 176191 │ 1274833 │ ████████████▋ │ + 58. │ UUDD │ 176099 │ 1368226 │ █████████████▋ │ + 59. │ RKSI │ 173466 │ 3079026 │ ██████████████████████████████▋ │ + 60. │ EKCH │ 172128 │ 1229895 │ ████████████▎ │ + 61. │ KOAK │ 171119 │ 1114447 │ ███████████▏ │ + 62. │ RPLL │ 170122 │ 1440735 │ ██████████████▍ │ + 63. │ KRDU │ 167001 │ 830521 │ ████████▎ │ + 64. │ KAUS │ 164524 │ 1256198 │ ████████████▌ │ + 65. │ KBNA │ 163242 │ 1022726 │ ██████████▏ │ + 66. │ KSDF │ 162655 │ 1380867 │ █████████████▋ │ + 67. │ ENGM │ 160732 │ 910108 │ █████████ │ + 68. │ LIMC │ 160696 │ 1564620 │ ███████████████▋ │ + 69. │ KSJC │ 159278 │ 1081125 │ ██████████▋ │ + 70. │ KSTL │ 157984 │ 1026699 │ ██████████▎ │ + 71. │ UUWW │ 156811 │ 1261155 │ ████████████▌ │ + 72. │ KIND │ 153929 │ 987944 │ █████████▊ │ + 73. │ ESSA │ 153390 │ 1203439 │ ████████████ │ + 74. │ KMCO │ 153351 │ 1508657 │ ███████████████ │ + 75. │ KDVT │ 152895 │ 74048 │ ▋ │ + 76. │ VTBS │ 152645 │ 2255591 │ ██████████████████████▌ │ + 77. │ CYVR │ 149574 │ 2027413 │ ████████████████████▎ │ + 78. │ EIDW │ 148723 │ 1503985 │ ███████████████ │ + 79. │ LFPO │ 143277 │ 1152964 │ ███████████▌ │ + 80. │ EGSS │ 140830 │ 1348183 │ █████████████▍ │ + 81. │ KAPA │ 140776 │ 420441 │ ████▏ │ + 82. │ KHOU │ 138985 │ 1068806 │ ██████████▋ │ + 83. │ KTPA │ 138033 │ 1338223 │ █████████████▍ │ + 84. │ KFFZ │ 137333 │ 55397 │ ▌ │ + 85. │ NZAA │ 136092 │ 1581264 │ ███████████████▋ │ + 86. │ YPPH │ 133916 │ 1271550 │ ████████████▋ │ + 87. │ RJBB │ 133522 │ 1805623 │ ██████████████████ │ + 88. │ EDDL │ 133018 │ 1265919 │ ████████████▋ │ + 89. │ ULLI │ 130501 │ 1197108 │ ███████████▊ │ + 90. │ KIWA │ 127195 │ 250876 │ ██▌ │ + 91. │ KTEB │ 126969 │ 1189414 │ ███████████▊ │ + 92. │ VOMM │ 125616 │ 1127757 │ ███████████▎ │ + 93. │ LSGG │ 123998 │ 1049101 │ ██████████▍ │ + 94. │ LPPT │ 122733 │ 1779187 │ █████████████████▋ │ + 95. │ WSSS │ 120493 │ 3264122 │ ████████████████████████████████▋ │ + 96. │ EBBR │ 118539 │ 1579939 │ ███████████████▋ │ + 97. │ VTBD │ 118107 │ 661627 │ ██████▌ │ + 98. │ KVNY │ 116326 │ 692960 │ ██████▊ │ + 99. │ EDDT │ 115122 │ 941740 │ █████████▍ │ +100. │ EFHK │ 114860 │ 1629143 │ ████████████████▎ │ + └────────┴─────────┴──────────┴────────────────────────────────────────┘ + +100 rows in set. Elapsed: 0.186 sec. Processed 48.31 million rows, 2.17 GB (259.27 million rows/s., 11.67 GB/s.) +``` + +### Number of flights from three major Moscow airports, weekly: + +``` +SELECT + toMonday(day) AS k, + count() AS c, + bar(c, 0, 10000, 100) AS bar +FROM opensky +WHERE origin IN ('UUEE', 'UUDD', 'UUWW') +GROUP BY k +ORDER BY k ASC + +Query id: 1b446157-9519-4cc4-a1cb-178dfcc15a8e + + ┌──────────k─┬────c─┬─bar──────────────────────────────────────────────────────────────────────────┐ + 1. │ 2018-12-31 │ 5248 │ ████████████████████████████████████████████████████▍ │ + 2. │ 2019-01-07 │ 6302 │ ███████████████████████████████████████████████████████████████ │ + 3. │ 2019-01-14 │ 5701 │ █████████████████████████████████████████████████████████ │ + 4. │ 2019-01-21 │ 5638 │ ████████████████████████████████████████████████████████▍ │ + 5. │ 2019-01-28 │ 5731 │ █████████████████████████████████████████████████████████▎ │ + 6. │ 2019-02-04 │ 5683 │ ████████████████████████████████████████████████████████▋ │ + 7. │ 2019-02-11 │ 5759 │ █████████████████████████████████████████████████████████▌ │ + 8. │ 2019-02-18 │ 5736 │ █████████████████████████████████████████████████████████▎ │ + 9. │ 2019-02-25 │ 5873 │ ██████████████████████████████████████████████████████████▋ │ + 10. │ 2019-03-04 │ 5965 │ ███████████████████████████████████████████████████████████▋ │ + 11. │ 2019-03-11 │ 5900 │ ███████████████████████████████████████████████████████████ │ + 12. │ 2019-03-18 │ 5823 │ ██████████████████████████████████████████████████████████▏ │ + 13. │ 2019-03-25 │ 5899 │ ██████████████████████████████████████████████████████████▊ │ + 14. │ 2019-04-01 │ 6043 │ ████████████████████████████████████████████████████████████▍ │ + 15. │ 2019-04-08 │ 6098 │ ████████████████████████████████████████████████████████████▊ │ + 16. │ 2019-04-15 │ 6196 │ █████████████████████████████████████████████████████████████▊ │ + 17. │ 2019-04-22 │ 6486 │ ████████████████████████████████████████████████████████████████▋ │ + 18. │ 2019-04-29 │ 6682 │ ██████████████████████████████████████████████████████████████████▋ │ + 19. │ 2019-05-06 │ 6739 │ ███████████████████████████████████████████████████████████████████▍ │ + 20. │ 2019-05-13 │ 6600 │ ██████████████████████████████████████████████████████████████████ │ + 21. │ 2019-05-20 │ 6575 │ █████████████████████████████████████████████████████████████████▋ │ + 22. │ 2019-05-27 │ 6786 │ ███████████████████████████████████████████████████████████████████▋ │ + 23. │ 2019-06-03 │ 6872 │ ████████████████████████████████████████████████████████████████████▋ │ + 24. │ 2019-06-10 │ 7045 │ ██████████████████████████████████████████████████████████████████████▍ │ + 25. │ 2019-06-17 │ 7045 │ ██████████████████████████████████████████████████████████████████████▍ │ + 26. │ 2019-06-24 │ 6852 │ ████████████████████████████████████████████████████████████████████▌ │ + 27. │ 2019-07-01 │ 7248 │ ████████████████████████████████████████████████████████████████████████▍ │ + 28. │ 2019-07-08 │ 7284 │ ████████████████████████████████████████████████████████████████████████▋ │ + 29. │ 2019-07-15 │ 7142 │ ███████████████████████████████████████████████████████████████████████▍ │ + 30. │ 2019-07-22 │ 7108 │ ███████████████████████████████████████████████████████████████████████ │ + 31. │ 2019-07-29 │ 7251 │ ████████████████████████████████████████████████████████████████████████▌ │ + 32. │ 2019-08-05 │ 7403 │ ██████████████████████████████████████████████████████████████████████████ │ + 33. │ 2019-08-12 │ 7457 │ ██████████████████████████████████████████████████████████████████████████▌ │ + 34. │ 2019-08-19 │ 7502 │ ███████████████████████████████████████████████████████████████████████████ │ + 35. │ 2019-08-26 │ 7540 │ ███████████████████████████████████████████████████████████████████████████▍ │ + 36. │ 2019-09-02 │ 7237 │ ████████████████████████████████████████████████████████████████████████▎ │ + 37. │ 2019-09-09 │ 7328 │ █████████████████████████████████████████████████████████████████████████▎ │ + 38. │ 2019-09-16 │ 5566 │ ███████████████████████████████████████████████████████▋ │ + 39. │ 2019-09-23 │ 7049 │ ██████████████████████████████████████████████████████████████████████▍ │ + 40. │ 2019-09-30 │ 6880 │ ████████████████████████████████████████████████████████████████████▋ │ + 41. │ 2019-10-07 │ 6518 │ █████████████████████████████████████████████████████████████████▏ │ + 42. │ 2019-10-14 │ 6688 │ ██████████████████████████████████████████████████████████████████▊ │ + 43. │ 2019-10-21 │ 6667 │ ██████████████████████████████████████████████████████████████████▋ │ + 44. │ 2019-10-28 │ 6303 │ ███████████████████████████████████████████████████████████████ │ + 45. │ 2019-11-04 │ 6298 │ ██████████████████████████████████████████████████████████████▊ │ + 46. │ 2019-11-11 │ 6137 │ █████████████████████████████████████████████████████████████▎ │ + 47. │ 2019-11-18 │ 6051 │ ████████████████████████████████████████████████████████████▌ │ + 48. │ 2019-11-25 │ 5820 │ ██████████████████████████████████████████████████████████▏ │ + 49. │ 2019-12-02 │ 5942 │ ███████████████████████████████████████████████████████████▍ │ + 50. │ 2019-12-09 │ 4891 │ ████████████████████████████████████████████████▊ │ + 51. │ 2019-12-16 │ 5682 │ ████████████████████████████████████████████████████████▋ │ + 52. │ 2019-12-23 │ 6111 │ █████████████████████████████████████████████████████████████ │ + 53. │ 2019-12-30 │ 5870 │ ██████████████████████████████████████████████████████████▋ │ + 54. │ 2020-01-06 │ 5953 │ ███████████████████████████████████████████████████████████▌ │ + 55. │ 2020-01-13 │ 5698 │ ████████████████████████████████████████████████████████▊ │ + 56. │ 2020-01-20 │ 5339 │ █████████████████████████████████████████████████████▍ │ + 57. │ 2020-01-27 │ 5566 │ ███████████████████████████████████████████████████████▋ │ + 58. │ 2020-02-03 │ 5801 │ ██████████████████████████████████████████████████████████ │ + 59. │ 2020-02-10 │ 5692 │ ████████████████████████████████████████████████████████▊ │ + 60. │ 2020-02-17 │ 5912 │ ███████████████████████████████████████████████████████████ │ + 61. │ 2020-02-24 │ 6031 │ ████████████████████████████████████████████████████████████▎ │ + 62. │ 2020-03-02 │ 6105 │ █████████████████████████████████████████████████████████████ │ + 63. │ 2020-03-09 │ 5823 │ ██████████████████████████████████████████████████████████▏ │ + 64. │ 2020-03-16 │ 4659 │ ██████████████████████████████████████████████▌ │ + 65. │ 2020-03-23 │ 3720 │ █████████████████████████████████████▏ │ + 66. │ 2020-03-30 │ 1720 │ █████████████████▏ │ + 67. │ 2020-04-06 │ 849 │ ████████▍ │ + 68. │ 2020-04-13 │ 710 │ ███████ │ + 69. │ 2020-04-20 │ 725 │ ███████▏ │ + 70. │ 2020-04-27 │ 920 │ █████████▏ │ + 71. │ 2020-05-04 │ 859 │ ████████▌ │ + 72. │ 2020-05-11 │ 1047 │ ██████████▍ │ + 73. │ 2020-05-18 │ 1135 │ ███████████▎ │ + 74. │ 2020-05-25 │ 1266 │ ████████████▋ │ + 75. │ 2020-06-01 │ 1793 │ █████████████████▊ │ + 76. │ 2020-06-08 │ 1979 │ ███████████████████▋ │ + 77. │ 2020-06-15 │ 2297 │ ██████████████████████▊ │ + 78. │ 2020-06-22 │ 2788 │ ███████████████████████████▊ │ + 79. │ 2020-06-29 │ 3389 │ █████████████████████████████████▊ │ + 80. │ 2020-07-06 │ 3545 │ ███████████████████████████████████▍ │ + 81. │ 2020-07-13 │ 3569 │ ███████████████████████████████████▋ │ + 82. │ 2020-07-20 │ 3784 │ █████████████████████████████████████▋ │ + 83. │ 2020-07-27 │ 3960 │ ███████████████████████████████████████▌ │ + 84. │ 2020-08-03 │ 4323 │ ███████████████████████████████████████████▏ │ + 85. │ 2020-08-10 │ 4581 │ █████████████████████████████████████████████▋ │ + 86. │ 2020-08-17 │ 4791 │ ███████████████████████████████████████████████▊ │ + 87. │ 2020-08-24 │ 4928 │ █████████████████████████████████████████████████▎ │ + 88. │ 2020-08-31 │ 4687 │ ██████████████████████████████████████████████▋ │ + 89. │ 2020-09-07 │ 4643 │ ██████████████████████████████████████████████▍ │ + 90. │ 2020-09-14 │ 4594 │ █████████████████████████████████████████████▊ │ + 91. │ 2020-09-21 │ 4478 │ ████████████████████████████████████████████▋ │ + 92. │ 2020-09-28 │ 4382 │ ███████████████████████████████████████████▋ │ + 93. │ 2020-10-05 │ 4261 │ ██████████████████████████████████████████▌ │ + 94. │ 2020-10-12 │ 4243 │ ██████████████████████████████████████████▍ │ + 95. │ 2020-10-19 │ 3941 │ ███████████████████████████████████████▍ │ + 96. │ 2020-10-26 │ 3616 │ ████████████████████████████████████▏ │ + 97. │ 2020-11-02 │ 3586 │ ███████████████████████████████████▋ │ + 98. │ 2020-11-09 │ 3403 │ ██████████████████████████████████ │ + 99. │ 2020-11-16 │ 3336 │ █████████████████████████████████▎ │ +100. │ 2020-11-23 │ 3230 │ ████████████████████████████████▎ │ +101. │ 2020-11-30 │ 3183 │ ███████████████████████████████▋ │ +102. │ 2020-12-07 │ 3285 │ ████████████████████████████████▋ │ +103. │ 2020-12-14 │ 3367 │ █████████████████████████████████▋ │ +104. │ 2020-12-21 │ 3748 │ █████████████████████████████████████▍ │ +105. │ 2020-12-28 │ 3986 │ ███████████████████████████████████████▋ │ +106. │ 2021-01-04 │ 3906 │ ███████████████████████████████████████ │ +107. │ 2021-01-11 │ 3425 │ ██████████████████████████████████▎ │ +108. │ 2021-01-18 │ 3144 │ ███████████████████████████████▍ │ +109. │ 2021-01-25 │ 3115 │ ███████████████████████████████▏ │ +110. │ 2021-02-01 │ 3285 │ ████████████████████████████████▋ │ +111. │ 2021-02-08 │ 3321 │ █████████████████████████████████▏ │ +112. │ 2021-02-15 │ 3475 │ ██████████████████████████████████▋ │ +113. │ 2021-02-22 │ 3549 │ ███████████████████████████████████▍ │ +114. │ 2021-03-01 │ 3755 │ █████████████████████████████████████▌ │ +115. │ 2021-03-08 │ 3080 │ ██████████████████████████████▋ │ +116. │ 2021-03-15 │ 3789 │ █████████████████████████████████████▊ │ +117. │ 2021-03-22 │ 3804 │ ██████████████████████████████████████ │ +118. │ 2021-03-29 │ 4238 │ ██████████████████████████████████████████▍ │ +119. │ 2021-04-05 │ 4307 │ ███████████████████████████████████████████ │ +120. │ 2021-04-12 │ 4225 │ ██████████████████████████████████████████▎ │ +121. │ 2021-04-19 │ 4391 │ ███████████████████████████████████████████▊ │ +122. │ 2021-04-26 │ 4868 │ ████████████████████████████████████████████████▋ │ +123. │ 2021-05-03 │ 4977 │ █████████████████████████████████████████████████▋ │ +124. │ 2021-05-10 │ 5164 │ ███████████████████████████████████████████████████▋ │ +125. │ 2021-05-17 │ 4986 │ █████████████████████████████████████████████████▋ │ +126. │ 2021-05-24 │ 5024 │ ██████████████████████████████████████████████████▏ │ +127. │ 2021-05-31 │ 4824 │ ████████████████████████████████████████████████▏ │ +128. │ 2021-06-07 │ 5652 │ ████████████████████████████████████████████████████████▌ │ +129. │ 2021-06-14 │ 5613 │ ████████████████████████████████████████████████████████▏ │ +130. │ 2021-06-21 │ 6061 │ ████████████████████████████████████████████████████████████▌ │ +131. │ 2021-06-28 │ 2554 │ █████████████████████████▌ │ + └────────────┴──────┴──────────────────────────────────────────────────────────────────────────────┘ + +131 rows in set. Elapsed: 0.014 sec. Processed 655.36 thousand rows, 11.14 MB (47.56 million rows/s., 808.48 MB/s.) +``` + +### Test it in Playground + +The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). From 578e55184e984c9ecb6d537fd74d2d1f4e46e08f Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Mon, 9 Aug 2021 00:32:36 +0300 Subject: [PATCH 147/147] add info about clang version choice --- docs/ru/development/developer-instruction.md | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index f23c2589c82..4474733fe3e 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -163,7 +163,7 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" export CC=clang CXX=clang++ cmake .. -Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву). +Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву). При получении ошибки типа `Could not find compiler set in environment variable CC: clang` необходимо указать в значениях для переменных CC и CXX явную версию компилятора, например, `clang-12` и `clang++-12`. Для более быстрой сборки, можно использовать debug вариант - сборку без оптимизаций. Для этого, укажите параметр `-D CMAKE_BUILD_TYPE=Debug`: @@ -195,6 +195,14 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" В процессе сборки могут появится сообщения `libprotobuf WARNING` про protobuf файлы в библиотеке libhdfs2. Это не имеет значения. +В случае получения ошибок вида `error: variable 'y' set but not used [-Werror,-Wunused-but-set-variable]` ножно попробовать использовать другую версию компилятора сlang. Например, на момент написания данного текста описанная выше команда по установке clang для Ubuntu 20.04 по-умолчанию устанавливает clang-13, с которым возникает эта ошибка. Для решения проблемы можно установить clang-12 с помощью команд: +```bash +wget https://apt.llvm.org/llvm.sh +chmod +x llvm.sh +sudo ./llvm.sh 12 +``` +И далее использовать именно его, указав соответствующую версию при установке переменных окружения CC и CXX перед вызовом cmake. + При успешной сборке, вы получите готовый исполняемый файл `ClickHouse/build/programs/clickhouse`: ls -l programs/clickhouse