From 31f2c65e4f9555f10406d3d20536fc97117776d3 Mon Sep 17 00:00:00 2001 From: fuqi Date: Sun, 18 Apr 2021 17:38:50 +0800 Subject: [PATCH 001/154] support join table mutation --- src/Storages/StorageJoin.cpp | 73 +++++++++++++++++-- src/Storages/StorageJoin.h | 5 ++ .../01821_join_table_mutation.reference | 5 ++ .../0_stateless/01821_join_table_mutation.sql | 23 ++++++ 4 files changed, 99 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/01821_join_table_mutation.reference create mode 100644 tests/queries/0_stateless/01821_join_table_mutation.sql diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 983b9213a35..a392c71ad92 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -1,24 +1,25 @@ #include #include +#include #include #include #include -#include #include #include #include #include #include #include +#include #include #include -#include #include +#include -#include /// toLower -#include -#include +#include #include +#include +#include /// toLower namespace DB @@ -70,6 +71,7 @@ StorageJoin::StorageJoin( void StorageJoin::truncate( const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder&) { + std::lock_guard lock(mutex); disk->removeRecursive(path); disk->createDirectories(path); disk->createDirectories(path + "tmp/"); @@ -78,6 +80,64 @@ void StorageJoin::truncate( join = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); } +void StorageJoin::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const +{ + for(const auto& command: commands) { + switch (command.type) { + case MutationCommand::Type::DELETE: + break; + case MutationCommand::Type::UPDATE: + throw Exception("Table engine Join doesn't support update mutation, please use insert instead", ErrorCodes::NOT_IMPLEMENTED); + default: + throw Exception("Table engine Join doesn't support this mutation", ErrorCodes::NOT_IMPLEMENTED); + } + } +} + +void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) +{ + // Only delete is supported + std::lock_guard lock(mutex); + auto metadata_snapshot = getInMemoryMetadataPtr(); + auto storage = getStorageID(); + auto storage_ptr = DatabaseCatalog::instance().getTable(storage, context); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true); + auto in = interpreter->execute(); + in->readPrefix(); + + auto new_data = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); + + const String backup_file_name = "1.bin"; // id starts from 1 + auto backup_buf = disk->writeFile(path + "tmp/" + backup_file_name); + auto compressed_backup_buf = CompressedWriteBuffer(*backup_buf); + auto backup_stream = NativeBlockOutputStream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()); + + while (const Block & block = in->read()) { + new_data->addJoinedBlock(block, true); + if (persistent) + backup_stream.write(block); + } + + join = std::move(new_data); + increment = 1; + + if (persistent) { + backup_stream.flush(); + compressed_backup_buf.next(); + backup_buf->next(); + backup_buf->finalize(); + + std::vector files; + disk->listFiles(path, files); + for (const auto& file_name: files) { + if (file_name.ends_with(".bin")) { + disk->removeFileIfExists(path + file_name); + } + } + + disk->replaceFile(path + "tmp/" + backup_file_name, path + backup_file_name); + } +} HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join) const { @@ -104,8 +164,7 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join) void StorageJoin::insertBlock(const Block & block) -{ - std::unique_lock lock(rwlock); +{std::unique_lock lock(rwlock); join->addJoinedBlock(block, true); } diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 4baac53c69c..2485fe015b0 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -29,6 +29,9 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; + void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; + void mutate(const MutationCommands & commands, ContextPtr context) override; + /// Return instance of HashJoin holding lock that protects from insertions to StorageJoin. /// HashJoin relies on structure of hash table that's why we need to return it with locked mutex. HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join) const; @@ -69,6 +72,8 @@ private: /// Lock is stored in HashJoin instance during query and blocks concurrent insertions. mutable std::shared_mutex rwlock; + mutable std::mutex mutex; + void insertBlock(const Block & block) override; void finishInsert() override {} size_t getSize() const override; diff --git a/tests/queries/0_stateless/01821_join_table_mutation.reference b/tests/queries/0_stateless/01821_join_table_mutation.reference new file mode 100644 index 00000000000..0d7ca32ed70 --- /dev/null +++ b/tests/queries/0_stateless/01821_join_table_mutation.reference @@ -0,0 +1,5 @@ +100 +10 +99 +m10 +50 \ No newline at end of file diff --git a/tests/queries/0_stateless/01821_join_table_mutation.sql b/tests/queries/0_stateless/01821_join_table_mutation.sql new file mode 100644 index 00000000000..9662a197b88 --- /dev/null +++ b/tests/queries/0_stateless/01821_join_table_mutation.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS join_table_mutation; + +CREATE TABLE join_table_mutation(id Int32, name String) ENGINE = Join(ANY, LEFT, id); + +INSERT INTO join_table_mutation select number, toString(number) from numbers(100); + +SELECT count(1) FROM join_table_mutation; + +SELECT name FROM join_table_mutation WHERE id = 10; + +ALTER TABLE join_table_mutation DELETE WHERE id = 10; + +SELECT count(1) FROM join_table_mutation; + +SELECT name FROM join_table_mutation WHERE id = 10; + +INSERT INTO join_table_mutation VALUES (10, 'm10'); + +SELECT name FROM join_table_mutation WHERE id = 10; + +ALTER TABLE join_table_mutation DELETE WHERE id % 2 = 0; + +SELECT count(1) FROM join_table_mutation; \ No newline at end of file From 9f4c6adb43b94dff88663cea3aac3678dc3dc9ce Mon Sep 17 00:00:00 2001 From: fuqi Date: Sun, 18 Apr 2021 18:05:18 +0800 Subject: [PATCH 002/154] fix code style --- src/Storages/StorageJoin.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index a392c71ad92..d5b858b46bc 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -82,7 +82,7 @@ void StorageJoin::truncate( void StorageJoin::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const { - for(const auto& command: commands) { + for (const auto & command: commands) { switch (command.type) { case MutationCommand::Type::DELETE: break; @@ -129,7 +129,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) std::vector files; disk->listFiles(path, files); - for (const auto& file_name: files) { + for (const auto & file_name: files) { if (file_name.ends_with(".bin")) { disk->removeFileIfExists(path + file_name); } @@ -164,7 +164,8 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join) void StorageJoin::insertBlock(const Block & block) -{std::unique_lock lock(rwlock); +{ + std::unique_lock lock(rwlock); join->addJoinedBlock(block, true); } From 73a5e7a7c3dd8960ace220f0080667bb25125f36 Mon Sep 17 00:00:00 2001 From: fuqi Date: Mon, 19 Apr 2021 10:22:18 +0800 Subject: [PATCH 003/154] fix test case new line --- tests/queries/0_stateless/01821_join_table_mutation.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01821_join_table_mutation.reference b/tests/queries/0_stateless/01821_join_table_mutation.reference index 0d7ca32ed70..e79d145b39b 100644 --- a/tests/queries/0_stateless/01821_join_table_mutation.reference +++ b/tests/queries/0_stateless/01821_join_table_mutation.reference @@ -2,4 +2,4 @@ 10 99 m10 -50 \ No newline at end of file +50 From 2fd04ec9353e27f85fd09bbcbfc0053ab4a7ecc5 Mon Sep 17 00:00:00 2001 From: fuqi Date: Mon, 19 Apr 2021 11:22:35 +0800 Subject: [PATCH 004/154] fix code style --- src/Storages/StorageJoin.cpp | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index d5b858b46bc..92f16d703c9 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -82,8 +82,10 @@ void StorageJoin::truncate( void StorageJoin::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const { - for (const auto & command: commands) { - switch (command.type) { + for (const auto & command: commands) + { + switch (command.type) + { case MutationCommand::Type::DELETE: break; case MutationCommand::Type::UPDATE: @@ -112,7 +114,8 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) auto compressed_backup_buf = CompressedWriteBuffer(*backup_buf); auto backup_stream = NativeBlockOutputStream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()); - while (const Block & block = in->read()) { + while (const Block & block = in->read()) + { new_data->addJoinedBlock(block, true); if (persistent) backup_stream.write(block); @@ -121,7 +124,8 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) join = std::move(new_data); increment = 1; - if (persistent) { + if (persistent) + { backup_stream.flush(); compressed_backup_buf.next(); backup_buf->next(); @@ -129,10 +133,10 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) std::vector files; disk->listFiles(path, files); - for (const auto & file_name: files) { - if (file_name.ends_with(".bin")) { + for (const auto & file_name: files) + { + if (file_name.ends_with(".bin")) disk->removeFileIfExists(path + file_name); - } } disk->replaceFile(path + "tmp/" + backup_file_name, path + backup_file_name); From 66913c58653a7851c6b3efac6dd5e5aadc32b103 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 19 Apr 2021 19:24:50 +0300 Subject: [PATCH 005/154] fix waiting while dropping empty parts --- src/Storages/MergeTree/MergeTreeData.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f28d87bb9be..a1a50fc6cb7 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1305,6 +1305,9 @@ void MergeTreeData::clearEmptyParts() if (!getSettings()->remove_empty_parts) return; + /// There is no need to wait for drop and hold thread in background pool. + auto context_for_drop = Context::createCopy(getContext()); + context_for_drop->setSetting("replication_alter_partitions_sync", Field(0)); auto parts = getDataPartsVector(); for (const auto & part : parts) { @@ -1312,7 +1315,7 @@ void MergeTreeData::clearEmptyParts() { ASTPtr literal = std::make_shared(part->name); /// If another replica has already started drop, it's ok, no need to throw. - dropPartition(literal, /* detach = */ false, /*drop_part = */ true, getContext(), /* throw_if_noop = */ false); + dropPartition(literal, /* detach = */ false, /*drop_part = */ true, context_for_drop, /* throw_if_noop = */ false); } } } From 42a0416a2bb0fe933218214420d958392b54c557 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 20 Apr 2021 05:31:08 +0300 Subject: [PATCH 006/154] better drop of empty parts --- src/Storages/MergeTree/MergeTreeData.cpp | 9 +- src/Storages/MergeTree/MergeTreeData.h | 4 +- src/Storages/StorageMergeTree.cpp | 104 +++++++++++--------- src/Storages/StorageMergeTree.h | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 15 ++- src/Storages/StorageReplicatedMergeTree.h | 3 +- 6 files changed, 81 insertions(+), 58 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a1a50fc6cb7..ed098862f92 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1305,18 +1305,11 @@ void MergeTreeData::clearEmptyParts() if (!getSettings()->remove_empty_parts) return; - /// There is no need to wait for drop and hold thread in background pool. - auto context_for_drop = Context::createCopy(getContext()); - context_for_drop->setSetting("replication_alter_partitions_sync", Field(0)); auto parts = getDataPartsVector(); for (const auto & part : parts) { if (part->rows_count == 0) - { - ASTPtr literal = std::make_shared(part->name); - /// If another replica has already started drop, it's ok, no need to throw. - dropPartition(literal, /* detach = */ false, /*drop_part = */ true, context_for_drop, /* throw_if_noop = */ false); - } + dropPart(part->name); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 46c0014d9f7..5b107308e2a 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -964,7 +964,9 @@ protected: // Partition helpers bool canReplacePartition(const DataPartPtr & src_part) const; - virtual void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr context, bool throw_if_noop = true) = 0; + /// Tries to drop part in background without any waits or throwing exceptions in case of errors. + virtual void dropPart(const String & name) = 0; + virtual void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr context) = 0; virtual PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr context) = 0; virtual void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr context) = 0; virtual void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr context) = 0; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4d7f7d8c887..dbe15ca4e3f 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1213,55 +1213,69 @@ MergeTreeDataPartPtr StorageMergeTree::outdatePart(const String & part_name, boo } } -void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr local_context, bool throw_if_noop) +void StorageMergeTree::dropPart(const String & name) { + auto part = outdatePart(name, false); + /// Nothing to do, part was removed in some different way + if (!part) + return; + + dropPartsImpl({part}, false); +} + +void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr local_context) +{ + DataPartsVector parts_to_remove; + + if (drop_part) { - MergeTreeData::DataPartsVector parts_to_remove; - auto metadata_snapshot = getInMemoryMetadataPtr(); + auto part = outdatePart(partition->as().value.safeGet(), true); + /// Nothing to do, part was removed in some different way + if (!part) + return; - if (drop_part) - { - auto part = outdatePart(partition->as().value.safeGet(), throw_if_noop); - /// Nothing to do, part was removed in some different way - if (!part) - return; - - parts_to_remove.push_back(part); - } - else - { - /// Asks to complete merges and does not allow them to start. - /// This protects against "revival" of data for a removed partition after completion of merge. - auto merge_blocker = stopMergesAndWait(); - String partition_id = getPartitionIDFromQuery(partition, local_context); - parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); - - /// TODO should we throw an exception if parts_to_remove is empty? - removePartsFromWorkingSet(parts_to_remove, true); - } - - if (detach) - { - /// If DETACH clone parts to detached/ directory - /// NOTE: no race with background cleanup until we hold pointers to parts - for (const auto & part : parts_to_remove) - { - LOG_INFO(log, "Detaching {}", part->relative_path); - part->makeCloneInDetached("", metadata_snapshot); - } - } - - if (deduplication_log) - { - for (const auto & part : parts_to_remove) - deduplication_log->dropPart(part->info); - } - - if (detach) - LOG_INFO(log, "Detached {} parts.", parts_to_remove.size()); - else - LOG_INFO(log, "Removed {} parts.", parts_to_remove.size()); + parts_to_remove.push_back(part); } + else + { + /// Asks to complete merges and does not allow them to start. + /// This protects against "revival" of data for a removed partition after completion of merge. + auto merge_blocker = stopMergesAndWait(); + String partition_id = getPartitionIDFromQuery(partition, local_context); + parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + + /// TODO should we throw an exception if parts_to_remove is empty? + removePartsFromWorkingSet(parts_to_remove, true); + } + + dropPartsImpl(parts_to_remove, detach); +} + +void StorageMergeTree::dropPartsImpl(const DataPartsVector & parts_to_remove, bool detach) +{ + auto metadata_snapshot = getInMemoryMetadataPtr(); + + if (detach) + { + /// If DETACH clone parts to detached/ directory + /// NOTE: no race with background cleanup until we hold pointers to parts + for (const auto & part : parts_to_remove) + { + LOG_INFO(log, "Detaching {}", part->relative_path); + part->makeCloneInDetached("", metadata_snapshot); + } + } + + if (deduplication_log) + { + for (const auto & part : parts_to_remove) + deduplication_log->dropPart(part->info); + } + + if (detach) + LOG_INFO(log, "Detached {} parts.", parts_to_remove.size()); + else + LOG_INFO(log, "Removed {} parts.", parts_to_remove.size()); clearOldPartsFromFilesystem(); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 2a50cb33912..0cd24f37518 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -211,7 +211,9 @@ private: void clearOldMutations(bool truncate = false); // Partition helpers - void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr context, bool throw_if_noop) override; + void dropPart(const String & name) override; + void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr context) override; + void dropPartsImpl(const DataPartsVector & parts_to_remove, bool detach); PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr context) override; void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr context) override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 10061af22e7..ce7c0b3cf59 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4777,8 +4777,19 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St return true; } +void StorageReplicatedMergeTree::dropPart(const String & name) +{ + assertNotReadonly(); + if (!is_leader) + throw Exception("DROP PART cannot be done on this replica because it is not a leader", ErrorCodes::NOT_A_LEADER); -void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr query_context, bool throw_if_noop) + zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); + LogEntry entry; + + dropPart(zookeeper, name, entry, false, false); +} + +void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr query_context) { assertNotReadonly(); if (!is_leader) @@ -4792,7 +4803,7 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool de if (drop_part) { String part_name = partition->as().value.safeGet(); - did_drop = dropPart(zookeeper, part_name, entry, detach, throw_if_noop); + did_drop = dropPart(zookeeper, part_name, entry, detach, true); } else { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 9122bdafbf0..fcf17ae07cc 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -620,12 +620,13 @@ private: /// Info about how other replicas can access this one. ReplicatedMergeTreeAddress getReplicatedMergeTreeAddress() const; + void dropPart(const String & name) override; bool dropPart(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop); bool dropAllPartsInPartition( zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, ContextPtr query_context, bool detach); // Partition helpers - void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr query_context, bool throw_if_noop) override; + void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr query_context) override; PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr query_context) override; void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr query_context) override; void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) override; From 572b825d4df9c9f1b802b1667aebfc87594742df Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 22 Apr 2021 23:07:00 +0300 Subject: [PATCH 007/154] fix dropPartition in StorageMergeTree --- src/Storages/StorageMergeTree.cpp | 6 ++++-- src/Storages/StorageMergeTree.h | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index dbe15ca4e3f..c150368922a 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1248,10 +1248,10 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool removePartsFromWorkingSet(parts_to_remove, true); } - dropPartsImpl(parts_to_remove, detach); + dropPartsImpl(std::move(parts_to_remove), detach); } -void StorageMergeTree::dropPartsImpl(const DataPartsVector & parts_to_remove, bool detach) +void StorageMergeTree::dropPartsImpl(DataPartsVector && parts_to_remove, bool detach) { auto metadata_snapshot = getInMemoryMetadataPtr(); @@ -1277,6 +1277,8 @@ void StorageMergeTree::dropPartsImpl(const DataPartsVector & parts_to_remove, bo else LOG_INFO(log, "Removed {} parts.", parts_to_remove.size()); + /// Need to destroy part objects before clearing them from filesystem. + parts_to_remove.clear(); clearOldPartsFromFilesystem(); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 0cd24f37518..e2513ce65a1 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -213,7 +213,7 @@ private: // Partition helpers void dropPart(const String & name) override; void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr context) override; - void dropPartsImpl(const DataPartsVector & parts_to_remove, bool detach); + void dropPartsImpl(DataPartsVector && parts_to_remove, bool detach); PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr context) override; void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr context) override; From eeb71672a0217f6b7660872ec8f84f3851627f0a Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Apr 2021 03:05:43 +0300 Subject: [PATCH 008/154] Change in Storages/* --- src/Storages/Distributed/DirectoryMonitor.cpp | 50 ++++++++----------- .../DistributedBlockOutputStream.cpp | 18 +++---- src/Storages/MergeTree/DataPartsExchange.cpp | 12 ++--- src/Storages/MergeTree/MergeTreeData.cpp | 9 ++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 - .../MergeTreeIndexGranularityInfo.cpp | 7 +-- .../MergeTree/MergeTreeReaderInMemory.cpp | 1 - .../RocksDB/StorageEmbeddedRocksDB.cpp | 12 +++-- src/Storages/StorageDistributed.cpp | 4 +- src/Storages/StorageFile.cpp | 34 ++++++------- src/Storages/StorageFile.h | 4 -- 11 files changed, 71 insertions(+), 82 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 2afa9747c60..ac376f92e73 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -25,11 +25,9 @@ #include #include #include - #include #include - -#include +#include namespace CurrentMetrics @@ -38,6 +36,8 @@ namespace CurrentMetrics extern const Metric DistributedFilesToInsert; } +namespace fs = std::filesystem; + namespace DB { @@ -332,7 +332,7 @@ void StorageDistributedDirectoryMonitor::shutdownAndDropAllData() } auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); - Poco::File(path).remove(true); + fs::remove_all(path); } @@ -475,16 +475,15 @@ std::map StorageDistributedDirectoryMonitor::getFiles() std::map files; size_t new_bytes_count = 0; - Poco::DirectoryIterator end; - for (Poco::DirectoryIterator it{path}; it != end; ++it) + fs::directory_iterator end; + for (fs::directory_iterator it{path}; it != end; ++it) { const auto & file_path_str = it->path(); - Poco::Path file_path{file_path_str}; - - if (!it->isDirectory() && startsWith(file_path.getExtension(), "bin")) + fs::path fs_file_path(file_path_str); + if (!it->is_directory() && startsWith(fs_file_path.extension(), ".bin")) { - files[parse(file_path.getBaseName())] = file_path_str; - new_bytes_count += Poco::File(file_path).getSize(); + files[parse(fs_file_path.stem())] = file_path_str; + new_bytes_count += fs::file_size(fs_file_path); } } @@ -646,8 +645,7 @@ struct StorageDistributedDirectoryMonitor::Batch String tmp_file{parent.current_batch_file_path + ".tmp"}; auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, parent.disk, parent.relative_path); - - if (Poco::File{tmp_file}.exists()) + if (fs::exists(tmp_file)) LOG_ERROR(parent.log, "Temporary file {} exists. Unclean shutdown?", backQuote(tmp_file)); { @@ -659,7 +657,7 @@ struct StorageDistributedDirectoryMonitor::Batch out.sync(); } - Poco::File{tmp_file}.renameTo(parent.current_batch_file_path); + fs::rename(tmp_file, parent.current_batch_file_path); } auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(parent.storage.getContext()->getSettingsRef()); auto connection = parent.pool->get(timeouts); @@ -732,7 +730,7 @@ struct StorageDistributedDirectoryMonitor::Batch total_bytes = 0; recovered = false; - Poco::File{parent.current_batch_file_path}.setSize(0); + fs::resize_file(parent.current_batch_file_path, 0); } void writeText(WriteBuffer & out) @@ -832,7 +830,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map { std::unordered_set file_indices_to_skip; - if (Poco::File{current_batch_file_path}.exists()) + if (fs::exists(current_batch_file_path)) { /// Possibly, we failed to send a batch on the previous iteration. Try to send exactly the same batch. Batch batch(*this, files); @@ -933,8 +931,8 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map /// current_batch.txt will not exist if there was no send /// (this is the case when all batches that was pending has been marked as pending) - if (Poco::File{current_batch_file_path}.exists()) - Poco::File{current_batch_file_path}.remove(); + if (fs::exists(current_batch_file_path)) + fs::remove(current_batch_file_path); } } @@ -946,30 +944,26 @@ void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_p const auto & broken_path = base_path + "broken/"; const auto & broken_file_path = broken_path + file_name; - Poco::File{broken_path}.createDirectory(); + fs::create_directory(broken_path); auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); auto broken_dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path + "/broken/"); - Poco::File file(file_path); - { std::lock_guard metrics_lock(metrics_mutex); - size_t file_size = file.getSize(); + size_t file_size = fs::file_size(file_path); --files_count; bytes_count -= file_size; } - file.renameTo(broken_file_path); - + fs::rename(file_path, broken_file_path); LOG_ERROR(log, "Renamed `{}` to `{}`", file_path, broken_file_path); } + void StorageDistributedDirectoryMonitor::markAsSend(const std::string & file_path) { - Poco::File file(file_path); - - size_t file_size = file.getSize(); + size_t file_size = fs::file_size(file_path); { std::lock_guard metrics_lock(metrics_mutex); @@ -978,7 +972,7 @@ void StorageDistributedDirectoryMonitor::markAsSend(const std::string & file_pat bytes_count -= file_size; } - file.remove(); + fs::remove(file_path); } bool StorageDistributedDirectoryMonitor::maybeMarkAsBroken(const std::string & file_path, const Exception & e) diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index d05fbae60d9..c5d4dc7486c 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -33,11 +33,10 @@ #include #include -#include - #include #include #include +#include namespace CurrentMetrics @@ -50,10 +49,11 @@ namespace ProfileEvents extern const Event DistributedSyncInsertionTimeoutExceeded; } +namespace fs = std::filesystem; + namespace DB { - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -656,10 +656,10 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: /// hardlinking to ensure the inode is not freed until we're done { const std::string path(disk_path + data_path + *it); - Poco::File(path).createDirectory(); - const std::string tmp_path(path + "/tmp/"); - Poco::File(tmp_path).createDirectory(); + + fs::create_directory(path); + fs::create_directory(tmp_path); const std::string file_name(toString(storage.file_names_increment.get()) + ".bin"); @@ -723,17 +723,17 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: for (; it != dir_names.end(); ++it) { const std::string path(disk_path + data_path + *it); - Poco::File(path).createDirectory(); + fs::create_directory(path); const std::string block_file_path(path + '/' + toString(storage.file_names_increment.get()) + ".bin"); createHardLink(first_file_tmp_path, block_file_path); auto dir_sync_guard = make_directory_sync_guard(*it); } - auto file_size = Poco::File(first_file_tmp_path).getSize(); + auto file_size = fs::file_size(first_file_tmp_path); /// remove the temporary file, enabling the OS to reclaim inode after all threads /// have removed their corresponding files - Poco::File(first_file_tmp_path).remove(); + fs::remove(first_file_tmp_path); /// Notify auto sleep_ms = context->getSettingsRef().distributed_directory_monitor_sleep_time_ms; diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 205d57f533e..dfeb31af7b7 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -13,11 +13,11 @@ #include #include #include - -#include #include +namespace fs = std::filesystem; + namespace CurrentMetrics { extern const Metric ReplicatedSend; @@ -281,13 +281,13 @@ void Service::sendPartS3Metadata(const MergeTreeData::DataPartPtr & part, WriteB String metadata_file = disk->getPath() + part->getFullRelativePath() + file_name; - Poco::File metadata(metadata_file); + fs::path metadata(metadata_file); - if (!metadata.exists()) + if (!fs::exists(metadata)) throw Exception("S3 metadata '" + file_name + "' is not exists", ErrorCodes::CORRUPTED_DATA); - if (!metadata.isFile()) + if (!fs::is_regular_file(metadata)) throw Exception("S3 metadata '" + file_name + "' is not a file", ErrorCodes::CORRUPTED_DATA); - UInt64 file_size = metadata.getSize(); + UInt64 file_size = fs::file_size(metadata); writeStringBinary(it.first, out); writeBinary(file_size, out); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee8e15008cb..426b95968b6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -51,8 +51,6 @@ #include #include -#include - #include #include @@ -66,8 +64,11 @@ #include #include #include +#include +namespace fs = std::filesystem; + namespace ProfileEvents { extern const Event RejectedInserts; @@ -3834,9 +3835,9 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( const String & with_name, ContextPtr local_context) { - String clickhouse_path = Poco::Path(local_context->getPath()).makeAbsolute().toString(); + String clickhouse_path = fs::absolute(local_context->getPath()); String default_shadow_path = clickhouse_path + "shadow/"; - Poco::File(default_shadow_path).createDirectories(); + fs::create_directories(default_shadow_path); auto increment = Increment(default_shadow_path + "increment.txt").get(true); const String shadow_path = "shadow/"; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ddb140989f6..0f30929eeb8 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -3,8 +3,6 @@ #include #include -#include - #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index ed0e7e55fc8..6da0a822f7f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -1,6 +1,8 @@ #include #include -#include + + +namespace fs = std::filesystem; namespace DB { @@ -17,8 +19,7 @@ std::optional MergeTreeIndexGranularityInfo::getMarksExtensionFromF { for (DiskDirectoryIteratorPtr it = disk->iterateDirectory(path_to_part); it->isValid(); it->next()) { - Poco::Path path(it->path()); - const auto & ext = "." + path.getExtension(); + const auto & ext = fs::path(it->path()).extension(); if (ext == getNonAdaptiveMrkExtension() || ext == getAdaptiveMrkExtension(MergeTreeDataPartType::WIDE) || ext == getAdaptiveMrkExtension(MergeTreeDataPartType::COMPACT)) diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 5ee4aa555e6..cf5c850daa1 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 9173c23ec5a..d878fc63655 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -26,14 +26,16 @@ #include #include -#include -#include #include #include #include #include +#include + + +namespace fs = std::filesystem; namespace DB { @@ -253,7 +255,7 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, rocksdb_dir = context_->getPath() + relative_data_path_; if (!attach) { - Poco::File(rocksdb_dir).createDirectories(); + fs::create_directories(rocksdb_dir); } initDb(); } @@ -261,8 +263,8 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) { rocksdb_ptr->Close(); - Poco::File(rocksdb_dir).remove(true); - Poco::File(rocksdb_dir).createDirectories(); + fs::remove_all(rocksdb_dir); + fs::create_directories(rocksdb_dir); initDb(); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index e42e53d3f1b..6c3278d9598 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -76,6 +76,8 @@ #include +namespace fs = std::filesystem; + namespace { const UInt64 FORCE_OPTIMIZE_SKIP_UNUSED_SHARDS_HAS_SHARDING_KEY = 1; @@ -845,7 +847,7 @@ StoragePolicyPtr StorageDistributed::getStoragePolicy() const void StorageDistributed::createDirectoryMonitors(const DiskPtr & disk) { const std::string path(disk->getPath() + relative_data_path); - Poco::File{path}.createDirectories(); + fs::create_directories(path); std::filesystem::directory_iterator begin(path); std::filesystem::directory_iterator end; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 14b91d29805..be052f91b39 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -28,9 +28,6 @@ #include #include -#include -#include - #include #include #include @@ -39,6 +36,7 @@ #include #include + namespace fs = std::filesystem; namespace DB @@ -78,10 +76,9 @@ std::vector listFilesWithRegexpMatching(const std::string & path_fo std::vector result; const std::string prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); - if (!fs::exists(fs::path(prefix_without_globs))) - { + if (!fs::exists(prefix_without_globs)) return result; - } + const fs::directory_iterator end; for (fs::directory_iterator it(prefix_without_globs); it != end; ++it) { @@ -125,21 +122,20 @@ void checkCreationIsAllowed(ContextPtr context_global, const std::string & db_di if (!startsWith(table_path, db_dir_path) && table_path != "/dev/null") throw Exception("File is not inside " + db_dir_path, ErrorCodes::DATABASE_ACCESS_DENIED); - Poco::File table_path_poco_file = Poco::File(table_path); - if (table_path_poco_file.exists() && table_path_poco_file.isDirectory()) + if (fs::exists(table_path) && fs::is_directory(table_path)) throw Exception("File must not be a directory", ErrorCodes::INCORRECT_FILE_NAME); } } Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context) { - String user_files_absolute_path = Poco::Path(user_files_path).makeAbsolute().makeDirectory().toString(); - Poco::Path poco_path = Poco::Path(table_path); - if (poco_path.isRelative()) - poco_path = Poco::Path(user_files_absolute_path, poco_path); + fs::path user_files_absolute_path = fs::absolute(user_files_path); + fs::path fs_table_path(table_path); + if (fs_table_path.is_relative()) + fs_table_path = user_files_absolute_path / fs_table_path; Strings paths; - const String path = poco_path.absolute().toString(); + const String path = fs::absolute(fs_table_path); if (path.find_first_of("*?{") == std::string::npos) paths.push_back(path); else @@ -205,7 +201,7 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu throw Exception("Distributed format is allowed only with explicit file path", ErrorCodes::INCORRECT_FILE_NAME); String table_dir_path = base_path + relative_table_dir_path + "/"; - Poco::File(table_dir_path).createDirectories(); + fs::create_directories(table_dir_path); paths = {getTablePath(table_dir_path, format_name)}; } @@ -455,7 +451,7 @@ Pipe StorageFile::read( if (use_table_fd) /// need to call ctr BlockInputStream paths = {""}; /// when use fd, paths are empty else - if (paths.size() == 1 && !Poco::File(paths[0]).exists()) + if (paths.size() == 1 && !fs::exists(paths[0])) { if (context->getSettingsRef().engine_file_empty_if_not_exists) return Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); @@ -598,7 +594,7 @@ BlockOutputStreamPtr StorageFile::write( if (!paths.empty()) { path = paths[0]; - Poco::File(Poco::Path(path).makeParent()).createDirectories(); + fs::create_directories(fs::path(path).parent_path()); } return std::make_shared( @@ -635,8 +631,8 @@ void StorageFile::rename(const String & new_path_to_table_data, const StorageID if (path_new == paths[0]) return; - Poco::File(Poco::Path(path_new).parent()).createDirectories(); - Poco::File(paths[0]).renameTo(path_new); + fs::create_directories(fs::path(path_new).parent_path()); + fs::rename(paths[0], path_new); paths[0] = std::move(path_new); renameInMemory(new_table_id); @@ -658,7 +654,7 @@ void StorageFile::truncate( } else { - if (!Poco::File(paths[0]).exists()) + if (!fs::exists(paths[0])) return; if (0 != ::truncate(paths[0].c_str(), 0)) diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index a277dda7cc0..b30e5eeb5d8 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -1,10 +1,6 @@ #pragma once #include - -#include -#include - #include #include From 1e4a61ce63347212bc41c5a91554e97a4cc97bd1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Apr 2021 20:22:39 +0300 Subject: [PATCH 009/154] Fix build --- src/Storages/Distributed/DirectoryMonitor.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index ac376f92e73..7381468a399 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -479,11 +479,10 @@ std::map StorageDistributedDirectoryMonitor::getFiles() for (fs::directory_iterator it{path}; it != end; ++it) { const auto & file_path_str = it->path(); - fs::path fs_file_path(file_path_str); - if (!it->is_directory() && startsWith(fs_file_path.extension(), ".bin")) + if (!it->is_directory() && startsWith(fs::path(file_path_str).extension(), ".bin")) { - files[parse(fs_file_path.stem())] = file_path_str; - new_bytes_count += fs::file_size(fs_file_path); + files[parse(fs::path(file_path_str).stem())] = file_path_str; + new_bytes_count += fs::file_size(fs::path(file_path_str)); } } From 5e429b1266feb652accb1ded3ca90e6dc04d6433 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 28 Apr 2021 13:42:07 +0300 Subject: [PATCH 010/154] Change in Interpreters/* --- src/Interpreters/DatabaseCatalog.cpp | 22 +++++------ .../ExternalLoaderXMLConfigRepository.cpp | 4 +- src/Interpreters/loadMetadata.cpp | 39 ++++++++++--------- src/Server/StaticRequestHandler.cpp | 16 ++++---- 4 files changed, 43 insertions(+), 38 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index d88b87a73d4..f9014063b6b 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -26,8 +27,7 @@ # include #endif -#include - +namespace fs = std::filesystem; namespace CurrentMetrics { @@ -353,10 +353,9 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d db->drop(getContext()); /// Old ClickHouse versions did not store database.sql files - Poco::File database_metadata_file( - getContext()->getPath() + "metadata/" + escapeForFileName(database_name) + ".sql"); - if (database_metadata_file.exists()) - database_metadata_file.remove(false); + fs::path database_metadata_file = fs::path(getContext()->getPath()) / "metadata" / (escapeForFileName(database_name) + ".sql"); + if (fs::exists(database_metadata_file)) + fs::remove_all(database_metadata_file); } return db; @@ -889,16 +888,15 @@ void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) /// Even if table is not loaded, try remove its data from disk. /// TODO remove data from all volumes - String data_path = getContext()->getPath() + "store/" + getPathForUUID(table.table_id.uuid); - Poco::File table_data_dir{data_path}; - if (table_data_dir.exists()) + fs::path data_path = fs::path(getContext()->getPath()) / "store" / getPathForUUID(table.table_id.uuid); + if (fs::exists(data_path)) { - LOG_INFO(log, "Removing data directory {} of dropped table {}", data_path, table.table_id.getNameForLogs()); - table_data_dir.remove(true); + LOG_INFO(log, "Removing data directory {} of dropped table {}", data_path.string(), table.table_id.getNameForLogs()); + fs::remove_all(data_path); } LOG_INFO(log, "Removing metadata {} of dropped table {}", table.metadata_path, table.table_id.getNameForLogs()); - Poco::File(table.metadata_path).remove(); + fs::remove(fs::path(table.metadata_path)); removeUUIDMappingFinally(table.table_id.uuid); CurrentMetrics::sub(CurrentMetrics::TablesToDropQueueSize, 1); diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index 63755ee1839..e1052817407 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -7,7 +7,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -59,7 +61,7 @@ std::set ExternalLoaderXMLConfigRepository::getAllLoadablesDefiniti bool ExternalLoaderXMLConfigRepository::exists(const std::string & definition_entity_name) { - return Poco::File(definition_entity_name).exists(); + return fs::exists(fs::path(definition_entity_name)); } Poco::AutoPtr ExternalLoaderXMLConfigRepository::load( diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 79076e57328..c109c53af20 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -18,7 +18,9 @@ #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -54,13 +56,13 @@ static void loadDatabase( String database_attach_query; String database_metadata_file = database_path + ".sql"; - if (Poco::File(database_metadata_file).exists()) + if (fs::exists(fs::path(database_metadata_file))) { /// There is .sql file with database creation statement. ReadBufferFromFile in(database_metadata_file, 1024); readStringUntilEOF(database_attach_query, in); } - else if (Poco::File(database_path).exists()) + else if (fs::exists(fs::path(database_path))) { /// Database exists, but .sql file is absent. It's old-style Ordinary database (e.g. system or default) database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database) + " ENGINE = Ordinary"; @@ -95,34 +97,35 @@ void loadMetadata(ContextPtr context, const String & default_database_name) * This file is deleted after successful loading of tables. * (flag is "one-shot") */ - Poco::File force_restore_data_flag_file(context->getFlagsPath() + "force_restore_data"); - bool has_force_restore_data_flag = force_restore_data_flag_file.exists(); + auto force_restore_data_flag_file = fs::path(context->getFlagsPath()) / "force_restore_data"; + bool has_force_restore_data_flag = fs::exists(force_restore_data_flag_file); /// Loop over databases. std::map databases; - Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator it(path); it != dir_end; ++it) + fs::directory_iterator dir_end; + for (fs::directory_iterator it(path); it != dir_end; ++it) { - if (it->isLink()) + if (it->is_symlink()) continue; - if (!it->isDirectory()) + const auto current_file = it->path().filename().string(); + if (!it->is_directory()) { /// TODO: DETACH DATABASE PERMANENTLY ? - if (endsWith(it.name(), ".sql")) + if (endsWith(current_file, ".sql")) { - String db_name = it.name().substr(0, it.name().size() - 4); + String db_name = current_file.substr(0, current_file.size() - 4); if (db_name != DatabaseCatalog::SYSTEM_DATABASE) databases.emplace(unescapeForFileName(db_name), path + "/" + db_name); } /// Temporary fails may be left from previous server runs. - if (endsWith(it.name(), ".tmp")) + if (endsWith(current_file, ".tmp")) { - LOG_WARNING(log, "Removing temporary file {}", it->path()); + LOG_WARNING(log, "Removing temporary file {}", it->path().string()); try { - it->remove(); + fs::remove(it->path()); } catch (...) { @@ -135,13 +138,13 @@ void loadMetadata(ContextPtr context, const String & default_database_name) } /// For '.svn', '.gitignore' directory and similar. - if (it.name().at(0) == '.') + if (current_file.at(0) == '.') continue; - if (it.name() == DatabaseCatalog::SYSTEM_DATABASE) + if (current_file == DatabaseCatalog::SYSTEM_DATABASE) continue; - databases.emplace(unescapeForFileName(it.name()), it.path().toString()); + databases.emplace(unescapeForFileName(current_file), it->path().string()); } /// clickhouse-local creates DatabaseMemory as default database by itself @@ -158,7 +161,7 @@ void loadMetadata(ContextPtr context, const String & default_database_name) { try { - force_restore_data_flag_file.remove(); + fs::remove(force_restore_data_flag_file); } catch (...) { @@ -172,7 +175,7 @@ void loadMetadataSystem(ContextPtr context) { String path = context->getPath() + "metadata/" + DatabaseCatalog::SYSTEM_DATABASE; String metadata_file = path + ".sql"; - if (Poco::File(path).exists() || Poco::File(metadata_file).exists()) + if (fs::exists(fs::path(path)) || fs::exists(fs::path(metadata_file))) { /// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted. loadDatabase(context, DatabaseCatalog::SYSTEM_DATABASE, path, true); diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index 169d6859b43..978271c1f04 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -14,14 +14,15 @@ #include -#include -#include #include #include #include #include +#include +namespace fs = std::filesystem; + namespace DB { @@ -137,14 +138,15 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) if (startsWith(response_expression, file_prefix)) { - const auto & user_files_absolute_path = Poco::Path(server.context()->getUserFilesPath()).makeAbsolute().makeDirectory().toString(); const auto & file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size()); - const auto & file_path = Poco::Path(user_files_absolute_path, file_name).makeAbsolute().toString(); - if (!Poco::File(file_path).exists()) - throw Exception("Invalid file name " + file_path + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); + fs::path user_files_absolute_path = fs::absolute(fs::path(server.context()->getUserFilesPath())); + fs::path file_path = user_files_absolute_path / file_name; - ReadBufferFromFile in(file_path); + if (!fs::exists(file_path)) + throw Exception("Invalid file name " + file_path.string() + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); + + ReadBufferFromFile in(file_path.string()); copyData(in, out); } else if (startsWith(response_expression, config_prefix)) From 3386dc32d3a4d42adfe177dc79ef38a9ee868e5f Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 28 Apr 2021 23:48:34 +0300 Subject: [PATCH 011/154] Change in Functions/* --- src/Functions/FunctionFile.cpp | 32 ++++++++++++++++---------------- src/Interpreters/Context.cpp | 15 +++++++++------ 2 files changed, 25 insertions(+), 22 deletions(-) diff --git a/src/Functions/FunctionFile.cpp b/src/Functions/FunctionFile.cpp index 9247152367b..7c87f27c731 100644 --- a/src/Functions/FunctionFile.cpp +++ b/src/Functions/FunctionFile.cpp @@ -3,10 +3,11 @@ #include #include #include -#include -#include #include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -68,21 +69,20 @@ public: { const char * filename = reinterpret_cast(&chars[source_offset]); - const String user_files_path = getContext()->getUserFilesPath(); - String user_files_absolute_path = Poco::Path(user_files_path).makeAbsolute().makeDirectory().toString(); - Poco::Path poco_filepath = Poco::Path(filename); - if (poco_filepath.isRelative()) - poco_filepath = Poco::Path(user_files_absolute_path, poco_filepath); - const String file_absolute_path = poco_filepath.absolute().toString(); - checkReadIsAllowedOrThrow(user_files_absolute_path, file_absolute_path); + fs::path user_files_absolute_path = fs::canonical(fs::path(getContext()->getUserFilesPath())); + fs::path file_path(filename); + if (file_path.is_relative()) + file_path = user_files_absolute_path / file_path; - checked_filenames[row] = file_absolute_path; - auto file = Poco::File(file_absolute_path); + fs::path file_absolute_path = fs::absolute(file_path); + checkReadIsAllowedOrThrow(user_files_absolute_path.string(), file_absolute_path); - if (!file.exists()) - throw Exception(fmt::format("File {} doesn't exist.", file_absolute_path), ErrorCodes::FILE_DOESNT_EXIST); + checked_filenames[row] = file_absolute_path.string(); - const auto current_file_size = Poco::File(file_absolute_path).getSize(); + if (!fs::exists(file_absolute_path)) + throw Exception(fmt::format("File {} doesn't exist.", file_absolute_path.string()), ErrorCodes::FILE_DOESNT_EXIST); + + const auto current_file_size = fs::file_size(file_absolute_path); result_offset += current_file_size + 1; res_offsets[row] = result_offset; @@ -117,8 +117,8 @@ private: if (file_absolute_path.find(user_files_absolute_path) != 0) throw Exception("File is not inside " + user_files_absolute_path, ErrorCodes::DATABASE_ACCESS_DENIED); - Poco::File path_poco_file = Poco::File(file_absolute_path); - if (path_poco_file.exists() && path_poco_file.isDirectory()) + fs::path fs_path(file_absolute_path); + if (fs::exists(fs_path) && fs::is_directory(fs_path)) throw Exception("File can't be a directory", ErrorCodes::INCORRECT_FILE_NAME); } }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8c0d8f9d48b..0a0c6f9638d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -74,8 +74,11 @@ #include #include #include +#include +namespace fs = std::filesystem; + namespace ProfileEvents { extern const Event ContextLock; @@ -2186,14 +2189,14 @@ void Context::checkCanBeDropped(const String & database, const String & table, c if (!max_size_to_drop || size <= max_size_to_drop) return; - Poco::File force_file(getFlagsPath() + "force_drop_table"); - bool force_file_exists = force_file.exists(); + fs::path force_file(getFlagsPath() + "force_drop_table"); + bool force_file_exists = fs::exists(force_file); if (force_file_exists) { try { - force_file.remove(); + fs::remove(force_file); return; } catch (...) @@ -2215,9 +2218,9 @@ void Context::checkCanBeDropped(const String & database, const String & table, c "Example:\nsudo touch '{}' && sudo chmod 666 '{}'", backQuoteIfNeed(database), backQuoteIfNeed(table), size_str, max_size_to_drop_str, - force_file.path(), force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist", - force_file.path(), - force_file.path(), force_file.path()); + force_file.string(), force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist", + force_file.string(), + force_file.string(), force_file.string()); } From 984d82c3f23ca8ba672b6e83167fae4d5d1ea8b1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 29 Apr 2021 02:00:04 +0300 Subject: [PATCH 012/154] Update DiskLocal --- src/Disks/DiskLocal.cpp | 45 +++++++++++++++++----------------- src/Functions/FunctionFile.cpp | 3 +-- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index d0cf6a00344..f567ec4c0ad 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -6,11 +6,12 @@ #include #include #include - #include +#include #include +namespace fs = std::filesystem; namespace DB { @@ -152,45 +153,43 @@ UInt64 DiskLocal::getUnreservedSpace() const bool DiskLocal::exists(const String & path) const { - return Poco::File(disk_path + path).exists(); + return fs::exists(fs::path(disk_path) / path); } bool DiskLocal::isFile(const String & path) const { - return Poco::File(disk_path + path).isFile(); + return fs::is_regular_file(fs::path(disk_path) / path); } bool DiskLocal::isDirectory(const String & path) const { - return Poco::File(disk_path + path).isDirectory(); + return fs::is_directory(fs::path(disk_path) / path); } size_t DiskLocal::getFileSize(const String & path) const { - return Poco::File(disk_path + path).getSize(); + return fs::file_size(fs::path(disk_path) / path); } void DiskLocal::createDirectory(const String & path) { - Poco::File(disk_path + path).createDirectory(); + fs::create_directory(fs::path(disk_path) / path); } void DiskLocal::createDirectories(const String & path) { - Poco::File(disk_path + path).createDirectories(); + fs::create_directories(fs::path(disk_path) / path); } void DiskLocal::clearDirectory(const String & path) { - std::vector files; - Poco::File(disk_path + path).list(files); - for (auto & file : files) - file.remove(); + for (auto & entry : fs::directory_iterator(fs::path(disk_path) / path)) + fs::remove(entry.path()); } void DiskLocal::moveDirectory(const String & from_path, const String & to_path) { - Poco::File(disk_path + from_path).renameTo(disk_path + to_path); + fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); } DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) @@ -200,22 +199,24 @@ DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) void DiskLocal::moveFile(const String & from_path, const String & to_path) { - Poco::File(disk_path + from_path).renameTo(disk_path + to_path); + fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); } void DiskLocal::replaceFile(const String & from_path, const String & to_path) { - Poco::File from_file(disk_path + from_path); - Poco::File to_file(disk_path + to_path); - if (to_file.exists()) + fs::path from_file = fs::path(disk_path) / from_path; + fs::path to_file = fs::path(disk_path) / to_path; + if (fs::exists(to_file)) { - Poco::File tmp_file(disk_path + to_path + ".old"); - to_file.renameTo(tmp_file.path()); - from_file.renameTo(disk_path + to_path); - tmp_file.remove(); + fs::path tmp_file(to_file.string() + ".old"); + fs::rename(to_file, tmp_file); + fs::rename(from_file, fs::path(disk_path) / to_path); + fs::remove(tmp_file); } else - from_file.renameTo(to_file.path()); + { + fs::rename(from_file, to_file); + } } std::unique_ptr @@ -255,7 +256,7 @@ void DiskLocal::removeDirectory(const String & path) void DiskLocal::removeRecursive(const String & path) { - Poco::File(disk_path + path).remove(true); + fs::remove_all(fs::path(disk_path) / path); } void DiskLocal::listFiles(const String & path, std::vector & file_names) diff --git a/src/Functions/FunctionFile.cpp b/src/Functions/FunctionFile.cpp index 7c87f27c731..3239705281e 100644 --- a/src/Functions/FunctionFile.cpp +++ b/src/Functions/FunctionFile.cpp @@ -73,8 +73,7 @@ public: fs::path file_path(filename); if (file_path.is_relative()) file_path = user_files_absolute_path / file_path; - - fs::path file_absolute_path = fs::absolute(file_path); + fs::path file_absolute_path = fs::canonical(file_path); checkReadIsAllowedOrThrow(user_files_absolute_path.string(), file_absolute_path); checked_filenames[row] = file_absolute_path.string(); From 8197e1c0c19d49e038bcc585f3ad0498734e424c Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 29 Apr 2021 17:30:02 +0300 Subject: [PATCH 013/154] Add qualified names for right columns for storage join --- src/Interpreters/ExpressionAnalyzer.cpp | 10 ++++- src/Interpreters/HashJoin.cpp | 42 ++++++++++++++----- src/Interpreters/TableJoin.cpp | 7 ++++ src/Interpreters/TableJoin.h | 2 + .../0_stateless/00118_storage_join.reference | 16 +++++++ .../0_stateless/00118_storage_join.sql | 6 +++ 6 files changed, 71 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index ee576c8deda..bd613662b7a 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -809,7 +809,15 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( /// Use StorageJoin if any. if (!subquery_for_join.join) - subquery_for_join.join = tryGetStorageJoin(syntax->analyzed_join); + { + if (auto storage_join = tryGetStorageJoin(syntax->analyzed_join)) + { + if (syntax->analyzed_join->hasOn()) + throw DB::Exception("`JOIN ON` with `Join` table engine not supported, use `USING` syntax", + ErrorCodes::NOT_IMPLEMENTED); + subquery_for_join.join = storage_join; + } + } if (!subquery_for_join.join) { diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index fcd89aed84d..3e501ad8eb7 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -680,7 +680,19 @@ namespace class AddedColumns { public: - using TypeAndNames = std::vector>; + + struct TypeAndName + { + DataTypePtr type; + String name; + String qualified_name; + + TypeAndName(DataTypePtr type_, const String & name_, const String & qualified_name_) + : type(type_) + , name(name_) + , qualified_name(qualified_name_) + {} + }; AddedColumns(const Block & block_with_columns_to_add, const Block & block, @@ -705,27 +717,30 @@ public: for (const auto & src_column : block_with_columns_to_add) { + /// Column names `src_column.name` and `qualified_name` can be differ for StorageJoin, + /// because it uses not qualified right block column names + auto qualified_name = join.getTableJoin().renamedRightColumnName(src_column.name); /// Don't insert column if it's in left block - if (!block.has(src_column.name)) - addColumn(src_column); + if (!block.has(qualified_name)) + addColumn(src_column, qualified_name); } if (is_asof_join) { const ColumnWithTypeAndName & right_asof_column = join.rightAsofKeyColumn(); - addColumn(right_asof_column); + addColumn(right_asof_column, right_asof_column.name); left_asof_key = key_columns.back(); } for (auto & tn : type_name) - right_indexes.push_back(saved_block_sample.getPositionByName(tn.second)); + right_indexes.push_back(saved_block_sample.getPositionByName(tn.name)); } size_t size() const { return columns.size(); } ColumnWithTypeAndName moveColumn(size_t i) { - return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].first, type_name[i].second); + return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].type, type_name[i].qualified_name); } template @@ -748,7 +763,7 @@ public: if (lazy_defaults_count) { for (size_t j = 0, size = right_indexes.size(); j < size; ++j) - JoinCommon::addDefaultValues(*columns[j], type_name[j].first, lazy_defaults_count); + JoinCommon::addDefaultValues(*columns[j], type_name[j].type, lazy_defaults_count); lazy_defaults_count = 0; } } @@ -764,7 +779,7 @@ public: bool need_filter = false; private: - TypeAndNames type_name; + std::vector type_name; MutableColumns columns; std::vector right_indexes; size_t lazy_defaults_count = 0; @@ -773,11 +788,11 @@ private: ASOF::Inequality asof_inequality; const IColumn * left_asof_key = nullptr; - void addColumn(const ColumnWithTypeAndName & src_column) + void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name) { columns.push_back(src_column.column->cloneEmpty()); columns.back()->reserve(src_column.column->size()); - type_name.emplace_back(src_column.type, src_column.name); + type_name.emplace_back(src_column.type, src_column.name, qualified_name); } }; @@ -1311,7 +1326,12 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) void HashJoin::joinTotals(Block & block) const { - JoinCommon::joinTotals(totals, sample_block_with_columns_to_add, *table_join, block); + Block sample_right_block = sample_block_with_columns_to_add.cloneEmpty(); + /// For StorageJoin column names isn't qualified in sample_block_with_columns_to_add + for (auto & col : sample_right_block) + col.name = getTableJoin().renamedRightColumnName(col.name); + + JoinCommon::joinTotals(totals, sample_right_block, *table_join, block); } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index f547e011a73..a088e4d0239 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -464,4 +464,11 @@ ActionsDAGPtr TableJoin::applyKeyConvertToTable( return dag; } +String TableJoin::renamedRightColumnName(const String & name) const +{ + if (const auto it = renames.find(name); it != renames.end()) + return it->second; + return name; +} + } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index b75ef848f13..dc8ebeb7413 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -203,6 +203,8 @@ public: /// Split key and other columns by keys name list void splitAdditionalColumns(const Block & sample_block, Block & block_keys, Block & block_others) const; Block getRequiredRightKeys(const Block & right_table_keys, std::vector & keys_sources) const; + + String renamedRightColumnName(const String & name) const; }; } diff --git a/tests/queries/0_stateless/00118_storage_join.reference b/tests/queries/0_stateless/00118_storage_join.reference index dad5b94e47b..05cb74361be 100644 --- a/tests/queries/0_stateless/00118_storage_join.reference +++ b/tests/queries/0_stateless/00118_storage_join.reference @@ -18,3 +18,19 @@ 7 8 9 +0 0 +1 1 abc +2 2 def +3 3 +4 4 +5 5 +6 6 ghi +7 7 +8 8 +9 9 +0 3 +3 9 +2 21 def +1 12 abc + +0 45 diff --git a/tests/queries/0_stateless/00118_storage_join.sql b/tests/queries/0_stateless/00118_storage_join.sql index 9da8829e5c7..17101f45679 100644 --- a/tests/queries/0_stateless/00118_storage_join.sql +++ b/tests/queries/0_stateless/00118_storage_join.sql @@ -8,4 +8,10 @@ SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT INSERT INTO join VALUES (6, 'ghi'); SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k; +SELECT k, js1.s, join.s FROM (SELECT number AS k, number as s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k; +SELECT k, js1.s, join.s FROM (SELECT toUInt64(number / 3) AS k, sum(number) as s FROM numbers(10) GROUP BY toUInt64(number / 3) WITH TOTALS) js1 ANY LEFT JOIN join USING k; + +-- JOIN ON not supported for storage join +SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join ON js1.k == join.k; -- { serverError 48 } + DROP TABLE join; From 38c8853f2bfcd01b68c8c055e1ad458af12b6f21 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 29 Apr 2021 17:34:26 +0300 Subject: [PATCH 014/154] Rename table in storage_join test --- .../0_stateless/00118_storage_join.sql | 20 ++++++++--------- .../0_stateless/00119_storage_join.sql | 22 +++++++++---------- 2 files changed, 21 insertions(+), 21 deletions(-) diff --git a/tests/queries/0_stateless/00118_storage_join.sql b/tests/queries/0_stateless/00118_storage_join.sql index 17101f45679..95b090d4837 100644 --- a/tests/queries/0_stateless/00118_storage_join.sql +++ b/tests/queries/0_stateless/00118_storage_join.sql @@ -1,17 +1,17 @@ -DROP TABLE IF EXISTS join; +DROP TABLE IF EXISTS t2; -CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k); +CREATE TABLE t2 (k UInt64, s String) ENGINE = Join(ANY, LEFT, k); -INSERT INTO join VALUES (1, 'abc'), (2, 'def'); -SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k; +INSERT INTO t2 VALUES (1, 'abc'), (2, 'def'); +SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k; -INSERT INTO join VALUES (6, 'ghi'); -SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k; +INSERT INTO t2 VALUES (6, 'ghi'); +SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k; -SELECT k, js1.s, join.s FROM (SELECT number AS k, number as s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k; -SELECT k, js1.s, join.s FROM (SELECT toUInt64(number / 3) AS k, sum(number) as s FROM numbers(10) GROUP BY toUInt64(number / 3) WITH TOTALS) js1 ANY LEFT JOIN join USING k; +SELECT k, js1.s, t2.s FROM (SELECT number AS k, number as s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k; +SELECT k, js1.s, t2.s FROM (SELECT toUInt64(number / 3) AS k, sum(number) as s FROM numbers(10) GROUP BY toUInt64(number / 3) WITH TOTALS) js1 ANY LEFT JOIN t2 USING k; -- JOIN ON not supported for storage join -SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join ON js1.k == join.k; -- { serverError 48 } +SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; -- { serverError 48 } -DROP TABLE join; +DROP TABLE t2; diff --git a/tests/queries/0_stateless/00119_storage_join.sql b/tests/queries/0_stateless/00119_storage_join.sql index e1cc7a67588..2569a64d2c3 100644 --- a/tests/queries/0_stateless/00119_storage_join.sql +++ b/tests/queries/0_stateless/00119_storage_join.sql @@ -1,18 +1,18 @@ -DROP TABLE IF EXISTS join; +DROP TABLE IF EXISTS t2; -CREATE TABLE join (s String, x Array(UInt8), k UInt64) ENGINE = Join(ANY, LEFT, k); +CREATE TABLE t2 (s String, x Array(UInt8), k UInt64) ENGINE = Join(ANY, LEFT, k); -INSERT INTO join VALUES ('abc', [0], 1), ('def', [1, 2], 2); -INSERT INTO join (k, s) VALUES (3, 'ghi'); -INSERT INTO join (x, k) VALUES ([3, 4, 5], 4); +INSERT INTO t2 VALUES ('abc', [0], 1), ('def', [1, 2], 2); +INSERT INTO t2 (k, s) VALUES (3, 'ghi'); +INSERT INTO t2 (x, k) VALUES ([3, 4, 5], 4); -SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k; -SELECT s, x FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k; -SELECT x, s, k FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k; -SELECT 1, x, 2, s, 3, k, 4 FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k; +SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k; +SELECT s, x FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k; +SELECT x, s, k FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k; +SELECT 1, x, 2, s, 3, k, 4 FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k; SELECT t1.k, t1.s, t2.x FROM ( SELECT number AS k, 'a' AS s FROM numbers(2) GROUP BY number WITH TOTALS ) AS t1 -ANY LEFT JOIN join AS t2 USING(k); +ANY LEFT JOIN t2 AS t2 USING(k); -DROP TABLE join; +DROP TABLE t2; From 654e58b77c2b376918847986e43cc07ff4d4f187 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 29 Apr 2021 22:58:42 +0300 Subject: [PATCH 015/154] Adjust test --- .../0_stateless/01658_read_file_to_stringcolumn.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01658_read_file_to_stringcolumn.reference b/tests/queries/0_stateless/01658_read_file_to_stringcolumn.reference index 87659c32e39..1d0901cf9f6 100644 --- a/tests/queries/0_stateless/01658_read_file_to_stringcolumn.reference +++ b/tests/queries/0_stateless/01658_read_file_to_stringcolumn.reference @@ -8,11 +8,11 @@ ccccccccc aaaaaaaaa bbbbbbbbb aaaaaaaaa bbbbbbbbb ccccccccc -:107 +:233 :79 :35 :35 -:35 +:233 699415 aaaaaaaaa bbbbbbbbb ccccccccc aaaaaaaaa bbbbbbbbb @@ -21,5 +21,5 @@ ccccccccc aaaaaaaaa bbbbbbbbb ccccccccc aaaaaaaaa bbbbbbbbb 699415 0 :0 -:107 +:70 :79 From 65865c1fc5abeaec43a26b1f6ff8a46b18e020d4 Mon Sep 17 00:00:00 2001 From: atereh Date: Thu, 29 Apr 2021 22:32:29 +0000 Subject: [PATCH 016/154] Initial commit --- docs/en/sql-reference/statements/optimize.md | 148 +++++++++++++++---- 1 file changed, 121 insertions(+), 27 deletions(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 247252d3f4e..47c79622e25 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -31,15 +31,17 @@ When `OPTIMIZE` is used with the [ReplicatedMergeTree](../../engines/table-engin If you want to perform deduplication on custom set of columns rather than on all, you can specify list of columns explicitly or use any combination of [`*`](../../sql-reference/statements/select/index.md#asterisk), [`COLUMNS`](../../sql-reference/statements/select/index.md#columns-expression) or [`EXCEPT`](../../sql-reference/statements/select/index.md#except-modifier) expressions. The explictly written or implicitly expanded list of columns must include all columns specified in row ordering expression (both primary and sorting keys) and partitioning expression (partitioning key). !!! note "Note" - Notice that `*` behaves just like in `SELECT`: `MATERIALIZED` and `ALIAS` columns are not used for expansion. - Also, it is an error to specify empty list of columns, or write an expression that results in an empty list of columns, or deduplicate by an ALIAS column. + Notice that `*` behaves just like in `SELECT`: [`MATERIALIZED`](../../sql-reference/statements/create/table/#materialized) and [`ALIAS`](../../sql-reference/statements/create/table/#alias) columns are not used for expansion. + Also, it is an error to specify empty list of columns, or write an expression that results in an empty list of columns, or deduplicate by an `ALIAS` column. + +**Syntax** ``` sql -OPTIMIZE TABLE table DEDUPLICATE; -- the old one -OPTIMIZE TABLE table DEDUPLICATE BY *; -- not the same as the old one, excludes MATERIALIZED columns (see the note above) +OPTIMIZE TABLE table DEDUPLICATE; -- all columns +OPTIMIZE TABLE table DEDUPLICATE BY *; -- excludes MATERIALIZED and ALIAS columns +OPTIMIZE TABLE table DEDUPLICATE BY colX,colY,colZ; OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT colX; OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT (colX, colY); -OPTIMIZE TABLE table DEDUPLICATE BY col1,col2,col3; OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex'); OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT colX; OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT (colX, colY); @@ -47,47 +49,139 @@ OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT (c **Examples** -Create a table: +1. Local example on a [`MergeTree`](../../engines/table-engines/mergetree-family/mergetree.md) table engine: ``` sql -CREATE TABLE example ( - primary_key Int32, - secondary_key Int32, +DROP TABLE IF EXISTS dup_example; + +CREATE TABLE dup_example ( + pk Int32, -- primary key + sk Int32, -- secondary key value UInt32, - partition_key UInt32, - materialized_value UInt32 MATERIALIZED 12345, - aliased_value UInt32 ALIAS 2, - PRIMARY KEY primary_key + mat UInt32 MATERIALIZED rand(), -- materialized value + alias UInt32 ALIAS 2, -- aliased value + PRIMARY KEY pk ) ENGINE=MergeTree -PARTITION BY partition_key -ORDER BY (primary_key, secondary_key); +ORDER BY (pk, sk); ``` -The 'old' deduplicate, all columns are taken into account, i.e. row is removed only if all values in all columns are equal to corresponding values in previous row. +The `MergeTree` engine does not have parameters. + +**Valid cases** + +In the case below all columns are taken into account, i.e. row is removed only if all values in all columns are equal to corresponding values in another row. +Here and below we need to add `FINAL` to force deduplication in case of a small set of data. ``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE; +OPTIMIZE TABLE dup_example FINAL DEDUPLICATE; ``` -Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED`: `primary_key`, `secondary_key`, `value`, `partition_key`, and `materialized_value` columns. +Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED`: in our case deduplicate by `pk`, `sk` and `value` columns. ``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE BY *; +OPTIMIZE TABLE dup_example FINAL DEDUPLICATE BY *; ``` -Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED` and explicitly not `materialized_value`: `primary_key`, `secondary_key`, `value`, and `partition_key` columns. +Deduplicate explicitly by `pk`, `sk`, `value` and `mat` columns. +``` sql +OPTIMIZE TABLE dup_example FINAL DEDUPLICATE BY pk, sk, value, mat; +``` + +Deduplicate by columns matching a regex `'.*k'`: `pk` and `sk` columns. ``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE BY * EXCEPT materialized_value; +OPTIMIZE TABLE dup_example FINAL DEDUPLICATE BY COLUMNS('.*k'); ``` -Deduplicate explicitly by `primary_key`, `secondary_key`, and `partition_key` columns. -``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE BY primary_key, secondary_key, partition_key; -``` +**Error cases** -Deduplicate by any column matching a regex: `primary_key`, `secondary_key`, and `partition_key` columns. +Note that **primary key** column should not be missed in any `BY` expression. These queries will face errors: ``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE BY COLUMNS('.*_key'); +OPTIMIZE TABLE dup_example DEDUPLICATE BY * EXCEPT(pk); +OPTIMIZE TABLE dup_example DEDUPLICATE BY sk, value; ``` + +Empty list cases: +``` sql +OPTIMIZE TABLE dup_example DEDUPLICATE BY * EXCEPT(pk, sk, value, mat, alias); -- server error +OPTIMIZE TABLE dup_example DEDUPLICATE BY; -- syntax error +``` + +2. Replicated example on a [`ReplicatedMergeTree`](../../engines/table-engines/mergetree-family/replication/#creating-replicated-tables) table engine: + +```sql +DROP TABLE IF EXISTS replicated_deduplicate_by_columns_r1; +DROP TABLE IF EXISTS replicated_deduplicate_by_columns_r2; + +SET replication_alter_partitions_sync = 2; + +CREATE TABLE IF NOT EXISTS replicated_deduplicate_by_columns_r1 ( + id Int32, + value UInt32, + insert_time_ns DateTime64(9) MATERIALIZED now64(9), + insert_replica_id UInt8 MATERIALIZED randConstant() +) ENGINE=ReplicatedMergeTree('zookeeper_name_configured_in_auxiliary_zookeepers:path', 'r1') +ORDER BY id; + +CREATE TABLE IF NOT EXISTS replicated_deduplicate_by_columns_r2 ( + id Int32, + value UInt32, + insert_time_ns DateTime64(9) MATERIALIZED now64(9), + insert_replica_id UInt8 MATERIALIZED randConstant() +) ENGINE=ReplicatedMergeTree('zookeeper_name_configured_in_auxiliary_zookeepers:path', 'r2') +ORDER BY id; +``` + +For the `ReplicatedMergeTree` engine we give the path to the table and name of the replica in Zookeeper. + +Insert some data into both replicas and wait for them to sync: + +```sql +SYSTEM SYNC REPLICA replicated_deduplicate_by_columns_r2; +SYSTEM SYNC REPLICA replicated_deduplicate_by_columns_r1; +``` + +Check that we have data on replicas: + +```sql +SELECT 'r1', id, value, count(), uniqExact(insert_time_ns), uniqExact(insert_replica_id) +FROM replicated_deduplicate_by_columns_r1 +GROUP BY id, value +ORDER BY id, value; + +SELECT 'r2', id, value, count(), uniqExact(insert_time_ns), uniqExact(insert_replica_id) +FROM replicated_deduplicate_by_columns_r2 +GROUP BY id, value +ORDER BY id, value; +``` + +Remove full duplicates from replica `r1` based on all columns: + +```sql +OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE; +``` + +Remove duplicates from replica `r1` based on all columns that are not `ALIAS` or `MATERIALIZED`: + +```sql +OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY *; -- except insert_time_ns, insert_replica_id +``` + +Deduplicate replica `r1` explicitly by `id` and `value` columns: + +```sql +OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY id, value; +``` + +Deduplicate by columns matching a regex: + +```sql +OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY COLUMNS('[id, value]'); + +OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY COLUMNS('[i]') EXCEPT(insert_time_ns, insert_replica_id); +``` + +Don't forget to `DROP` tables and replicas `SYSTEM DROP REPLICA` afterwards. + + From 03d6a7769042da632ecdfe1bdd5b3c9e43338fdc Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Fri, 30 Apr 2021 00:36:50 +0200 Subject: [PATCH 017/154] Initial commit --- docs/en/sql-reference/statements/optimize.md | 146 +++++++++++++++---- 1 file changed, 119 insertions(+), 27 deletions(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 247252d3f4e..8a12f073f74 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -31,15 +31,17 @@ When `OPTIMIZE` is used with the [ReplicatedMergeTree](../../engines/table-engin If you want to perform deduplication on custom set of columns rather than on all, you can specify list of columns explicitly or use any combination of [`*`](../../sql-reference/statements/select/index.md#asterisk), [`COLUMNS`](../../sql-reference/statements/select/index.md#columns-expression) or [`EXCEPT`](../../sql-reference/statements/select/index.md#except-modifier) expressions. The explictly written or implicitly expanded list of columns must include all columns specified in row ordering expression (both primary and sorting keys) and partitioning expression (partitioning key). !!! note "Note" - Notice that `*` behaves just like in `SELECT`: `MATERIALIZED` and `ALIAS` columns are not used for expansion. - Also, it is an error to specify empty list of columns, or write an expression that results in an empty list of columns, or deduplicate by an ALIAS column. + Notice that `*` behaves just like in `SELECT`: [`MATERIALIZED`](../../sql-reference/statements/create/table/#materialized) and [`ALIAS`](../../sql-reference/statements/create/table/#alias) columns are not used for expansion. + Also, it is an error to specify empty list of columns, or write an expression that results in an empty list of columns, or deduplicate by an `ALIAS` column. + +**Syntax** ``` sql -OPTIMIZE TABLE table DEDUPLICATE; -- the old one -OPTIMIZE TABLE table DEDUPLICATE BY *; -- not the same as the old one, excludes MATERIALIZED columns (see the note above) +OPTIMIZE TABLE table DEDUPLICATE; -- all columns +OPTIMIZE TABLE table DEDUPLICATE BY *; -- excludes MATERIALIZED and ALIAS columns +OPTIMIZE TABLE table DEDUPLICATE BY colX,colY,colZ; OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT colX; OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT (colX, colY); -OPTIMIZE TABLE table DEDUPLICATE BY col1,col2,col3; OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex'); OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT colX; OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT (colX, colY); @@ -47,47 +49,137 @@ OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT (c **Examples** -Create a table: +1. Local example on a [`MergeTree`](../../engines/table-engines/mergetree-family/mergetree.md) table engine: ``` sql -CREATE TABLE example ( - primary_key Int32, - secondary_key Int32, +DROP TABLE IF EXISTS dup_example; + +CREATE TABLE dup_example ( + pk Int32, -- primary key + sk Int32, -- secondary key value UInt32, - partition_key UInt32, - materialized_value UInt32 MATERIALIZED 12345, - aliased_value UInt32 ALIAS 2, - PRIMARY KEY primary_key + mat UInt32 MATERIALIZED rand(), -- materialized value + alias UInt32 ALIAS 2, -- aliased value + PRIMARY KEY pk ) ENGINE=MergeTree -PARTITION BY partition_key -ORDER BY (primary_key, secondary_key); +ORDER BY (pk, sk); ``` -The 'old' deduplicate, all columns are taken into account, i.e. row is removed only if all values in all columns are equal to corresponding values in previous row. +The `MergeTree` engine does not have parameters. + +**Valid cases** + +In the case below all columns are taken into account, i.e. row is removed only if all values in all columns are equal to corresponding values in another row. +Here and below we need to add `FINAL` to force deduplication in case of a small set of data. ``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE; +OPTIMIZE TABLE dup_example FINAL DEDUPLICATE; ``` -Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED`: `primary_key`, `secondary_key`, `value`, `partition_key`, and `materialized_value` columns. +Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED`: in our case deduplicate by `pk`, `sk` and `value` columns. ``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE BY *; +OPTIMIZE TABLE dup_example FINAL DEDUPLICATE BY *; ``` -Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED` and explicitly not `materialized_value`: `primary_key`, `secondary_key`, `value`, and `partition_key` columns. +Deduplicate explicitly by `pk`, `sk`, `value` and `mat` columns. +``` sql +OPTIMIZE TABLE dup_example FINAL DEDUPLICATE BY pk, sk, value, mat; +``` + +Deduplicate by columns matching a regex `'.*k'`: `pk` and `sk` columns. ``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE BY * EXCEPT materialized_value; +OPTIMIZE TABLE dup_example FINAL DEDUPLICATE BY COLUMNS('.*k'); ``` -Deduplicate explicitly by `primary_key`, `secondary_key`, and `partition_key` columns. -``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE BY primary_key, secondary_key, partition_key; -``` +**Error cases** -Deduplicate by any column matching a regex: `primary_key`, `secondary_key`, and `partition_key` columns. +Note that **primary key** column should not be missed in any `BY` expression. These queries will face errors: ``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE BY COLUMNS('.*_key'); +OPTIMIZE TABLE dup_example DEDUPLICATE BY * EXCEPT(pk); +OPTIMIZE TABLE dup_example DEDUPLICATE BY sk, value; ``` + +Empty list cases: +``` sql +OPTIMIZE TABLE dup_example DEDUPLICATE BY * EXCEPT(pk, sk, value, mat, alias); -- server error +OPTIMIZE TABLE dup_example DEDUPLICATE BY; -- syntax error +``` + +2. Replicated example on a [`ReplicatedMergeTree`](../../engines/table-engines/mergetree-family/replication/#creating-replicated-tables) table engine: + +```sql +DROP TABLE IF EXISTS replicated_deduplicate_by_columns_r1; +DROP TABLE IF EXISTS replicated_deduplicate_by_columns_r2; + +SET replication_alter_partitions_sync = 2; + +CREATE TABLE IF NOT EXISTS replicated_deduplicate_by_columns_r1 ( + id Int32, + value UInt32, + insert_time_ns DateTime64(9) MATERIALIZED now64(9), + insert_replica_id UInt8 MATERIALIZED randConstant() +) ENGINE=ReplicatedMergeTree('zookeeper_name_configured_in_auxiliary_zookeepers:path', 'r1') +ORDER BY id; + +CREATE TABLE IF NOT EXISTS replicated_deduplicate_by_columns_r2 ( + id Int32, + value UInt32, + insert_time_ns DateTime64(9) MATERIALIZED now64(9), + insert_replica_id UInt8 MATERIALIZED randConstant() +) ENGINE=ReplicatedMergeTree('zookeeper_name_configured_in_auxiliary_zookeepers:path', 'r2') +ORDER BY id; +``` + +For the `ReplicatedMergeTree` engine we give the path to the table and name of the replica in Zookeeper. + +Insert some data into both replicas and wait for them to sync: + +```sql +SYSTEM SYNC REPLICA replicated_deduplicate_by_columns_r2; +SYSTEM SYNC REPLICA replicated_deduplicate_by_columns_r1; +``` + +Check that we have data on replicas: + +```sql +SELECT 'r1', id, value, count(), uniqExact(insert_time_ns), uniqExact(insert_replica_id) +FROM replicated_deduplicate_by_columns_r1 +GROUP BY id, value +ORDER BY id, value; + +SELECT 'r2', id, value, count(), uniqExact(insert_time_ns), uniqExact(insert_replica_id) +FROM replicated_deduplicate_by_columns_r2 +GROUP BY id, value +ORDER BY id, value; +``` + +Remove full duplicates from replica `r1` based on all columns: + +```sql +OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE; +``` + +Remove duplicates from replica `r1` based on all columns that are not `ALIAS` or `MATERIALIZED`: + +```sql +OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY *; -- except insert_time_ns, insert_replica_id +``` + +Deduplicate replica `r1` explicitly by `id` and `value` columns: + +```sql +OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY id, value; +``` + +Deduplicate by columns matching a regex: + +```sql +OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY COLUMNS('[id, value]'); + +OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY COLUMNS('[i]') EXCEPT(insert_time_ns, insert_replica_id); +``` + +Don't forget to `DROP` tables and replicas `SYSTEM DROP REPLICA` afterwards. \ No newline at end of file From e6e37fd098d882be9ef4f26696dfcd2e7e1a62c6 Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Fri, 30 Apr 2021 01:09:13 +0200 Subject: [PATCH 018/154] link edits --- docs/en/sql-reference/statements/optimize.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 8a12f073f74..72989045b5f 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -108,7 +108,7 @@ OPTIMIZE TABLE dup_example DEDUPLICATE BY * EXCEPT(pk, sk, value, mat, alias); - OPTIMIZE TABLE dup_example DEDUPLICATE BY; -- syntax error ``` -2. Replicated example on a [`ReplicatedMergeTree`](../../engines/table-engines/mergetree-family/replication/#creating-replicated-tables) table engine: +2. Replicated example on a [`ReplicatedMergeTree`](../../engines/table-engines/mergetree-family/replication.md) table engine: ```sql DROP TABLE IF EXISTS replicated_deduplicate_by_columns_r1; From f8631a8077da1baa61499ded1fe31f8044864d50 Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Fri, 30 Apr 2021 01:21:11 +0200 Subject: [PATCH 019/154] Link fix --- docs/en/sql-reference/statements/optimize.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 72989045b5f..0eacc7e57b4 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -31,7 +31,7 @@ When `OPTIMIZE` is used with the [ReplicatedMergeTree](../../engines/table-engin If you want to perform deduplication on custom set of columns rather than on all, you can specify list of columns explicitly or use any combination of [`*`](../../sql-reference/statements/select/index.md#asterisk), [`COLUMNS`](../../sql-reference/statements/select/index.md#columns-expression) or [`EXCEPT`](../../sql-reference/statements/select/index.md#except-modifier) expressions. The explictly written or implicitly expanded list of columns must include all columns specified in row ordering expression (both primary and sorting keys) and partitioning expression (partitioning key). !!! note "Note" - Notice that `*` behaves just like in `SELECT`: [`MATERIALIZED`](../../sql-reference/statements/create/table/#materialized) and [`ALIAS`](../../sql-reference/statements/create/table/#alias) columns are not used for expansion. + Notice that `*` behaves just like in `SELECT`: [`MATERIALIZED`](./create/table/#materialized) and [`ALIAS`](./create/table/#alias) columns are not used for expansion. Also, it is an error to specify empty list of columns, or write an expression that results in an empty list of columns, or deduplicate by an `ALIAS` column. **Syntax** From 7f2f138d5692d4b9e6a3ab3d72936fa3287a2751 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 09:52:40 +0300 Subject: [PATCH 020/154] Update DiskLocal and DiskMemory --- src/Disks/DiskLocal.cpp | 88 ++++++++++++++++++++-------------------- src/Disks/DiskLocal.h | 14 ++++--- src/Disks/DiskMemory.cpp | 15 +++---- 3 files changed, 62 insertions(+), 55 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index f567ec4c0ad..3e109ad8da4 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -60,28 +60,28 @@ private: class DiskLocalDirectoryIterator : public IDiskDirectoryIterator { public: - explicit DiskLocalDirectoryIterator(const String & disk_path_, const String & dir_path_) - : dir_path(dir_path_), iter(disk_path_ + dir_path_) + explicit DiskLocalDirectoryIterator(const fs::path & disk_path_, const String & dir_path_) + : dir_path(dir_path_), entry(disk_path_ / dir_path_) { } - void next() override { ++iter; } + void next() override { ++entry; } - bool isValid() const override { return iter != Poco::DirectoryIterator(); } + bool isValid() const override { return entry != fs::directory_iterator(); } String path() const override { - if (iter->isDirectory()) - return dir_path + iter.name() + '/'; + if (entry->is_directory()) + return dir_path / entry->path().filename() / ""; else - return dir_path + iter.name(); + return dir_path / entry->path().filename(); } - String name() const override { return iter.name(); } + String name() const override { return entry->path().filename(); } private: - String dir_path; - Poco::DirectoryIterator iter; + fs::path dir_path; + fs::directory_iterator entry; }; @@ -119,9 +119,9 @@ UInt64 DiskLocal::getTotalSpace() const { struct statvfs fs; if (name == "default") /// for default disk we get space from path/data/ - fs = getStatVFS(disk_path + "data/"); + fs = getStatVFS(fs::path(disk_path / "data/").string()); else - fs = getStatVFS(disk_path); + fs = getStatVFS(disk_path.string()); UInt64 total_size = fs.f_blocks * fs.f_bsize; if (total_size < keep_free_space_bytes) return 0; @@ -134,9 +134,9 @@ UInt64 DiskLocal::getAvailableSpace() const /// available for superuser only and for system purposes struct statvfs fs; if (name == "default") /// for default disk we get space from path/data/ - fs = getStatVFS(disk_path + "data/"); + fs = getStatVFS(fs::path(disk_path / "data/").string()); else - fs = getStatVFS(disk_path); + fs = getStatVFS(disk_path.string()); UInt64 total_size = fs.f_bavail * fs.f_bsize; if (total_size < keep_free_space_bytes) return 0; @@ -153,43 +153,43 @@ UInt64 DiskLocal::getUnreservedSpace() const bool DiskLocal::exists(const String & path) const { - return fs::exists(fs::path(disk_path) / path); + return fs::exists(disk_path / path); } bool DiskLocal::isFile(const String & path) const { - return fs::is_regular_file(fs::path(disk_path) / path); + return fs::is_regular_file(disk_path / path); } bool DiskLocal::isDirectory(const String & path) const { - return fs::is_directory(fs::path(disk_path) / path); + return fs::is_directory(disk_path / path); } size_t DiskLocal::getFileSize(const String & path) const { - return fs::file_size(fs::path(disk_path) / path); + return fs::file_size(disk_path / path); } void DiskLocal::createDirectory(const String & path) { - fs::create_directory(fs::path(disk_path) / path); + fs::create_directory(disk_path / path); } void DiskLocal::createDirectories(const String & path) { - fs::create_directories(fs::path(disk_path) / path); + fs::create_directories(disk_path / path); } void DiskLocal::clearDirectory(const String & path) { - for (auto & entry : fs::directory_iterator(fs::path(disk_path) / path)) + for (auto & entry : fs::directory_iterator(disk_path / path)) fs::remove(entry.path()); } void DiskLocal::moveDirectory(const String & from_path, const String & to_path) { - fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); + fs::rename(disk_path / from_path, disk_path / to_path); } DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) @@ -199,18 +199,18 @@ DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) void DiskLocal::moveFile(const String & from_path, const String & to_path) { - fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); + fs::rename(disk_path / from_path, disk_path / to_path); } void DiskLocal::replaceFile(const String & from_path, const String & to_path) { - fs::path from_file = fs::path(disk_path) / from_path; - fs::path to_file = fs::path(disk_path) / to_path; + fs::path from_file = disk_path / from_path; + fs::path to_file = disk_path / to_path; if (fs::exists(to_file)) { fs::path tmp_file(to_file.string() + ".old"); fs::rename(to_file, tmp_file); - fs::rename(from_file, fs::path(disk_path) / to_path); + fs::rename(from_file, disk_path / to_path); fs::remove(tmp_file); } else @@ -223,35 +223,35 @@ std::unique_ptr DiskLocal::readFile( const String & path, size_t buf_size, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, MMappedFileCache * mmap_cache) const { - return createReadBufferFromFileBase(disk_path + path, estimated_size, aio_threshold, mmap_threshold, mmap_cache, buf_size); + return createReadBufferFromFileBase(disk_path / path, estimated_size, aio_threshold, mmap_threshold, mmap_cache, buf_size); } std::unique_ptr DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode) { int flags = (mode == WriteMode::Append) ? (O_APPEND | O_CREAT | O_WRONLY) : -1; - return std::make_unique(disk_path + path, buf_size, flags); + return std::make_unique(disk_path / path, buf_size, flags); } void DiskLocal::removeFile(const String & path) { - auto fs_path = disk_path + path; + auto fs_path = disk_path / path; if (0 != unlink(fs_path.c_str())) - throwFromErrnoWithPath("Cannot unlink file " + fs_path, fs_path, ErrorCodes::CANNOT_UNLINK); + throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK); } void DiskLocal::removeFileIfExists(const String & path) { - auto fs_path = disk_path + path; + auto fs_path = disk_path / path; if (0 != unlink(fs_path.c_str()) && errno != ENOENT) - throwFromErrnoWithPath("Cannot unlink file " + fs_path, fs_path, ErrorCodes::CANNOT_UNLINK); + throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK); } void DiskLocal::removeDirectory(const String & path) { - auto fs_path = disk_path + path; + auto fs_path = disk_path / path; if (0 != rmdir(fs_path.c_str())) - throwFromErrnoWithPath("Cannot rmdir " + fs_path, fs_path, ErrorCodes::CANNOT_RMDIR); + throwFromErrnoWithPath("Cannot rmdir " + fs_path.string(), fs_path, ErrorCodes::CANNOT_RMDIR); } void DiskLocal::removeRecursive(const String & path) @@ -261,39 +261,41 @@ void DiskLocal::removeRecursive(const String & path) void DiskLocal::listFiles(const String & path, std::vector & file_names) { - Poco::File(disk_path + path).list(file_names); + file_names.clear(); + for (auto & entry : fs::directory_iterator(disk_path / path)) + file_names.emplace_back(entry.path().filename()); } void DiskLocal::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - Poco::File(disk_path + path).setLastModified(timestamp); + Poco::File(disk_path / path).setLastModified(timestamp); } Poco::Timestamp DiskLocal::getLastModified(const String & path) { - return Poco::File(disk_path + path).getLastModified(); + return Poco::File(disk_path / path).getLastModified(); } void DiskLocal::createHardLink(const String & src_path, const String & dst_path) { - DB::createHardLink(disk_path + src_path, disk_path + dst_path); + DB::createHardLink(disk_path / src_path, disk_path / dst_path); } void DiskLocal::truncateFile(const String & path, size_t size) { - int res = truncate((disk_path + path).c_str(), size); + int res = truncate((disk_path / path).string().data(), size); if (-1 == res) throwFromErrnoWithPath("Cannot truncate file " + path, path, ErrorCodes::CANNOT_TRUNCATE_FILE); } void DiskLocal::createFile(const String & path) { - Poco::File(disk_path + path).createFile(); + Poco::File(disk_path / path).createFile(); } void DiskLocal::setReadOnly(const String & path) { - Poco::File(disk_path + path).setReadOnly(true); + Poco::File(disk_path / path).setReadOnly(true); } bool inline isSameDiskType(const IDisk & one, const IDisk & another) @@ -304,14 +306,14 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - Poco::File(disk_path + from_path).copyTo(to_disk->getPath() + to_path); /// Use more optimal way. + Poco::File(disk_path / from_path).copyTo(to_disk->getPath() + to_path); /// Use more optimal way. else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } SyncGuardPtr DiskLocal::getDirectorySyncGuard(const String & path) const { - return std::make_unique(disk_path + path); + return std::make_unique(disk_path / path); } DiskPtr DiskLocalReservation::getDisk(size_t i) const diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 567ca24eb50..8cdadd095c6 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -8,6 +8,9 @@ #include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -24,15 +27,15 @@ public: friend class DiskLocalReservation; DiskLocal(const String & name_, const String & path_, UInt64 keep_free_space_bytes_) - : name(name_), disk_path(path_), keep_free_space_bytes(keep_free_space_bytes_) + : name(name_), disk_path(path_), disk_path_str(path_), keep_free_space_bytes(keep_free_space_bytes_) { - if (disk_path.back() != '/') - throw Exception("Disk path must ends with '/', but '" + disk_path + "' doesn't.", ErrorCodes::LOGICAL_ERROR); + if (disk_path_str.back() != '/') + throw Exception("Disk path must ends with '/', but '" + disk_path_str + "' doesn't.", ErrorCodes::LOGICAL_ERROR); } const String & getName() const override { return name; } - const String & getPath() const override { return disk_path; } + const String & getPath() const override { return disk_path_str; } ReservationPtr reserve(UInt64 bytes) override; @@ -109,7 +112,8 @@ private: private: const String name; - const String disk_path; + const fs::path disk_path; + const String disk_path_str; const UInt64 keep_free_space_bytes; UInt64 reserved_bytes = 0; diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 68257ec4948..0e93d469df7 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -6,8 +6,9 @@ #include #include #include -#include +#include +namespace fs = std::filesystem; namespace DB { @@ -24,7 +25,7 @@ namespace ErrorCodes class DiskMemoryDirectoryIterator final : public IDiskDirectoryIterator { public: - explicit DiskMemoryDirectoryIterator(std::vector && dir_file_paths_) + explicit DiskMemoryDirectoryIterator(std::vector && dir_file_paths_) : dir_file_paths(std::move(dir_file_paths_)), iter(dir_file_paths.begin()) { } @@ -33,13 +34,13 @@ public: bool isValid() const override { return iter != dir_file_paths.end(); } - String path() const override { return (*iter).toString(); } + String path() const override { return iter->string(); } - String name() const override { return (*iter).getFileName(); } + String name() const override { return iter->filename(); } private: - std::vector dir_file_paths; - std::vector::iterator iter; + std::vector dir_file_paths; + std::vector::iterator iter; }; @@ -268,7 +269,7 @@ DiskDirectoryIteratorPtr DiskMemory::iterateDirectory(const String & path) if (!path.empty() && files.find(path) == files.end()) throw Exception("Directory '" + path + "' does not exist", ErrorCodes::DIRECTORY_DOESNT_EXIST); - std::vector dir_file_paths; + std::vector dir_file_paths; for (const auto & file : files) if (parentPath(file.first) == path) dir_file_paths.emplace_back(file.first); From c609119daf710f1ccfebb0ebd26c109294d579f1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 18:13:19 +0300 Subject: [PATCH 021/154] Update Disks/* --- src/Disks/DiskLocal.cpp | 32 ++++++++++++++++++++++---------- src/Disks/DiskLocal.h | 5 ----- src/Disks/DiskMemory.cpp | 2 -- src/Disks/IDisk.cpp | 7 +++---- src/Disks/IDisk.h | 10 ++++++---- 5 files changed, 31 insertions(+), 25 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 3e109ad8da4..aa03cd02a56 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -8,10 +8,9 @@ #include #include -#include +#include #include -namespace fs = std::filesystem; namespace DB { @@ -25,6 +24,7 @@ namespace ErrorCodes extern const int CANNOT_TRUNCATE_FILE; extern const int CANNOT_UNLINK; extern const int CANNOT_RMDIR; + extern const int CANNOT_OPEN_FILE; } std::mutex DiskLocal::reservation_mutex; @@ -268,12 +268,14 @@ void DiskLocal::listFiles(const String & path, std::vector & file_names) void DiskLocal::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - Poco::File(disk_path / path).setLastModified(timestamp); + fs::last_write_time(disk_path / path, static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); } Poco::Timestamp DiskLocal::getLastModified(const String & path) { - return Poco::File(disk_path / path).getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(disk_path / path); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } void DiskLocal::createHardLink(const String & src_path, const String & dst_path) @@ -290,12 +292,16 @@ void DiskLocal::truncateFile(const String & path, size_t size) void DiskLocal::createFile(const String & path) { - Poco::File(disk_path / path).createFile(); + FILE * file = fopen((disk_path / path).string().data(), "a+"); + if (file == nullptr) + throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Cannot create file {}", path); } void DiskLocal::setReadOnly(const String & path) { - Poco::File(disk_path / path).setReadOnly(true); + fs::permissions(disk_path / path, + fs::perms::owner_read | fs::perms::group_read | fs::perms::others_read, + fs::perm_options::remove); /// bitwise AND } bool inline isSameDiskType(const IDisk & one, const IDisk & another) @@ -306,7 +312,7 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - Poco::File(disk_path / from_path).copyTo(to_disk->getPath() + to_path); /// Use more optimal way. + fs::copy_file(disk_path / from_path, to_disk->getPath() + to_path); else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } @@ -384,10 +390,16 @@ void registerDiskLocal(DiskFactory & factory) throw Exception("Disk path must end with /. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); } - if (Poco::File disk{path}; !disk.canRead() || !disk.canWrite()) - { + fs::path disk(path); + fs::perms p = fs::status(disk).permissions(); + bool is_readable = (p & fs::perms::owner_read) != fs::perms::none + | (p & fs::perms::group_read) != fs::perms::none + | (p & fs::perms::others_read) != fs::perms::none; + bool is_writable = (p & fs::perms::owner_write) != fs::perms::none + | (p & fs::perms::group_write) != fs::perms::none + | (p & fs::perms::others_write) != fs::perms::none; + if (!is_readable || !is_writable) throw Exception("There is no RW access to the disk " + name + " (" + path + ")", ErrorCodes::PATH_ACCESS_DENIED); - } bool has_space_ratio = config.has(config_prefix + ".keep_free_space_ratio"); diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 8cdadd095c6..91835707725 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -6,11 +6,6 @@ #include #include -#include -#include -#include - -namespace fs = std::filesystem; namespace DB { diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 0e93d469df7..423dba6bed6 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -6,9 +6,7 @@ #include #include #include -#include -namespace fs = std::filesystem; namespace DB { diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index ee7f57af771..8c69398d3ad 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -48,10 +48,9 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p } else { - Poco::Path path(from_path); - const String & dir_name = path.directory(path.depth() - 1); - const String dest = to_path + dir_name + "/"; - to_disk.createDirectories(dest); + const String & dir_name = directoryPath(from_path); + fs::path dest(fs::path(to_path) / dir_name); + fs::create_directories(dest); for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next()) asyncCopy(from_disk, it->path(), to_disk, dest, exec, results); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 726145cb5d2..778b3dc9bf3 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -11,9 +11,10 @@ #include #include #include -#include #include +#include +namespace fs = std::filesystem; namespace CurrentMetrics { @@ -288,19 +289,20 @@ inline String fullPath(const DiskPtr & disk, const String & path) /// Return parent path for the specified path. inline String parentPath(const String & path) { - return Poco::Path(path).parent().toString(); + auto fs_path = fs::path(path).parent_path() / ""; + return fs_path.string(); } /// Return file name for the specified path. inline String fileName(const String & path) { - return Poco::Path(path).getFileName(); + return fs::path(path).filename(); } /// Return directory path for the specified path. inline String directoryPath(const String & path) { - return Poco::Path(path).setFileName("").toString(); + return fs::is_directory(path) ? path : fs::path(path).parent_path().string(); } } From 6867ea2c73b0aabcc425be1e7e5ec533b989eefd Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 19:14:04 +0300 Subject: [PATCH 022/154] Temporary fix --- src/Disks/IDisk.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 778b3dc9bf3..34936373d67 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -13,6 +13,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -289,8 +290,7 @@ inline String fullPath(const DiskPtr & disk, const String & path) /// Return parent path for the specified path. inline String parentPath(const String & path) { - auto fs_path = fs::path(path).parent_path() / ""; - return fs_path.string(); + return Poco::Path(path).parent().toString(); } /// Return file name for the specified path. From 1b75f12fcb93e5e2dc4b0ee2df73ff3cbc0e3dd4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 19:48:42 +0300 Subject: [PATCH 023/154] Update Databases/* --- src/Databases/DatabaseAtomic.cpp | 5 ++- src/Databases/DatabaseOrdinary.cpp | 8 ++-- .../PostgreSQL/DatabasePostgreSQL.cpp | 40 +++++++++---------- 3 files changed, 25 insertions(+), 28 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 9b9ea572c3d..ad6af3ff8ab 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -12,6 +12,9 @@ #include #include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -42,7 +45,7 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, c , db_uuid(uuid) { assert(db_uuid != UUIDHelpers::Nil); - Poco::File(path_to_table_symlinks).createDirectories(); + fs::create_directories(path_to_table_symlinks); tryCreateMetadataSymlink(); } diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index cea31d88d57..6b91649c3b2 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include #include @@ -112,8 +111,7 @@ void DatabaseOrdinary::loadStoredObjects(ContextPtr local_context, bool has_forc auto * create_query = ast->as(); create_query->database = database_name; - auto detached_permanently_flag = Poco::File(full_path.string() + detached_suffix); - if (detached_permanently_flag.exists()) + if (fs::exists(full_path.string() + detached_suffix)) { /// FIXME: even if we don't load the table we can still mark the uuid of it as taken. /// if (create_query->uuid != UUIDHelpers::Nil) @@ -282,11 +280,11 @@ void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_ try { /// rename atomically replaces the old file with the new one. - Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path); + fs::rename(table_metadata_tmp_path, table_metadata_path); } catch (...) { - Poco::File(table_metadata_tmp_path).remove(); + fs::remove(table_metadata_tmp_path); throw; } } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index e12d7f975ce..d0864d619b1 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -12,12 +12,12 @@ #include #include #include -#include -#include #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -29,6 +29,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int TABLE_IS_DROPPED; extern const int TABLE_ALREADY_EXISTS; + extern const int CANNOT_OPEN_FILE; } static const auto suffix = ".removed"; @@ -205,9 +206,9 @@ void DatabasePostgreSQL::attachTable(const String & table_name, const StoragePtr detached_or_dropped.erase(table_name); - Poco::File table_marked_as_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); - if (table_marked_as_removed.exists()) - table_marked_as_removed.remove(); + fs::path table_marked_as_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); + if (fs::exists(table_marked_as_removed)) + fs::remove(table_marked_as_removed); } @@ -252,16 +253,11 @@ void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool / if (detached_or_dropped.count(table_name)) throw Exception(fmt::format("Table {}.{} is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED); - Poco::File mark_table_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); + fs::path mark_table_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); - try - { - mark_table_removed.createFile(); - } - catch (...) - { - throw; - } + FILE * file = fopen(mark_table_removed.string().data(), "a+"); + if (file == nullptr) + throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Cannot create file {}", mark_table_removed.string()); if (cache_tables) cached_tables.erase(table_name); @@ -272,7 +268,7 @@ void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool / void DatabasePostgreSQL::drop(ContextPtr /*context*/) { - Poco::File(getMetadataPath()).remove(true); + fs::remove_all(getMetadataPath()); } @@ -280,14 +276,14 @@ void DatabasePostgreSQL::loadStoredObjects(ContextPtr /* context */, bool, bool { { std::lock_guard lock{mutex}; - Poco::DirectoryIterator iterator(getMetadataPath()); + fs::directory_iterator iter(getMetadataPath()); /// Check for previously dropped tables - for (Poco::DirectoryIterator end; iterator != end; ++iterator) + for (fs::directory_iterator end; iter != end; ++iter) { - if (iterator->isFile() && endsWith(iterator.name(), suffix)) + if (fs::is_regular_file(iter->path()) && endsWith(iter->path().filename(), suffix)) { - const auto & file_name = iterator.name(); + const auto & file_name = iter->path().filename().string(); const auto & table_name = unescapeForFileName(file_name.substr(0, file_name.size() - strlen(suffix))); detached_or_dropped.emplace(table_name); } @@ -321,9 +317,9 @@ void DatabasePostgreSQL::removeOutdatedTables() { auto table_name = *iter; iter = detached_or_dropped.erase(iter); - Poco::File table_marked_as_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); - if (table_marked_as_removed.exists()) - table_marked_as_removed.remove(); + fs::path table_marked_as_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); + if (fs::exists(table_marked_as_removed)) + fs::remove(table_marked_as_removed); } else ++iter; From 6e0cfdaeb569241b7a719979abae9085b7e02507 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 20:20:53 +0300 Subject: [PATCH 024/154] Update Databases/* part 2 --- src/Databases/DatabaseAtomic.cpp | 38 +++++++++++----------- src/Databases/DatabaseFactory.cpp | 12 +++---- src/Databases/DatabaseOnDisk.cpp | 53 ++++++++++++++++--------------- 3 files changed, 52 insertions(+), 51 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ad6af3ff8ab..12d1fa48670 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -1,7 +1,5 @@ #include #include -#include -#include #include #include #include @@ -76,14 +74,14 @@ void DatabaseAtomic::drop(ContextPtr) assert(tables.empty()); try { - Poco::File(path_to_metadata_symlink).remove(); - Poco::File(path_to_table_symlinks).remove(true); + fs::remove(path_to_metadata_symlink); + fs::remove_all(path_to_table_symlinks); } catch (...) { LOG_WARNING(log, getCurrentExceptionMessage(true)); } - Poco::File(getMetadataPath()).remove(true); + fs::remove_all(getMetadataPath()); } void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) @@ -135,8 +133,8 @@ void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_na /// (it's more likely to lost connection, than to fail before applying local changes). /// TODO better detection and recovery - Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); /// Mark table as dropped - DatabaseOrdinary::detachTableUnlocked(table_name, lock); /// Should never throw + fs::rename(table_metadata_path, table_metadata_path_drop); /// Mark table as dropped + DatabaseOrdinary::detachTableUnlocked(table_name, lock); /// Should never throw table_name_to_path.erase(table_name); } @@ -319,7 +317,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora } catch (...) { - Poco::File(table_metadata_tmp_path).remove(); + fs::remove(table_metadata_tmp_path); if (locked_uuid) DatabaseCatalog::instance().removeUUIDMappingFinally(query.uuid); throw; @@ -423,7 +421,7 @@ void DatabaseAtomic::loadStoredObjects(ContextPtr local_context, bool has_force_ { /// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken if (has_force_restore_data_flag) - Poco::File(path_to_table_symlinks).remove(true); + fs::remove_all(path_to_table_symlinks); DatabaseOrdinary::loadStoredObjects(local_context, has_force_restore_data_flag, force_attach); @@ -435,7 +433,7 @@ void DatabaseAtomic::loadStoredObjects(ContextPtr local_context, bool has_force_ table_names = table_name_to_path; } - Poco::File(path_to_table_symlinks).createDirectories(); + fs::create_directories(path_to_table_symlinks); for (const auto & table : table_names) tryCreateSymlink(table.first, table.second, true); } @@ -446,9 +444,9 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & try { String link = path_to_table_symlinks + escapeForFileName(table_name); - Poco::File data = Poco::Path(getContext()->getPath()).makeAbsolute().toString() + actual_data_path; - if (!if_data_path_exist || data.exists()) - data.linkTo(link, Poco::File::LINK_SYMBOLIC); + fs::path data = fs::absolute(getContext()->getPath()).string() + actual_data_path; + if (!if_data_path_exist || fs::exists(data)) + fs::create_symlink(data, link); } catch (...) { @@ -461,7 +459,7 @@ void DatabaseAtomic::tryRemoveSymlink(const String & table_name) try { String path = path_to_table_symlinks + escapeForFileName(table_name); - Poco::File{path}.remove(); + fs::remove(path); } catch (...) { @@ -474,17 +472,17 @@ void DatabaseAtomic::tryCreateMetadataSymlink() /// Symlinks in data/db_name/ directory and metadata/db_name/ are not used by ClickHouse, /// it's needed only for convenient introspection. assert(path_to_metadata_symlink != metadata_path); - Poco::File metadata_symlink(path_to_metadata_symlink); - if (metadata_symlink.exists()) + fs::path metadata_symlink(path_to_metadata_symlink); + if (fs::exists(metadata_symlink)) { - if (!metadata_symlink.isLink()) + if (!fs::is_symlink(metadata_symlink)) throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Directory {} exists", path_to_metadata_symlink); } else { try { - Poco::File{metadata_path}.linkTo(path_to_metadata_symlink, Poco::File::LINK_SYMBOLIC); + fs::create_symlink(metadata_path, path_to_metadata_symlink); } catch (...) { @@ -498,7 +496,7 @@ void DatabaseAtomic::renameDatabase(const String & new_name) /// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard try { - Poco::File(path_to_metadata_symlink).remove(); + fs::remove(path_to_metadata_symlink); } catch (...) { @@ -529,7 +527,7 @@ void DatabaseAtomic::renameDatabase(const String & new_name) path_to_table_symlinks = getContext()->getPath() + "data/" + new_name_escaped + "/"; } - Poco::File(old_path_to_table_symlinks).renameTo(path_to_table_symlinks); + fs::rename(old_path_to_table_symlinks, path_to_table_symlinks); tryCreateMetadataSymlink(); } diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 9d09ac731d2..93511a99879 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -40,6 +41,8 @@ #include #endif +namespace fs = std::filesystem; + namespace DB { @@ -62,7 +65,7 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m /// Before 20.7 it's possible that .sql metadata file does not exist for some old database. /// In this case Ordinary database is created on server startup if the corresponding metadata directory exists. /// So we should remove metadata directory if database creation failed. - created = Poco::File(metadata_path).createDirectory(); + created = fs::create_directory(metadata_path); DatabasePtr impl = getImpl(create, metadata_path, context); @@ -74,11 +77,8 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m } catch (...) { - Poco::File metadata_dir(metadata_path); - - if (created && metadata_dir.exists()) - metadata_dir.remove(true); - + if (created && fs::exists(metadata_path)) + fs::remove_all(metadata_path); throw; } } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 2ad961ccce4..44ec5d27250 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -21,7 +21,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -199,8 +201,8 @@ DatabaseOnDisk::DatabaseOnDisk( , metadata_path(metadata_path_) , data_path(data_path_) { - Poco::File(local_context->getPath() + data_path).createDirectories(); - Poco::File(metadata_path).createDirectories(); + fs::create_directories(local_context->getPath() + data_path); + fs::create_directories(metadata_path); } @@ -243,7 +245,7 @@ void DatabaseOnDisk::createTable( if (!create.attach) checkMetadataFilenameAvailability(table_name); - if (create.attach && Poco::File(table_metadata_path).exists()) + if (create.attach && fs::exists(table_metadata_path)) { ASTPtr ast_detached = parseQueryFromMetadata(log, local_context, table_metadata_path); auto & create_detached = ast_detached->as(); @@ -283,10 +285,10 @@ void DatabaseOnDisk::removeDetachedPermanentlyFlag(ContextPtr, const String & ta { try { - auto detached_permanently_flag = Poco::File(table_metadata_path + detached_suffix); + fs::path detached_permanently_flag(table_metadata_path + detached_suffix); - if (detached_permanently_flag.exists()) - detached_permanently_flag.remove(); + if (fs::exists(detached_permanently_flag)) + fs::remove(detached_permanently_flag); } catch (Exception & e) { @@ -306,11 +308,11 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora /// If it was ATTACH query and file with table metadata already exist /// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one. - Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path); + fs::rename(table_metadata_tmp_path, table_metadata_path); } catch (...) { - Poco::File(table_metadata_tmp_path).remove(); + fs::remove(table_metadata_tmp_path); throw; } } @@ -348,25 +350,25 @@ void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_na bool renamed = false; try { - Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); + fs::rename(table_metadata_path, table_metadata_path_drop); renamed = true; table->drop(); table->is_dropped = true; - Poco::File table_data_dir{local_context->getPath() + table_data_path_relative}; - if (table_data_dir.exists()) - table_data_dir.remove(true); + fs::path table_data_dir(local_context->getPath() + table_data_path_relative); + if (fs::exists(table_data_dir)) + fs::remove_all(table_data_dir); } catch (...) { LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__)); attachTable(table_name, table, table_data_path_relative); if (renamed) - Poco::File(table_metadata_path_drop).renameTo(table_metadata_path); + fs::rename(table_metadata_path_drop, table_metadata_path); throw; } - Poco::File(table_metadata_path_drop).remove(); + fs::remove(table_metadata_path_drop); } void DatabaseOnDisk::checkMetadataFilenameAvailability(const String & to_table_name) const @@ -379,11 +381,11 @@ void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to { String table_metadata_path = getObjectMetadataPath(to_table_name); - if (Poco::File(table_metadata_path).exists()) + if (fs::exists(table_metadata_path)) { - auto detached_permanently_flag = Poco::File(table_metadata_path + detached_suffix); + fs::path detached_permanently_flag(table_metadata_path + detached_suffix); - if (detached_permanently_flag.exists()) + if (fs::exists(detached_permanently_flag)) throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached permanently)", backQuote(database_name), backQuote(to_table_name)); else throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached)", backQuote(database_name), backQuote(to_table_name)); @@ -461,7 +463,7 @@ void DatabaseOnDisk::renameTable( /// Now table data are moved to new database, so we must add metadata and attach table to new database to_database.createTable(local_context, to_table_name, table, attach_query); - Poco::File(table_metadata_path).remove(); + fs::remove(table_metadata_path); if (from_atomic_to_ordinary) { @@ -526,8 +528,8 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const void DatabaseOnDisk::drop(ContextPtr local_context) { assert(tables.empty()); - Poco::File(local_context->getPath() + getDataPath()).remove(false); - Poco::File(getMetadataPath()).remove(false); + fs::remove(local_context->getPath() + getDataPath()); + fs::remove(getMetadataPath()); } String DatabaseOnDisk::getObjectMetadataPath(const String & object_name) const @@ -553,16 +555,17 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat assert(getUUID() == UUIDHelpers::Nil); static const char * tmp_drop_ext = ".sql.tmp_drop"; const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext)); - if (Poco::File(local_context->getPath() + getDataPath() + '/' + object_name).exists()) + + if (fs::exists(local_context->getPath() + getDataPath() + '/' + object_name)) { - Poco::File(getMetadataPath() + file_name).renameTo(getMetadataPath() + object_name + ".sql"); + fs::rename(getMetadataPath() + file_name, getMetadataPath() + object_name + ".sql"); LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name)); process_metadata_file(object_name + ".sql"); } else { LOG_INFO(log, "Removing file {}", getMetadataPath() + file_name); - Poco::File(getMetadataPath() + file_name).remove(); + fs::remove(getMetadataPath() + file_name); } }; @@ -593,7 +596,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat { /// There are files .sql.tmp - delete LOG_INFO(log, "Removing file {}", dir_it->path()); - Poco::File(dir_it->path()).remove(); + fs::remove(dir_it->path()); } else if (endsWith(dir_it.name(), ".sql")) { @@ -649,7 +652,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata( { if (logger) LOG_ERROR(logger, "File {} is empty. Removing.", metadata_file_path); - Poco::File(metadata_file_path).remove(); + fs::remove(metadata_file_path); return nullptr; } From e717b4c5c76e982d7ea3d3129d8f1ea0dc4dad3c Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 22:38:07 +0300 Subject: [PATCH 025/154] Update databases/* --- src/Databases/DatabaseLazy.cpp | 7 ++++--- src/Databases/DatabaseMemory.cpp | 8 ++++---- .../MySQL/DatabaseConnectionMySQL.cpp | 20 +++++++++---------- .../MySQL/DatabaseMaterializeMySQL.cpp | 10 ++++++---- src/Databases/MySQL/MaterializeMetadata.cpp | 11 +++++----- 5 files changed, 30 insertions(+), 26 deletions(-) diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index d498cb96062..416a94db929 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -12,8 +12,9 @@ #include #include #include -#include +#include +namespace fs = std::filesystem; namespace DB { @@ -43,8 +44,8 @@ void DatabaseLazy::loadStoredObjects( { const std::string table_name = file_name.substr(0, file_name.size() - 4); - auto detached_permanently_flag = Poco::File(getMetadataPath() + "/" + file_name + detached_suffix); - if (detached_permanently_flag.exists()) + fs::path detached_permanently_flag = fs::path(getMetadataPath()) / (file_name + detached_suffix); + if (fs::exists(detached_permanently_flag)) { LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name)); return; diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index f21a145df55..c0af027e027 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -4,9 +4,9 @@ #include #include #include -#include #include +namespace fs = std::filesystem; namespace DB { @@ -42,9 +42,9 @@ void DatabaseMemory::dropTable( try { table->drop(); - Poco::File table_data_dir{getTableDataPath(table_name)}; - if (table_data_dir.exists()) - table_data_dir.remove(true); + fs::path table_data_dir{getTableDataPath(table_name)}; + if (fs::exists(table_data_dir)) + fs::remove_all(table_data_dir); } catch (...) { diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 4e63bed9c6d..2b6c6d243a7 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -23,10 +23,10 @@ # include # include # include - -# include +# include # include +namespace fs = std::filesystem; namespace DB { @@ -314,7 +314,7 @@ void DatabaseConnectionMySQL::shutdown() void DatabaseConnectionMySQL::drop(ContextPtr /*context*/) { - Poco::File(getMetadataPath()).remove(true); + fs::remove_all(getMetadataPath()); } void DatabaseConnectionMySQL::cleanOutdatedTables() @@ -360,10 +360,10 @@ void DatabaseConnectionMySQL::attachTable(const String & table_name, const Stora local_tables_cache[table_name].second = storage; remove_or_detach_tables.erase(table_name); - Poco::File remove_flag(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); + fs::path remove_flag = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); - if (remove_flag.exists()) - remove_flag.remove(); + if (fs::exists(remove_flag)) + fs::remove(remove_flag); } StoragePtr DatabaseConnectionMySQL::detachTable(const String & table_name) @@ -391,13 +391,13 @@ void DatabaseConnectionMySQL::loadStoredObjects(ContextPtr, bool, bool /*force_a { std::lock_guard lock{mutex}; - Poco::DirectoryIterator iterator(getMetadataPath()); + fs::directory_iterator iter(getMetadataPath()); - for (Poco::DirectoryIterator end; iterator != end; ++iterator) + for (fs::directory_iterator end; iter != end; ++iter) { - if (iterator->isFile() && endsWith(iterator.name(), suffix)) + if (fs::is_regular_file(iter->path()) && endsWith(iter->path().filename(), suffix)) { - const auto & filename = iterator.name(); + const auto & filename = iter->path().filename().string(); const auto & table_name = unescapeForFileName(filename.substr(0, filename.size() - strlen(suffix))); remove_or_detach_tables.emplace(table_name); } diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 62a66b22c93..e4470f9371f 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -13,9 +13,11 @@ # include # include # include -# include # include # include +# include + +namespace fs = std::filesystem; namespace DB { @@ -158,10 +160,10 @@ template void DatabaseMaterializeMySQL::drop(ContextPtr context_) { /// Remove metadata info - Poco::File metadata(Base::getMetadataPath() + "/.metadata"); + fs::path metadata(Base::getMetadataPath() + "/.metadata"); - if (metadata.exists()) - metadata.remove(false); + if (fs::exists(metadata)) + fs::remove(metadata); Base::drop(context_); } diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index c389ab5a1b0..4fd99ca3438 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -8,11 +8,13 @@ #include #include #include -#include #include #include #include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -193,12 +195,11 @@ void commitMetadata(const std::function & function, const String & persi try { function(); - - Poco::File(persistent_tmp_path).renameTo(persistent_path); + fs::rename(persistent_tmp_path, persistent_path); } catch (...) { - Poco::File(persistent_tmp_path).remove(); + fs::remove(persistent_tmp_path); throw; } } @@ -231,7 +232,7 @@ void MaterializeMetadata::transaction(const MySQLReplication::Position & positio MaterializeMetadata::MaterializeMetadata(const String & path_, const Settings & settings_) : persistent_path(path_), settings(settings_) { - if (Poco::File(persistent_path).exists()) + if (fs::exists(persistent_path)) { ReadBufferFromFile in(persistent_path, DBMS_DEFAULT_BUFFER_SIZE); assertString("Version:\t" + toString(meta_version), in); From 549b5f8a0cd037bc9032d976921d64500bfe52e6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 23:16:35 +0300 Subject: [PATCH 026/154] Update Common/* --- src/Common/Config/ConfigProcessor.cpp | 2 -- src/Common/Config/configReadClient.cpp | 10 ++++++---- src/Common/FileUpdatesTracker.h | 8 +++++--- src/Common/StatusFile.cpp | 5 +++-- src/Common/filesystemHelpers.cpp | 4 +++- src/Common/renameat2.cpp | 8 +++++--- 6 files changed, 22 insertions(+), 15 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 5399826fe20..149de3cfcbf 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -20,10 +20,8 @@ #include #include - #define PREPROCESSED_SUFFIX "-preprocessed" - namespace fs = std::filesystem; using namespace Poco::XML; diff --git a/src/Common/Config/configReadClient.cpp b/src/Common/Config/configReadClient.cpp index 61d8e507c05..8ce2a8b03e9 100644 --- a/src/Common/Config/configReadClient.cpp +++ b/src/Common/Config/configReadClient.cpp @@ -1,8 +1,10 @@ #include "configReadClient.h" #include -#include #include "ConfigProcessor.h" +#include + +namespace fs = std::filesystem; namespace DB { @@ -11,11 +13,11 @@ bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::stri std::string config_path; if (config.has("config-file")) config_path = config.getString("config-file"); - else if (Poco::File("./clickhouse-client.xml").exists()) + else if (fs::exists("./clickhouse-client.xml")) config_path = "./clickhouse-client.xml"; - else if (!home_path.empty() && Poco::File(home_path + "/.clickhouse-client/config.xml").exists()) + else if (!home_path.empty() && fs::exists(home_path + "/.clickhouse-client/config.xml")) config_path = home_path + "/.clickhouse-client/config.xml"; - else if (Poco::File("/etc/clickhouse-client/config.xml").exists()) + else if (fs::exists("/etc/clickhouse-client/config.xml")) config_path = "/etc/clickhouse-client/config.xml"; if (!config_path.empty()) diff --git a/src/Common/FileUpdatesTracker.h b/src/Common/FileUpdatesTracker.h index 0914c75693c..471a62cf07a 100644 --- a/src/Common/FileUpdatesTracker.h +++ b/src/Common/FileUpdatesTracker.h @@ -1,10 +1,10 @@ #pragma once -#include #include - #include +#include +namespace fs = std::filesystem; class FileUpdatesTracker { @@ -31,6 +31,8 @@ public: private: Poco::Timestamp getLastModificationTime() const { - return Poco::File(path).getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(path); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } }; diff --git a/src/Common/StatusFile.cpp b/src/Common/StatusFile.cpp index b21454c9ed8..ceedf518608 100644 --- a/src/Common/StatusFile.cpp +++ b/src/Common/StatusFile.cpp @@ -4,7 +4,6 @@ #include #include -#include #include #include #include @@ -14,7 +13,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -45,7 +46,7 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_) : path(std::move(path_)), fill(std::move(fill_)) { /// If file already exists. NOTE Minor race condition. - if (Poco::File(path).exists()) + if (fs::exists(path)) { std::string contents; { diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 51a66fba3aa..bfcb75235c0 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -9,7 +9,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -48,7 +50,7 @@ bool enoughSpaceInDirectory(const std::string & path [[maybe_unused]], size_t da std::unique_ptr createTemporaryFile(const std::string & path) { - Poco::File(path).createDirectories(); + fs::create_directories(path); /// NOTE: std::make_shared cannot use protected constructors return std::make_unique(path); diff --git a/src/Common/renameat2.cpp b/src/Common/renameat2.cpp index a735a9d72d4..26d90427889 100644 --- a/src/Common/renameat2.cpp +++ b/src/Common/renameat2.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #if defined(linux) || defined(__linux) || defined(__linux__) #include @@ -10,6 +10,8 @@ #include #endif +namespace fs = std::filesystem; + namespace DB { @@ -93,9 +95,9 @@ static bool renameat2(const std::string &, const std::string &, int) static void renameNoReplaceFallback(const std::string & old_path, const std::string & new_path) { /// NOTE it's unsafe - if (Poco::File{new_path}.exists()) + if (fs::exists(new_path)) throw Exception("File " + new_path + " exists", ErrorCodes::FILE_ALREADY_EXISTS); - Poco::File{old_path}.renameTo(new_path); + fs::rename(old_path, new_path); } /// Do not use [[noreturn]] to avoid warnings like "code will never be executed" in other places From 378195a6d8990b8e62f5fa6f36d45111c5ae8653 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 23:35:44 +0300 Subject: [PATCH 027/154] Update Dictionaries/* --- src/Common/Config/ConfigProcessor.cpp | 19 +++++++++---------- .../GeodataProviders/HierarchiesProvider.cpp | 4 +++- .../GeodataProviders/NamesProvider.cpp | 6 ++++-- src/Dictionaries/FileDictionarySource.cpp | 8 +++++++- src/Dictionaries/LibraryDictionarySource.cpp | 7 ++++--- 5 files changed, 27 insertions(+), 17 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 149de3cfcbf..957934770e7 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -424,20 +424,19 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string & for (const std::string & merge_dir_name : merge_dirs) { - Poco::File merge_dir(merge_dir_name); - if (!merge_dir.exists() || !merge_dir.isDirectory()) + fs::path merge_dir(merge_dir_name); + if (!fs::exists(merge_dir) || !is_directory(merge_dir)) continue; - for (Poco::DirectoryIterator it(merge_dir_name); it != Poco::DirectoryIterator(); ++it) + for (fs::directory_iterator it(merge_dir_name); it != fs::directory_iterator(); ++it) { - Poco::File & file = *it; - Poco::Path path(file.path()); - std::string extension = path.getExtension(); - std::string base_name = path.getBaseName(); + fs::path path(it->path()); + std::string extension = path.extension(); + std::string base_name = path.stem(); // Skip non-config and temporary files - if (file.isFile() && (extension == "xml" || extension == "conf") && !startsWith(base_name, ".")) - files.push_back(file.path()); + if (fs::is_regular_file(path) && (extension == ".xml" || extension == ".conf") && !startsWith(base_name, ".")) + files.push_back(it->path()); } } @@ -512,7 +511,7 @@ XMLDocumentPtr ConfigProcessor::processConfig( else { std::string default_path = "/etc/metrika.xml"; - if (Poco::File(default_path).exists()) + if (fs::exists(default_path)) include_from_path = default_path; } if (!include_from_path.empty()) diff --git a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp index 3b8848ab19b..7257caa066f 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp @@ -5,7 +5,9 @@ #include #include #include "HierarchyFormatReader.h" +#include +namespace fs = std::filesystem; bool RegionsHierarchyDataSource::isModified() const { @@ -27,7 +29,7 @@ RegionsHierarchiesDataProvider::RegionsHierarchiesDataProvider(const std::string void RegionsHierarchiesDataProvider::discoverFilesWithCustomHierarchies() { - std::string basename = Poco::Path(path).getBaseName(); + std::string basename = fs::path(path).stem(); Poco::Path dir_path = Poco::Path(path).absolute().parent(); diff --git a/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp b/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp index f3e49545481..5f79fda070f 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp @@ -2,7 +2,9 @@ #include #include "NamesFormatReader.h" +#include +namespace fs = std::filesystem; bool LanguageRegionsNamesDataSource::isModified() const { @@ -11,7 +13,7 @@ bool LanguageRegionsNamesDataSource::isModified() const size_t LanguageRegionsNamesDataSource::estimateTotalSize() const { - return Poco::File(path).getSize(); + return fs::file_size(path); } ILanguageRegionsNamesReaderPtr LanguageRegionsNamesDataSource::createReader() @@ -39,7 +41,7 @@ RegionsNamesDataProvider::RegionsNamesDataProvider(const std::string & directory ILanguageRegionsNamesDataSourcePtr RegionsNamesDataProvider::getLanguageRegionsNamesSource(const std::string & language) const { const auto data_file = getDataFilePath(language); - if (Poco::File(data_file).exists()) + if (fs::exists(data_file)) return std::make_unique(data_file, language); else return {}; diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 378c6f11857..d4bcc97d563 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -12,6 +12,9 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" #include "DictionarySourceHelpers.h" +#include + +namespace fs = std::filesystem; namespace DB { @@ -80,9 +83,12 @@ std::string FileDictionarySource::toString() const Poco::Timestamp FileDictionarySource::getLastModification() const { - return Poco::File{filepath}.getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(filepath); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } + void registerDictionarySourceFile(DictionarySourceFactory & factory) { auto create_table_source = [=](const DictionaryStructure & dict_struct, diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index a971ba4b1be..ce580d6b803 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -13,7 +12,9 @@ #include "registerDictionaries.h" #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -49,8 +50,8 @@ LibraryDictionarySource::LibraryDictionarySource( throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "LibraryDictionarySource: Library path {} is not inside {}", path, dictionaries_lib_path); } - if (!Poco::File(path).exists()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "LibraryDictionarySource: Can't load library {}: file doesn't exist", Poco::File(path).path()); + if (!fs::exists(path)) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "LibraryDictionarySource: Can't load library {}: file doesn't exist", path); description.init(sample_block); bridge_helper = std::make_shared(context, description.sample_block, dictionary_id); From a2753a46c609ba77a1974c721b86434885b95309 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 May 2021 00:37:10 +0300 Subject: [PATCH 028/154] Adjust test --- .../queries/0_stateless/01601_detach_permanently.reference | 2 +- tests/queries/0_stateless/01601_detach_permanently.sql | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01601_detach_permanently.reference b/tests/queries/0_stateless/01601_detach_permanently.reference index 6683603c972..0232c1694e4 100644 --- a/tests/queries/0_stateless/01601_detach_permanently.reference +++ b/tests/queries/0_stateless/01601_detach_permanently.reference @@ -50,7 +50,7 @@ After database reattachement the table is still absent (it was detached permamen And we can not detach it permanently But we can attach it back And detach permanently again to check how database drop will behave -DROP database - Directory not empty error, but database deteched +DROP database - Directory not empty error, but database detached DROP database - now success ----------------------- database lazy tests diff --git a/tests/queries/0_stateless/01601_detach_permanently.sql b/tests/queries/0_stateless/01601_detach_permanently.sql index 0e21fd8f463..3af8ed573ef 100644 --- a/tests/queries/0_stateless/01601_detach_permanently.sql +++ b/tests/queries/0_stateless/01601_detach_permanently.sql @@ -128,8 +128,8 @@ ATTACH TABLE test1601_detach_permanently_ordinary.test_name_reuse; SELECT 'And detach permanently again to check how database drop will behave'; DETACH table test1601_detach_permanently_ordinary.test_name_reuse PERMANENTLY; -SELECT 'DROP database - Directory not empty error, but database deteched'; -DROP DATABASE test1601_detach_permanently_ordinary; -- { serverError 1000 } +SELECT 'DROP database - Directory not empty error, but database detached'; +DROP DATABASE test1601_detach_permanently_ordinary; -- { serverError 1001 } ATTACH DATABASE test1601_detach_permanently_ordinary; @@ -203,7 +203,7 @@ SELECT 'And detach permanently again to check how database drop will behave'; DETACH table test1601_detach_permanently_lazy.test_name_reuse PERMANENTLY; SELECT 'DROP database - Directory not empty error, but database deteched'; -DROP DATABASE test1601_detach_permanently_lazy; -- { serverError 1000 } +DROP DATABASE test1601_detach_permanently_lazy; -- { serverError 1001 } ATTACH DATABASE test1601_detach_permanently_lazy; From 2887ce1f69c430ea053a6b472ee9406b337b0f68 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 May 2021 01:55:14 +0300 Subject: [PATCH 029/154] Temporary rollback --- src/Dictionaries/FileDictionarySource.cpp | 1 - src/Disks/DiskLocal.cpp | 22 +++++++++------------- src/Disks/tests/gtest_disk.cpp | 7 +++++-- 3 files changed, 14 insertions(+), 16 deletions(-) diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index d4bcc97d563..eb1ecb8c723 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -12,7 +12,6 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" #include "DictionarySourceHelpers.h" -#include namespace fs = std::filesystem; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index aa03cd02a56..726b8181fc5 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -24,7 +25,6 @@ namespace ErrorCodes extern const int CANNOT_TRUNCATE_FILE; extern const int CANNOT_UNLINK; extern const int CANNOT_RMDIR; - extern const int CANNOT_OPEN_FILE; } std::mutex DiskLocal::reservation_mutex; @@ -292,16 +292,12 @@ void DiskLocal::truncateFile(const String & path, size_t size) void DiskLocal::createFile(const String & path) { - FILE * file = fopen((disk_path / path).string().data(), "a+"); - if (file == nullptr) - throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Cannot create file {}", path); + Poco::File(disk_path / path).createFile(); } void DiskLocal::setReadOnly(const String & path) { - fs::permissions(disk_path / path, - fs::perms::owner_read | fs::perms::group_read | fs::perms::others_read, - fs::perm_options::remove); /// bitwise AND + Poco::File(disk_path / path).setReadOnly(true); } bool inline isSameDiskType(const IDisk & one, const IDisk & another) @@ -392,12 +388,12 @@ void registerDiskLocal(DiskFactory & factory) fs::path disk(path); fs::perms p = fs::status(disk).permissions(); - bool is_readable = (p & fs::perms::owner_read) != fs::perms::none - | (p & fs::perms::group_read) != fs::perms::none - | (p & fs::perms::others_read) != fs::perms::none; - bool is_writable = (p & fs::perms::owner_write) != fs::perms::none - | (p & fs::perms::group_write) != fs::perms::none - | (p & fs::perms::others_write) != fs::perms::none; + bool is_readable = ((p & fs::perms::owner_read) != fs::perms::none) + | ((p & fs::perms::group_read) != fs::perms::none) + | ((p & fs::perms::others_read) != fs::perms::none); + bool is_writable = ((p & fs::perms::owner_write) != fs::perms::none) + | ((p & fs::perms::group_write) != fs::perms::none) + | ((p & fs::perms::others_write) != fs::perms::none); if (!is_readable || !is_writable) throw Exception("There is no RW access to the disk " + name + " (" + path + ")", ErrorCodes::PATH_ACCESS_DENIED); diff --git a/src/Disks/tests/gtest_disk.cpp b/src/Disks/tests/gtest_disk.cpp index 3b9dca63002..cffbcf57e76 100644 --- a/src/Disks/tests/gtest_disk.cpp +++ b/src/Disks/tests/gtest_disk.cpp @@ -3,6 +3,9 @@ #include #include #include "gtest_disk.h" +#include + +namespace fs = std::filesystem; #if !defined(__clang__) # pragma GCC diagnostic push @@ -21,7 +24,7 @@ DB::DiskPtr createDisk() template <> DB::DiskPtr createDisk() { - Poco::File("tmp/").createDirectory(); + fs::create_directory("tmp/"); return std::make_shared("local_disk", "tmp/", 0); } @@ -42,7 +45,7 @@ template <> void destroyDisk(DB::DiskPtr & disk) { disk.reset(); - Poco::File("tmp/").remove(true); + fs::remove_all("tmp/"); } From 32b291714109092f716067a815888d9b277ee0d8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 May 2021 17:25:38 +0300 Subject: [PATCH 030/154] Update s3/* --- src/Common/CounterInFile.h | 5 +- src/Databases/DatabaseAtomic.cpp | 1 - src/Disks/DiskLocal.cpp | 4 +- src/Disks/S3/DiskS3.cpp | 54 ++++++++++--------- .../gtest_cascade_and_memory_write_buffer.cpp | 5 +- 5 files changed, 35 insertions(+), 34 deletions(-) diff --git a/src/Common/CounterInFile.h b/src/Common/CounterInFile.h index 8cd4534d413..6326261234d 100644 --- a/src/Common/CounterInFile.h +++ b/src/Common/CounterInFile.h @@ -7,7 +7,6 @@ #include #include -#include #include #include @@ -59,7 +58,7 @@ public: Int64 res = -1; - bool file_doesnt_exists = !Poco::File(path).exists(); + bool file_doesnt_exists = !fs::exists(path); if (file_doesnt_exists && !create_if_need) { throw Poco::Exception("File " + path + " does not exist. " @@ -138,7 +137,7 @@ public: // Not thread-safe and not synchronized between processes. void fixIfBroken(UInt64 value) { - bool file_exists = Poco::File(path).exists(); + bool file_exists = fs::exists(path); int fd = ::open(path.c_str(), O_RDWR | O_CREAT | O_CLOEXEC, 0666); if (-1 == fd) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 12d1fa48670..fa4e2eed382 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -10,7 +10,6 @@ #include #include #include -#include namespace fs = std::filesystem; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 726b8181fc5..49ca4a32293 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -183,7 +183,7 @@ void DiskLocal::createDirectories(const String & path) void DiskLocal::clearDirectory(const String & path) { - for (auto & entry : fs::directory_iterator(disk_path / path)) + for (const auto & entry : fs::directory_iterator(disk_path / path)) fs::remove(entry.path()); } @@ -262,7 +262,7 @@ void DiskLocal::removeRecursive(const String & path) void DiskLocal::listFiles(const String & path, std::vector & file_names) { file_names.clear(); - for (auto & entry : fs::directory_iterator(disk_path / path)) + for (const auto & entry : fs::directory_iterator(disk_path / path)) file_names.emplace_back(entry.path().filename()); } diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 325e887073c..07e2d254632 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -443,20 +443,20 @@ public: void next() override { ++iter; } - bool isValid() const override { return iter != Poco::DirectoryIterator(); } + bool isValid() const override { return iter != fs::directory_iterator(); } String path() const override { - if (iter->isDirectory()) - return folder_path + iter.name() + '/'; + if (fs::is_directory(iter->path())) + return folder_path + iter->path().filename().string() + '/'; else - return folder_path + iter.name(); + return folder_path + iter->path().filename().string(); } - String name() const override { return iter.name(); } + String name() const override { return iter->path().filename(); } private: - Poco::DirectoryIterator iter; + fs::directory_iterator iter; String folder_path; }; @@ -590,17 +590,17 @@ ReservationPtr DiskS3::reserve(UInt64 bytes) bool DiskS3::exists(const String & path) const { - return Poco::File(metadata_path + path).exists(); + return fs::exists(fs::path(metadata_path) / path); } bool DiskS3::isFile(const String & path) const { - return Poco::File(metadata_path + path).isFile(); + return fs::is_regular_file(fs::path(metadata_path) / path); } bool DiskS3::isDirectory(const String & path) const { - return Poco::File(metadata_path + path).isDirectory(); + return fs::is_directory(fs::path(metadata_path) / path); } size_t DiskS3::getFileSize(const String & path) const @@ -611,12 +611,12 @@ size_t DiskS3::getFileSize(const String & path) const void DiskS3::createDirectory(const String & path) { - Poco::File(metadata_path + path).createDirectory(); + fs::create_directory(fs::path(metadata_path) / path); } void DiskS3::createDirectories(const String & path) { - Poco::File(metadata_path + path).createDirectories(); + fs::create_directories(fs::path(metadata_path) / path); } String DiskS3::getUniqueId(const String & path) const @@ -662,7 +662,7 @@ void DiskS3::moveFile(const String & from_path, const String & to_path, bool sen createFileOperationObject("rename", revision, object_metadata); } - Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path); + fs::rename(fs::path(metadata_path) / from_path, fs::path(metadata_path) / to_path); } void DiskS3::replaceFile(const String & from_path, const String & to_path) @@ -727,9 +727,9 @@ void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) { LOG_DEBUG(log, "Remove file by path: {}", backQuote(metadata_path + path)); - Poco::File file(metadata_path + path); + fs::path file = fs::path(metadata_path) / path; - if (!file.isFile()) + if (!is_regular_file(file)) throw Exception(ErrorCodes::CANNOT_DELETE_DIRECTORY, "Path '{}' is a directory", path); try @@ -739,7 +739,7 @@ void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) /// If there is no references - delete content from S3. if (metadata.ref_count == 0) { - file.remove(); + fs::remove(file); for (const auto & [s3_object_path, _] : metadata.s3_objects) keys.addKey(s3_root_path + s3_object_path); @@ -748,7 +748,7 @@ void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) { --metadata.ref_count; metadata.save(); - file.remove(); + fs::remove(file); } } catch (const Exception & e) @@ -762,7 +762,7 @@ void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) backQuote(path), e.nested() ? e.nested()->message() : e.message()); - file.remove(); + fs::remove(file); } else throw; @@ -773,8 +773,8 @@ void DiskS3::removeMetaRecursive(const String & path, AwsS3KeyKeeper & keys) { checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. - Poco::File file(metadata_path + path); - if (file.isFile()) + fs::path file = fs::path(metadata_path) / path; + if (fs::is_regular_file(file)) { removeMeta(path, keys); } @@ -782,7 +782,7 @@ void DiskS3::removeMetaRecursive(const String & path, AwsS3KeyKeeper & keys) { for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) removeMetaRecursive(it->path(), keys); - file.remove(); + fs::remove(file); } } @@ -810,7 +810,7 @@ void DiskS3::removeAws(const AwsS3KeyKeeper & keys) void DiskS3::removeFileIfExists(const String & path) { AwsS3KeyKeeper keys; - if (Poco::File(metadata_path + path).exists()) + if (fs::exists(fs::path(metadata_path) / path)) { removeMeta(path, keys); removeAws(keys); @@ -819,7 +819,7 @@ void DiskS3::removeFileIfExists(const String & path) void DiskS3::removeDirectory(const String & path) { - Poco::File(metadata_path + path).remove(); + fs::remove(fs::path(metadata_path) / path); } void DiskS3::removeSharedFile(const String & path, bool keep_s3) @@ -869,12 +869,14 @@ void DiskS3::listFiles(const String & path, std::vector & file_names) void DiskS3::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - Poco::File(metadata_path + path).setLastModified(timestamp); + fs::last_write_time(fs::path(metadata_path) / path, static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); } Poco::Timestamp DiskS3::getLastModified(const String & path) { - return Poco::File(metadata_path + path).getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(fs::path(metadata_path) / path); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } void DiskS3::createHardLink(const String & src_path, const String & dst_path) @@ -1306,8 +1308,8 @@ void DiskS3::restore() restoreFiles(information); restoreFileOperations(information); - Poco::File restore_file(metadata_path + RESTORE_FILE_NAME); - restore_file.remove(); + fs::path restore_file = fs::path(metadata_path) / RESTORE_FILE_NAME; + fs::remove(restore_file); saveSchemaVersion(RESTORABLE_SCHEMA_VERSION); diff --git a/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp b/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp index 4936307a5e3..d013077b7c2 100644 --- a/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp +++ b/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include @@ -9,7 +8,9 @@ #include #include #include +#include +namespace fs = std::filesystem; using namespace DB; @@ -236,7 +237,7 @@ try buf.reset(); reread_buf.reset(); - ASSERT_TRUE(!Poco::File(tmp_filename).exists()); + ASSERT_TRUE(!fs::exists(tmp_filename)); } } catch (...) From ad69fceea52d17db774cf2620ad9fdb2adad0e84 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 2 May 2021 19:59:18 +0300 Subject: [PATCH 031/154] Restore copyFile --- src/Disks/DiskLocal.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 49ca4a32293..2dbee7fd782 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -308,7 +308,7 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - fs::copy_file(disk_path / from_path, to_disk->getPath() + to_path); + Poco::File(disk_path / from_path).copyTo(to_disk->getPath() + to_path); /// Use more optimal way. else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } From 2b67341f281b53021a3be0faad54f5665a3668c1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 4 May 2021 15:55:21 +0300 Subject: [PATCH 032/154] Finish s3 --- src/Disks/S3/DiskS3.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 07e2d254632..a28e7e0f387 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include @@ -1496,8 +1495,8 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio continue; /// Skip not finished parts. They shouldn't be in 'detached' directory, because CH wouldn't be able to finish processing them. - Poco::Path directory_path (path); - auto directory_name = directory_path.directory(directory_path.depth() - 1); + fs::path directory_path(path); + auto directory_name = directory_path.parent_path().filename().string(); auto predicate = [&directory_name](String & prefix) { return directory_name.starts_with(prefix); }; if (std::any_of(not_finished_prefixes.begin(), not_finished_prefixes.end(), predicate)) continue; @@ -1506,7 +1505,7 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); - Poco::File(metadata_path + path).moveTo(metadata_path + detached_path); + fs::rename(fs::path(metadata_path) / path, fs::path(metadata_path) / detached_path); } } @@ -1538,7 +1537,7 @@ String DiskS3::revisionToString(UInt64 revision) String DiskS3::pathToDetached(const String & source_path) { - return Poco::Path(source_path).parent().append(Poco::Path("detached")).toString() + '/'; + return fs::path(source_path).parent_path() / "detached" / ""; } void DiskS3::onFreeze(const String & path) From 9ec92ec5145d3620cfa0030034fc625ffe7efbae Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 5 May 2021 18:10:14 +0300 Subject: [PATCH 033/154] Fix tests, less manual concatination of paths --- src/Disks/DiskLocal.cpp | 70 ++++++++-------- src/Disks/DiskLocal.h | 11 ++- src/Disks/IDisk.cpp | 9 ++- src/Disks/IDisk.h | 10 +-- src/Disks/S3/DiskS3.cpp | 80 +++++++++---------- src/Storages/MergeTree/DataPartsExchange.cpp | 16 ++-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 78 +++++++++--------- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 +- .../MergeTree/MergedBlockOutputStream.cpp | 8 +- src/Storages/MergeTree/checkDataPart.cpp | 6 +- src/Storages/StorageMergeTree.cpp | 4 +- 11 files changed, 146 insertions(+), 152 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 2dbee7fd782..64c599c5b81 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -69,13 +69,7 @@ public: bool isValid() const override { return entry != fs::directory_iterator(); } - String path() const override - { - if (entry->is_directory()) - return dir_path / entry->path().filename() / ""; - else - return dir_path / entry->path().filename(); - } + String path() const override { return dir_path / entry->path().filename(); } String name() const override { return entry->path().filename(); } @@ -119,9 +113,9 @@ UInt64 DiskLocal::getTotalSpace() const { struct statvfs fs; if (name == "default") /// for default disk we get space from path/data/ - fs = getStatVFS(fs::path(disk_path / "data/").string()); + fs = getStatVFS((fs::path(disk_path) / "data/").string()); else - fs = getStatVFS(disk_path.string()); + fs = getStatVFS(disk_path); UInt64 total_size = fs.f_blocks * fs.f_bsize; if (total_size < keep_free_space_bytes) return 0; @@ -134,9 +128,9 @@ UInt64 DiskLocal::getAvailableSpace() const /// available for superuser only and for system purposes struct statvfs fs; if (name == "default") /// for default disk we get space from path/data/ - fs = getStatVFS(fs::path(disk_path / "data/").string()); + fs = getStatVFS((fs::path(disk_path) / "data/").string()); else - fs = getStatVFS(disk_path.string()); + fs = getStatVFS(disk_path); UInt64 total_size = fs.f_bavail * fs.f_bsize; if (total_size < keep_free_space_bytes) return 0; @@ -153,64 +147,64 @@ UInt64 DiskLocal::getUnreservedSpace() const bool DiskLocal::exists(const String & path) const { - return fs::exists(disk_path / path); + return fs::exists(fs::path(disk_path) / path); } bool DiskLocal::isFile(const String & path) const { - return fs::is_regular_file(disk_path / path); + return fs::is_regular_file(fs::path(disk_path) / path); } bool DiskLocal::isDirectory(const String & path) const { - return fs::is_directory(disk_path / path); + return fs::is_directory(fs::path(disk_path) / path); } size_t DiskLocal::getFileSize(const String & path) const { - return fs::file_size(disk_path / path); + return fs::file_size(fs::path(disk_path) / path); } void DiskLocal::createDirectory(const String & path) { - fs::create_directory(disk_path / path); + fs::create_directory(fs::path(disk_path) / path); } void DiskLocal::createDirectories(const String & path) { - fs::create_directories(disk_path / path); + fs::create_directories(fs::path(disk_path) / path); } void DiskLocal::clearDirectory(const String & path) { - for (const auto & entry : fs::directory_iterator(disk_path / path)) + for (const auto & entry : fs::directory_iterator(fs::path(disk_path) / path)) fs::remove(entry.path()); } void DiskLocal::moveDirectory(const String & from_path, const String & to_path) { - fs::rename(disk_path / from_path, disk_path / to_path); + fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); } DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { - return std::make_unique(disk_path, path); + return std::make_unique(fs::path(disk_path), path); } void DiskLocal::moveFile(const String & from_path, const String & to_path) { - fs::rename(disk_path / from_path, disk_path / to_path); + fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); } void DiskLocal::replaceFile(const String & from_path, const String & to_path) { - fs::path from_file = disk_path / from_path; - fs::path to_file = disk_path / to_path; + fs::path from_file = fs::path(disk_path) / from_path; + fs::path to_file = fs::path(disk_path) / to_path; if (fs::exists(to_file)) { fs::path tmp_file(to_file.string() + ".old"); fs::rename(to_file, tmp_file); - fs::rename(from_file, disk_path / to_path); + fs::rename(from_file, fs::path(disk_path) / to_path); fs::remove(tmp_file); } else @@ -223,33 +217,33 @@ std::unique_ptr DiskLocal::readFile( const String & path, size_t buf_size, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, MMappedFileCache * mmap_cache) const { - return createReadBufferFromFileBase(disk_path / path, estimated_size, aio_threshold, mmap_threshold, mmap_cache, buf_size); + return createReadBufferFromFileBase(fs::path(disk_path) / path, estimated_size, aio_threshold, mmap_threshold, mmap_cache, buf_size); } std::unique_ptr DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode) { int flags = (mode == WriteMode::Append) ? (O_APPEND | O_CREAT | O_WRONLY) : -1; - return std::make_unique(disk_path / path, buf_size, flags); + return std::make_unique(fs::path(disk_path) / path, buf_size, flags); } void DiskLocal::removeFile(const String & path) { - auto fs_path = disk_path / path; + auto fs_path = fs::path(disk_path) / path; if (0 != unlink(fs_path.c_str())) throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK); } void DiskLocal::removeFileIfExists(const String & path) { - auto fs_path = disk_path / path; + auto fs_path = fs::path(disk_path) / path; if (0 != unlink(fs_path.c_str()) && errno != ENOENT) throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK); } void DiskLocal::removeDirectory(const String & path) { - auto fs_path = disk_path / path; + auto fs_path = fs::path(disk_path) / path; if (0 != rmdir(fs_path.c_str())) throwFromErrnoWithPath("Cannot rmdir " + fs_path.string(), fs_path, ErrorCodes::CANNOT_RMDIR); } @@ -262,42 +256,42 @@ void DiskLocal::removeRecursive(const String & path) void DiskLocal::listFiles(const String & path, std::vector & file_names) { file_names.clear(); - for (const auto & entry : fs::directory_iterator(disk_path / path)) + for (const auto & entry : fs::directory_iterator(fs::path(disk_path) / path)) file_names.emplace_back(entry.path().filename()); } void DiskLocal::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - fs::last_write_time(disk_path / path, static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); + fs::last_write_time(fs::path(disk_path) / path, static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); } Poco::Timestamp DiskLocal::getLastModified(const String & path) { - fs::file_time_type fs_time = fs::last_write_time(disk_path / path); + fs::file_time_type fs_time = fs::last_write_time(fs::path(disk_path) / path); auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); return Poco::Timestamp(micro_sec.count()); } void DiskLocal::createHardLink(const String & src_path, const String & dst_path) { - DB::createHardLink(disk_path / src_path, disk_path / dst_path); + DB::createHardLink(fs::path(disk_path) / src_path, fs::path(disk_path) / dst_path); } void DiskLocal::truncateFile(const String & path, size_t size) { - int res = truncate((disk_path / path).string().data(), size); + int res = truncate((fs::path(disk_path) / path).string().data(), size); if (-1 == res) throwFromErrnoWithPath("Cannot truncate file " + path, path, ErrorCodes::CANNOT_TRUNCATE_FILE); } void DiskLocal::createFile(const String & path) { - Poco::File(disk_path / path).createFile(); + Poco::File(fs::path(disk_path) / path).createFile(); } void DiskLocal::setReadOnly(const String & path) { - Poco::File(disk_path / path).setReadOnly(true); + Poco::File(fs::path(disk_path) / path).setReadOnly(true); } bool inline isSameDiskType(const IDisk & one, const IDisk & another) @@ -308,14 +302,14 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - Poco::File(disk_path / from_path).copyTo(to_disk->getPath() + to_path); /// Use more optimal way. + Poco::File(fs::path(disk_path) / from_path).copyTo(fs::path(to_disk->getPath()) / to_path); /// Use more optimal way. else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } SyncGuardPtr DiskLocal::getDirectorySyncGuard(const String & path) const { - return std::make_unique(disk_path / path); + return std::make_unique(fs::path(disk_path) / path); } DiskPtr DiskLocalReservation::getDisk(size_t i) const diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 91835707725..47482ad8d67 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -22,15 +22,15 @@ public: friend class DiskLocalReservation; DiskLocal(const String & name_, const String & path_, UInt64 keep_free_space_bytes_) - : name(name_), disk_path(path_), disk_path_str(path_), keep_free_space_bytes(keep_free_space_bytes_) + : name(name_), disk_path(path_), keep_free_space_bytes(keep_free_space_bytes_) { - if (disk_path_str.back() != '/') - throw Exception("Disk path must ends with '/', but '" + disk_path_str + "' doesn't.", ErrorCodes::LOGICAL_ERROR); + if (disk_path.back() != '/') + throw Exception("Disk path must end with '/', but '" + disk_path + "' doesn't.", ErrorCodes::LOGICAL_ERROR); } const String & getName() const override { return name; } - const String & getPath() const override { return disk_path_str; } + const String & getPath() const override { return disk_path; } ReservationPtr reserve(UInt64 bytes) override; @@ -107,8 +107,7 @@ private: private: const String name; - const fs::path disk_path; - const String disk_path_str; + const String disk_path; const UInt64 keep_free_space_bytes; UInt64 reserved_bytes = 0; diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 8c69398d3ad..82705b5dcc8 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -22,7 +22,8 @@ bool IDisk::isDirectoryEmpty(const String & path) void copyFile(IDisk & from_disk, const String & from_path, IDisk & to_disk, const String & to_path) { - LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} {} to {} {}.", from_disk.getName(), from_path, to_disk.getName(), to_path); + LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} (path: {}) {} to {} (path: {}) {}.", + from_disk.getName(), from_disk.getPath(), from_path, to_disk.getName(), to_disk.getPath(), to_path); auto in = from_disk.readFile(from_path); auto out = to_disk.writeFile(to_path); @@ -41,16 +42,16 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p [&from_disk, from_path, &to_disk, to_path]() { setThreadName("DiskCopier"); - DB::copyFile(from_disk, from_path, to_disk, to_path + fileName(from_path)); + DB::copyFile(from_disk, from_path, to_disk, fs::path(to_path) / fileName(from_path)); }); results.push_back(std::move(result)); } else { - const String & dir_name = directoryPath(from_path); + fs::path dir_name = fs::path(from_path).parent_path().filename(); fs::path dest(fs::path(to_path) / dir_name); - fs::create_directories(dest); + to_disk.createDirectories(dest); for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next()) asyncCopy(from_disk, it->path(), to_disk, dest, exec, results); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index a70404cbfb4..62ab54e90c2 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -213,10 +213,10 @@ public: virtual DiskType::Type getType() const = 0; /// Invoked when Global Context is shutdown. - virtual void shutdown() { } + virtual void shutdown() {} /// Performs action on disk startup. - virtual void startup() { } + virtual void startup() {} /// Return some uniq string for file, overrode for S3 /// Required for distinguish different copies of the same part on S3 @@ -234,7 +234,7 @@ public: virtual SyncGuardPtr getDirectorySyncGuard(const String & path) const; /// Applies new settings for disk in runtime. - virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextConstPtr) { } + virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextConstPtr) {} protected: friend class DiskDecorator; @@ -295,7 +295,7 @@ public: /// Return full path to a file on disk. inline String fullPath(const DiskPtr & disk, const String & path) { - return disk->getPath() + path; + return fs::path(disk->getPath()) / path; } /// Return parent path for the specified path. @@ -313,7 +313,7 @@ inline String fileName(const String & path) /// Return directory path for the specified path. inline String directoryPath(const String & path) { - return fs::is_directory(path) ? path : fs::path(path).parent_path().string(); + return Poco::Path(path).setFileName("").toString(); } } diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index a28e7e0f387..442f015b3ec 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -117,12 +117,12 @@ struct DiskS3::Metadata using PathAndSize = std::pair; /// S3 root path. - const String & s3_root_path; + fs::path s3_root_path; /// Disk path. - const String & disk_path; + fs::path disk_path; /// Relative path to metadata file on local FS. - String metadata_file_path; + fs::path metadata_file_path; /// Total size of all S3 objects. size_t total_size; /// S3 objects paths and their sizes. @@ -141,14 +141,14 @@ struct DiskS3::Metadata try { - ReadBufferFromFile buf(disk_path + metadata_file_path, 1024); /* reasonable buffer size for small file */ + ReadBufferFromFile buf(disk_path / metadata_file_path, 1024); /* reasonable buffer size for small file */ UInt32 version; readIntText(version, buf); if (version < VERSION_ABSOLUTE_PATHS || version > VERSION_READ_ONLY_FLAG) throw Exception( - "Unknown metadata file version. Path: " + disk_path + metadata_file_path + "Unknown metadata file version. Path: " + (disk_path / metadata_file_path).string() + " Version: " + std::to_string(version) + ", Maximum expected version: " + std::to_string(VERSION_READ_ONLY_FLAG), ErrorCodes::UNKNOWN_FORMAT); @@ -169,12 +169,12 @@ struct DiskS3::Metadata readEscapedString(s3_object_path, buf); if (version == VERSION_ABSOLUTE_PATHS) { - if (!boost::algorithm::starts_with(s3_object_path, s3_root_path)) + if (!boost::algorithm::starts_with(s3_object_path, s3_root_path.string())) throw Exception( "Path in metadata does not correspond S3 root path. Path: " + s3_object_path - + ", root path: " + s3_root_path + ", disk path: " + disk_path_, + + ", root path: " + s3_root_path.string() + ", disk path: " + disk_path_, ErrorCodes::UNKNOWN_FORMAT); - s3_object_path = s3_object_path.substr(s3_root_path.size()); + s3_object_path = s3_object_path.substr(s3_root_path.string().size()); } assertChar('\n', buf); s3_objects[i] = {s3_object_path, s3_object_size}; @@ -207,7 +207,7 @@ struct DiskS3::Metadata /// Fsync metadata file if 'sync' flag is set. void save(bool sync = false) { - WriteBufferFromFile buf(disk_path + metadata_file_path, 1024); + WriteBufferFromFile buf(disk_path / metadata_file_path, 1024); writeIntText(VERSION_RELATIVE_PATHS, buf); writeChar('\n', buf); @@ -338,7 +338,7 @@ private: const auto & [path, size] = metadata.s3_objects[i]; if (size > offset) { - auto buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, s3_max_single_read_retries, buf_size); + auto buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path / path, s3_max_single_read_retries, buf_size); buf->seek(offset, SEEK_SET); return buf; } @@ -367,7 +367,7 @@ private: ++current_buf_idx; const auto & path = metadata.s3_objects[current_buf_idx].first; - current_buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, s3_max_single_read_retries, buf_size); + current_buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path / path, s3_max_single_read_retries, buf_size); current_buf->next(); working_buffer = current_buf->buffer(); absolute_position += working_buffer.size(); @@ -447,16 +447,16 @@ public: String path() const override { if (fs::is_directory(iter->path())) - return folder_path + iter->path().filename().string() + '/'; + return folder_path / iter->path().filename().string() / ""; else - return folder_path + iter->path().filename().string(); + return folder_path / iter->path().filename().string(); } String name() const override { return iter->path().filename(); } private: fs::directory_iterator iter; - String folder_path; + fs::path folder_path; }; @@ -623,13 +623,13 @@ String DiskS3::getUniqueId(const String & path) const Metadata metadata(s3_root_path, metadata_path, path); String id; if (!metadata.s3_objects.empty()) - id = metadata.s3_root_path + metadata.s3_objects[0].first; + id = metadata.s3_root_path / metadata.s3_objects[0].first; return id; } DiskDirectoryIteratorPtr DiskS3::iterateDirectory(const String & path) { - return std::make_unique(metadata_path + path, path); + return std::make_unique(fs::path(metadata_path) / path, path); } void DiskS3::clearDirectory(const String & path) @@ -683,7 +683,7 @@ std::unique_ptr DiskS3::readFile(const String & path, si auto metadata = readMeta(path); LOG_DEBUG(log, "Read from file by path: {}. Existing S3 objects: {}", - backQuote(metadata_path + path), metadata.s3_objects.size()); + backQuote((fs::path(metadata_path) / path).string()), metadata.s3_objects.size()); auto reader = std::make_unique(settings->client, bucket, metadata, settings->s3_max_single_read_retries, buf_size); return std::make_unique(std::move(reader), settings->min_bytes_for_seek); @@ -708,12 +708,12 @@ std::unique_ptr DiskS3::writeFile(const String & path, } LOG_DEBUG(log, "{} to file by path: {}. S3 path: {}", - mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_path + path), s3_root_path + s3_path); + mode == WriteMode::Rewrite ? "Write" : "Append", backQuote((fs::path(metadata_path) / path).string()), (fs::path(s3_root_path) / s3_path).string()); auto s3_buffer = std::make_unique( settings->client, bucket, - metadata.s3_root_path + s3_path, + fs::path(metadata.s3_root_path) / s3_path, settings->s3_min_upload_part_size, settings->s3_max_single_part_upload_size, std::move(object_metadata), @@ -724,7 +724,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) { - LOG_DEBUG(log, "Remove file by path: {}", backQuote(metadata_path + path)); + LOG_DEBUG(log, "Remove file by path: {}", backQuote((fs::path(metadata_path) / path).string())); fs::path file = fs::path(metadata_path) / path; @@ -741,7 +741,7 @@ void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) fs::remove(file); for (const auto & [s3_object_path, _] : metadata.s3_objects) - keys.addKey(s3_root_path + s3_object_path); + keys.addKey(fs::path(s3_root_path) / s3_object_path); } else /// In other case decrement number of references, save metadata and delete file. { @@ -904,7 +904,7 @@ void DiskS3::createHardLink(const String & src_path, const String & dst_path, bo src.save(); /// Create FS hardlink to metadata file. - DB::createHardLink(metadata_path + src_path, metadata_path + dst_path); + DB::createHardLink(fs::path(metadata_path) / src_path, fs::path(metadata_path) / dst_path); } void DiskS3::createFile(const String & path) @@ -940,7 +940,7 @@ void DiskS3::createFileOperationObject(const String & operation_name, UInt64 rev WriteBufferFromS3 buffer( settings->client, bucket, - s3_root_path + key, + fs::path(s3_root_path) / key, settings->s3_min_upload_part_size, settings->s3_max_single_part_upload_size, metadata); @@ -993,14 +993,14 @@ void DiskS3::findLastRevision() int DiskS3::readSchemaVersion(const String & source_bucket, const String & source_path) { int version = 0; - if (!checkObjectExists(source_bucket, source_path + SCHEMA_VERSION_OBJECT)) + if (!checkObjectExists(source_bucket, fs::path(source_path) / SCHEMA_VERSION_OBJECT)) return version; auto settings = current_settings.get(); ReadBufferFromS3 buffer( settings->client, source_bucket, - source_path + SCHEMA_VERSION_OBJECT, + fs::path(source_path) / SCHEMA_VERSION_OBJECT, settings->s3_max_single_read_retries); readIntText(version, buffer); @@ -1015,7 +1015,7 @@ void DiskS3::saveSchemaVersion(const int & version) WriteBufferFromS3 buffer( settings->client, bucket, - s3_root_path + SCHEMA_VERSION_OBJECT, + fs::path(s3_root_path) / SCHEMA_VERSION_OBJECT, settings->s3_min_upload_part_size, settings->s3_max_single_part_upload_size); @@ -1027,7 +1027,7 @@ void DiskS3::updateObjectMetadata(const String & key, const ObjectMetadata & met { auto settings = current_settings.get(); Aws::S3::Model::CopyObjectRequest request; - request.SetCopySource(bucket + "/" + key); + request.SetCopySource(fs::path(bucket) / key); request.SetBucket(bucket); request.SetKey(key); request.SetMetadata(metadata); @@ -1039,7 +1039,7 @@ void DiskS3::updateObjectMetadata(const String & key, const ObjectMetadata & met void DiskS3::migrateFileToRestorableSchema(const String & path) { - LOG_DEBUG(log, "Migrate file {} to restorable schema", metadata_path + path); + LOG_DEBUG(log, "Migrate file {} to restorable schema", (fs::path(metadata_path) / path).string()); auto meta = readMeta(path); @@ -1048,7 +1048,7 @@ void DiskS3::migrateFileToRestorableSchema(const String & path) ObjectMetadata metadata { {"path", path} }; - updateObjectMetadata(s3_root_path + key, metadata); + updateObjectMetadata(fs::path(s3_root_path) / key, metadata); } } @@ -1056,7 +1056,7 @@ void DiskS3::migrateToRestorableSchemaRecursive(const String & path, Futures & r { checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. - LOG_DEBUG(log, "Migrate directory {} to restorable schema", metadata_path + path); + LOG_DEBUG(log, "Migrate directory {} to restorable schema", (fs::path(metadata_path) / path).string()); bool dir_contains_only_files = true; for (auto it = iterateDirectory(path); it->isValid(); it->next()) @@ -1105,7 +1105,7 @@ void DiskS3::migrateToRestorableSchema() for (const auto & root : data_roots) if (exists(root)) - migrateToRestorableSchemaRecursive(root + '/', results); + migrateToRestorableSchemaRecursive(root, results); for (auto & result : results) result.wait(); @@ -1194,7 +1194,7 @@ void DiskS3::copyObject(const String & src_bucket, const String & src_key, const { auto settings = current_settings.get(); Aws::S3::Model::CopyObjectRequest request; - request.SetCopySource(src_bucket + "/" + src_key); + request.SetCopySource(fs::path(src_bucket) / src_key); request.SetBucket(dst_bucket); request.SetKey(dst_key); @@ -1212,7 +1212,7 @@ struct DiskS3::RestoreInformation void DiskS3::readRestoreInformation(DiskS3::RestoreInformation & restore_information) { - ReadBufferFromFile buffer(metadata_path + RESTORE_FILE_NAME, 512); + ReadBufferFromFile buffer(fs::path(metadata_path) / RESTORE_FILE_NAME, 512); buffer.next(); try @@ -1302,7 +1302,7 @@ void DiskS3::restore() bool cleanup_s3 = information.source_bucket != bucket || information.source_path != s3_root_path; for (const auto & root : data_roots) if (exists(root)) - removeSharedRecursive(root + '/', !cleanup_s3); + removeSharedRecursive(root, !cleanup_s3); restoreFiles(information); restoreFileOperations(information); @@ -1393,8 +1393,8 @@ void DiskS3::processRestoreFiles(const String & source_bucket, const String & so auto relative_key = shrinkKey(source_path, key); /// Copy object if we restore to different bucket / path. - if (bucket != source_bucket || s3_root_path != source_path) - copyObject(source_bucket, key, bucket, s3_root_path + relative_key); + if (bucket != source_bucket || fs::path(s3_root_path) != fs::path(source_path)) + copyObject(source_bucket, key, bucket, fs::path(s3_root_path) / relative_key); metadata.addObject(relative_key, head_result.GetContentLength()); metadata.save(); @@ -1482,7 +1482,7 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio }; /// Execute. - listObjects(restore_information.source_bucket, restore_information.source_path + "operations/", restore_file_operations); + listObjects(restore_information.source_bucket, fs::path(restore_information.source_path) / "operations/", restore_file_operations); if (restore_information.detached) { @@ -1505,7 +1505,7 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); - fs::rename(fs::path(metadata_path) / path, fs::path(metadata_path) / detached_path); + Poco::File(fs::path(metadata_path) / path).moveTo(fs::path(metadata_path) / detached_path); } } @@ -1537,13 +1537,13 @@ String DiskS3::revisionToString(UInt64 revision) String DiskS3::pathToDetached(const String & source_path) { - return fs::path(source_path).parent_path() / "detached" / ""; + return Poco::Path(source_path).parent().append(Poco::Path("detached")).toString() + '/'; } void DiskS3::onFreeze(const String & path) { createDirectories(path); - WriteBufferFromFile revision_file_buf(metadata_path + path + "revision.txt", 32); + WriteBufferFromFile revision_file_buf(fs::path(metadata_path) / path / "revision.txt", 32); writeIntText(revision_counter.load(), revision_file_buf); revision_file_buf.finalize(); } diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index dfeb31af7b7..82617e6e0af 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -230,7 +230,7 @@ void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuf { String file_name = it.first; - String path = part->getFullRelativePath() + file_name; + String path = fs::path(part->getFullRelativePath()) / file_name; UInt64 size = disk->getFileSize(path); @@ -279,7 +279,7 @@ void Service::sendPartS3Metadata(const MergeTreeData::DataPartPtr & part, WriteB { String file_name = it.first; - String metadata_file = disk->getPath() + part->getFullRelativePath() + file_name; + String metadata_file = fs::path(disk->getPath()) / part->getFullRelativePath() / file_name; fs::path metadata(metadata_file); @@ -480,7 +480,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( readUUIDText(part_uuid, in); auto storage_id = data.getStorageID(); - String new_part_path = part_type == "InMemory" ? "memory" : data.getFullPathOnDisk(reservation->getDisk()) + part_name + "/"; + String new_part_path = part_type == "InMemory" ? "memory" : fs::path(data.getFullPathOnDisk(reservation->getDisk())) / part_name / ""; auto entry = data.getContext()->getReplicatedFetchList().insert( storage_id.getDatabaseName(), storage_id.getTableName(), part_info.partition_id, part_name, new_part_path, @@ -551,7 +551,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( throw Exception("Logical error: tmp_prefix and part_name cannot be empty or contain '.' or '/' characters.", ErrorCodes::LOGICAL_ERROR); String part_relative_path = String(to_detached ? "detached/" : "") + tmp_prefix + part_name; - String part_download_path = data.getRelativeDataPath() + part_relative_path + "/"; + String part_download_path = fs::path(data.getRelativeDataPath()) / part_relative_path / ""; if (disk->exists(part_download_path)) { @@ -583,7 +583,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( " This may happen if we are trying to download part from malicious replica or logical error.", ErrorCodes::INSECURE_PATH); - auto file_out = disk->writeFile(part_download_path + file_name); + auto file_out = disk->writeFile(fs::path(part_download_path) / file_name); HashingWriteBuffer hashing_out(*file_out); copyData(in, hashing_out, file_size, blocker.getCounter()); @@ -600,7 +600,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( readPODBinary(expected_hash, in); if (expected_hash != hashing_out.getHash()) - throw Exception("Checksum mismatch for file " + fullPath(disk, part_download_path + file_name) + " transferred from " + replica_path, + throw Exception("Checksum mismatch for file " + fullPath(disk, (fs::path(part_download_path) / file_name).string()) + " transferred from " + replica_path, ErrorCodes::CHECKSUM_DOESNT_MATCH); if (file_name != "checksums.txt" && @@ -654,7 +654,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToS3( String tmp_prefix = tmp_prefix_.empty() ? TMP_PREFIX : tmp_prefix_; String part_relative_path = String(to_detached ? "detached/" : "") + tmp_prefix + part_name; - String part_download_path = data.getRelativeDataPath() + part_relative_path + "/"; + String part_download_path = fs::path(data.getRelativeDataPath()) / part_relative_path / ""; if (disk->exists(part_download_path)) throw Exception("Directory " + fullPath(disk, part_download_path) + " already exists.", ErrorCodes::DIRECTORY_ALREADY_EXISTS); @@ -677,7 +677,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToS3( readStringBinary(file_name, in); readBinary(file_size, in); - String data_path = new_data_part->getFullRelativePath() + file_name; + String data_path = fs::path(new_data_part->getFullRelativePath()) / file_name; String metadata_file = fullPath(disk, data_path); { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 36032f9208f..437718954d4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -69,7 +69,7 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Dis hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { - String file_name = part_path + "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; + String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); auto file = openForReading(disk_, file_name); auto serialization = minmax_column_types[i]->getDefaultSerialization(); @@ -111,7 +111,7 @@ void IMergeTreeDataPart::MinMaxIndex::store( String file_name = "minmax_" + escapeForFileName(column_names[i]) + ".idx"; auto serialization = data_types.at(i)->getDefaultSerialization(); - auto out = disk_->writeFile(part_path + file_name); + auto out = disk_->writeFile(fs::path(part_path) / file_name); HashingWriteBuffer out_hashing(*out); serialization->serializeBinary(hyperrectangle[i].left, out_hashing); serialization->serializeBinary(hyperrectangle[i].right, out_hashing); @@ -543,7 +543,7 @@ String IMergeTreeDataPart::getFullPath() const if (relative_path.empty()) throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - return storage.getFullPathOnDisk(volume->getDisk()) + relative_path + "/"; + return fs::path(storage.getFullPathOnDisk(volume->getDisk())) / relative_path / ""; } String IMergeTreeDataPart::getFullRelativePath() const @@ -551,7 +551,7 @@ String IMergeTreeDataPart::getFullRelativePath() const if (relative_path.empty()) throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - return storage.relative_data_path + relative_path + "/"; + return fs::path(storage.relative_data_path) / relative_path / ""; } void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency) @@ -604,7 +604,7 @@ void IMergeTreeDataPart::loadIndex() loaded_index[i]->reserve(index_granularity.getMarksCount()); } - String index_path = getFullRelativePath() + "primary.idx"; + String index_path = fs::path(getFullRelativePath()) / "primary.idx"; auto index_file = openForReading(volume->getDisk(), index_path); size_t marks_count = index_granularity.getMarksCount(); @@ -639,7 +639,7 @@ NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const return {}; NameSet result = {"checksums.txt", "columns.txt"}; - String default_codec_path = getFullRelativePath() + DEFAULT_COMPRESSION_CODEC_FILE_NAME; + String default_codec_path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; if (volume->getDisk()->exists(default_codec_path)) result.emplace(DEFAULT_COMPRESSION_CODEC_FILE_NAME); @@ -656,7 +656,7 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() return; } - String path = getFullRelativePath() + DEFAULT_COMPRESSION_CODEC_FILE_NAME; + String path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; if (!volume->getDisk()->exists(path)) { default_codec = detectDefaultCompressionCodec(); @@ -717,7 +717,7 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const { if (path_to_data_file.empty()) { - String candidate_path = getFullRelativePath() + ISerialization::getFileNameForStream(part_column, substream_path) + ".bin"; + String candidate_path = fs::path(getFullRelativePath()) / (ISerialization::getFileNameForStream(part_column, substream_path) + ".bin"); /// We can have existing, but empty .bin files. Example: LowCardinality(Nullable(...)) columns and column_name.dict.null.bin file. if (volume->getDisk()->exists(candidate_path) && volume->getDisk()->getFileSize(candidate_path) != 0) @@ -773,7 +773,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() void IMergeTreeDataPart::loadChecksums(bool require) { - const String path = getFullRelativePath() + "checksums.txt"; + const String path = fs::path(getFullRelativePath()) / "checksums.txt"; if (volume->getDisk()->exists(path)) { @@ -798,11 +798,11 @@ void IMergeTreeDataPart::loadChecksums(bool require) checksums = checkDataPart(shared_from_this(), false); { - auto out = volume->getDisk()->writeFile(getFullRelativePath() + "checksums.txt.tmp", 4096); + auto out = volume->getDisk()->writeFile(fs::path(getFullRelativePath()) / "checksums.txt.tmp", 4096); checksums.write(*out); } - volume->getDisk()->moveFile(getFullRelativePath() + "checksums.txt.tmp", getFullRelativePath() + "checksums.txt"); + volume->getDisk()->moveFile(fs::path(getFullRelativePath()) / "checksums.txt.tmp", fs::path(getFullRelativePath()) / "checksums.txt"); bytes_on_disk = checksums.getTotalSizeOnDisk(); } @@ -810,7 +810,7 @@ void IMergeTreeDataPart::loadChecksums(bool require) void IMergeTreeDataPart::loadRowsCount() { - String path = getFullRelativePath() + "count.txt"; + String path = fs::path(getFullRelativePath()) / "count.txt"; if (index_granularity.empty()) { rows_count = 0; @@ -911,7 +911,7 @@ void IMergeTreeDataPart::loadRowsCount() void IMergeTreeDataPart::loadTTLInfos() { - String path = getFullRelativePath() + "ttl.txt"; + String path = fs::path(getFullRelativePath()) / "ttl.txt"; if (volume->getDisk()->exists(path)) { auto in = openForReading(volume->getDisk(), path); @@ -938,7 +938,7 @@ void IMergeTreeDataPart::loadTTLInfos() void IMergeTreeDataPart::loadUUID() { - String path = getFullRelativePath() + UUID_FILE_NAME; + String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; if (volume->getDisk()->exists(path)) { @@ -951,7 +951,7 @@ void IMergeTreeDataPart::loadUUID() void IMergeTreeDataPart::loadColumns(bool require) { - String path = getFullRelativePath() + "columns.txt"; + String path = fs::path(getFullRelativePath()) / "columns.txt"; auto metadata_snapshot = storage.getInMemoryMetadataPtr(); NamesAndTypesList loaded_columns; @@ -964,7 +964,7 @@ void IMergeTreeDataPart::loadColumns(bool require) /// If there is no file with a list of columns, write it down. for (const NameAndTypePair & column : metadata_snapshot->getColumns().getAllPhysical()) - if (volume->getDisk()->exists(getFullRelativePath() + getFileNameForColumn(column) + ".bin")) + if (volume->getDisk()->exists(fs::path(getFullRelativePath()) / (getFileNameForColumn(column) + ".bin"))) loaded_columns.push_back(column); if (columns.empty()) @@ -1002,7 +1002,7 @@ UInt64 IMergeTreeDataPart::calculateTotalSizeOnDisk(const DiskPtr & disk_, const disk_->listFiles(from, files); UInt64 res = 0; for (const auto & file : files) - res += calculateTotalSizeOnDisk(disk_, from + file); + res += calculateTotalSizeOnDisk(disk_, fs::path(from) / file); return res; } @@ -1012,7 +1012,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ assertOnDisk(); String from = getFullRelativePath(); - String to = storage.relative_data_path + new_relative_path + "/"; + String to = fs::path(storage.relative_data_path) / new_relative_path / ""; if (!volume->getDisk()->exists(from)) throw Exception("Part directory " + fullPath(volume->getDisk(), from) + " doesn't exist. Most likely it is a logical error.", ErrorCodes::FILE_DOESNT_EXIST); @@ -1066,8 +1066,8 @@ void IMergeTreeDataPart::remove(bool keep_s3) const * And a race condition can happen that will lead to "File not found" error here. */ - String from = storage.relative_data_path + relative_path; - String to = storage.relative_data_path + "delete_tmp_" + name; + fs::path from = fs::path(storage.relative_data_path) / relative_path; + fs::path to = fs::path(storage.relative_data_path) / ("delete_tmp_" + name); // TODO directory delete_tmp_ is never removed if server crashes before returning from this function if (volume->getDisk()->exists(to)) @@ -1076,7 +1076,7 @@ void IMergeTreeDataPart::remove(bool keep_s3) const try { - volume->getDisk()->removeSharedRecursive(to + "/", keep_s3); + volume->getDisk()->removeSharedRecursive(to / "", keep_s3); } catch (...) { @@ -1099,7 +1099,7 @@ void IMergeTreeDataPart::remove(bool keep_s3) const if (checksums.empty()) { /// If the part is not completely written, we cannot use fast path by listing files. - volume->getDisk()->removeSharedRecursive(to + "/", keep_s3); + volume->getDisk()->removeSharedRecursive(to / "", keep_s3); } else { @@ -1112,16 +1112,16 @@ void IMergeTreeDataPart::remove(bool keep_s3) const # pragma GCC diagnostic ignored "-Wunused-variable" #endif for (const auto & [file, _] : checksums.files) - volume->getDisk()->removeSharedFile(to + "/" + file, keep_s3); + volume->getDisk()->removeSharedFile(to / file, keep_s3); #if !defined(__clang__) # pragma GCC diagnostic pop #endif for (const auto & file : {"checksums.txt", "columns.txt"}) - volume->getDisk()->removeSharedFile(to + "/" + file, keep_s3); + volume->getDisk()->removeSharedFile(to / file, keep_s3); - volume->getDisk()->removeSharedFileIfExists(to + "/" + DEFAULT_COMPRESSION_CODEC_FILE_NAME, keep_s3); - volume->getDisk()->removeSharedFileIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME, keep_s3); + volume->getDisk()->removeSharedFileIfExists(to / DEFAULT_COMPRESSION_CODEC_FILE_NAME, keep_s3); + volume->getDisk()->removeSharedFileIfExists(to / DELETE_ON_DESTROY_MARKER_FILE_NAME, keep_s3); volume->getDisk()->removeDirectory(to); } @@ -1131,7 +1131,7 @@ void IMergeTreeDataPart::remove(bool keep_s3) const LOG_ERROR(storage.log, "Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: {}", fullPath(volume->getDisk(), to), getCurrentExceptionMessage(false)); - volume->getDisk()->removeSharedRecursive(to + "/", keep_s3); + volume->getDisk()->removeSharedRecursive(to / "", keep_s3); } } } @@ -1149,7 +1149,7 @@ String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix) const { res = (prefix.empty() ? "" : prefix + "_") + name + (try_no ? "_try" + DB::toString(try_no) : ""); - if (!volume->getDisk()->exists(getFullRelativePath() + res)) + if (!volume->getDisk()->exists(fs::path(getFullRelativePath()) / res)) return res; LOG_WARNING(storage.log, "Directory {} (to detach to) already exists. Will detach to directory with '_tryN' suffix.", res); @@ -1172,11 +1172,11 @@ void IMergeTreeDataPart::renameToDetached(const String & prefix) const void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & /*metadata_snapshot*/) const { - String destination_path = storage.relative_data_path + getRelativePathForDetachedPart(prefix); + String destination_path = fs::path(storage.relative_data_path) / getRelativePathForDetachedPart(prefix); /// Backup is not recursive (max_level is 0), so do not copy inner directories localBackup(volume->getDisk(), getFullRelativePath(), destination_path, 0); - volume->getDisk()->removeFileIfExists(destination_path + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME); + volume->getDisk()->removeFileIfExists(fs::path(destination_path) / DELETE_ON_DESTROY_MARKER_FILE_NAME); } void IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & directory_name) const @@ -1188,16 +1188,16 @@ void IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & di if (directory_name.empty()) throw Exception("Can not clone data part " + name + " to empty directory.", ErrorCodes::LOGICAL_ERROR); - String path_to_clone = storage.relative_data_path + directory_name + '/'; + String path_to_clone = fs::path(storage.relative_data_path) / directory_name / ""; - if (disk->exists(path_to_clone + relative_path)) + if (disk->exists(fs::path(path_to_clone) / relative_path)) { LOG_WARNING(storage.log, "Path " + fullPath(disk, path_to_clone + relative_path) + " already exists. Will remove it and clone again."); - disk->removeRecursive(path_to_clone + relative_path + '/'); + disk->removeRecursive(fs::path(path_to_clone) / relative_path / ""); } disk->createDirectories(path_to_clone); volume->getDisk()->copy(getFullRelativePath(), disk, path_to_clone); - volume->getDisk()->removeFileIfExists(path_to_clone + '/' + DELETE_ON_DESTROY_MARKER_FILE_NAME); + volume->getDisk()->removeFileIfExists(fs::path(path_to_clone) / DELETE_ON_DESTROY_MARKER_FILE_NAME); } void IMergeTreeDataPart::checkConsistencyBase() const @@ -1244,17 +1244,17 @@ void IMergeTreeDataPart::checkConsistencyBase() const /// Check that the primary key index is not empty. if (!pk.column_names.empty()) - check_file_not_empty(volume->getDisk(), path + "primary.idx"); + check_file_not_empty(volume->getDisk(), fs::path(path) / "primary.idx"); if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - check_file_not_empty(volume->getDisk(), path + "count.txt"); + check_file_not_empty(volume->getDisk(), fs::path(path) / "count.txt"); if (metadata_snapshot->hasPartitionKey()) - check_file_not_empty(volume->getDisk(), path + "partition.dat"); + check_file_not_empty(volume->getDisk(), fs::path(path) / "partition.dat"); for (const String & col_name : storage.getMinMaxColumnsNames(partition_key)) - check_file_not_empty(volume->getDisk(), path + "minmax_" + escapeForFileName(col_name) + ".idx"); + check_file_not_empty(volume->getDisk(), fs::path(path) / ("minmax_" + escapeForFileName(col_name) + ".idx")); } } } @@ -1348,7 +1348,7 @@ String IMergeTreeDataPart::getUniqueId() const auto disk = volume->getDisk(); if (disk->getType() == DB::DiskType::Type::S3) - id = disk->getUniqueId(getFullRelativePath() + "checksums.txt"); + id = disk->getUniqueId(fs::path(getFullRelativePath()) / "checksums.txt"); if (id.empty()) throw Exception("Can't get unique S3 object", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index dfebd88abe9..17673bbdd97 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1868,7 +1868,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart( if (need_remove_expired_values) { /// Write a file with ttl infos in json format. - auto out_ttl = disk->writeFile(new_data_part->getFullRelativePath() + "ttl.txt", 4096); + auto out_ttl = disk->writeFile(fs::path(new_data_part->getFullRelativePath()) / "ttl.txt", 4096); HashingWriteBuffer out_hashing(*out_ttl); new_data_part->ttl_infos.write(out_hashing); new_data_part->checksums.files["ttl.txt"].file_size = out_hashing.count(); @@ -1877,7 +1877,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart( { /// Write file with checksums. - auto out_checksums = disk->writeFile(new_data_part->getFullRelativePath() + "checksums.txt", 4096); + auto out_checksums = disk->writeFile(fs::path(new_data_part->getFullRelativePath()) / "checksums.txt", 4096); new_data_part->checksums.write(*out_checksums); } /// close fd @@ -1888,7 +1888,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart( { /// Write a file with a description of columns. - auto out_columns = disk->writeFile(new_data_part->getFullRelativePath() + "columns.txt", 4096); + auto out_columns = disk->writeFile(fs::path(new_data_part->getFullRelativePath()) / "columns.txt", 4096); new_data_part->getColumns().writeText(*out_columns); } /// close fd diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index ab364e0e5aa..bc330ce3dae 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -121,7 +121,7 @@ void MergedBlockOutputStream::finalizePartOnDisk( throw Exception("MinMax index was not initialized for new non-empty part " + new_part->name + ". It is a bug.", ErrorCodes::LOGICAL_ERROR); - auto count_out = volume->getDisk()->writeFile(part_path + "count.txt", 4096); + auto count_out = volume->getDisk()->writeFile(fs::path(part_path) / "count.txt", 4096); HashingWriteBuffer count_out_hashing(*count_out); writeIntText(rows_count, count_out_hashing); count_out_hashing.next(); @@ -135,7 +135,7 @@ void MergedBlockOutputStream::finalizePartOnDisk( if (!new_part->ttl_infos.empty()) { /// Write a file with ttl infos in json format. - auto out = volume->getDisk()->writeFile(part_path + "ttl.txt", 4096); + auto out = volume->getDisk()->writeFile(fs::path(part_path) / "ttl.txt", 4096); HashingWriteBuffer out_hashing(*out); new_part->ttl_infos.write(out_hashing); checksums.files["ttl.txt"].file_size = out_hashing.count(); @@ -149,7 +149,7 @@ void MergedBlockOutputStream::finalizePartOnDisk( { /// Write a file with a description of columns. - auto out = volume->getDisk()->writeFile(part_path + "columns.txt", 4096); + auto out = volume->getDisk()->writeFile(fs::path(part_path) / "columns.txt", 4096); part_columns.writeText(*out); out->finalize(); if (sync) @@ -170,7 +170,7 @@ void MergedBlockOutputStream::finalizePartOnDisk( { /// Write file with checksums. - auto out = volume->getDisk()->writeFile(part_path + "checksums.txt", 4096); + auto out = volume->getDisk()->writeFile(fs::path(part_path) / "checksums.txt", 4096); checksums.write(*out); out->finalize(); if (sync) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index ac28f84db43..35c73145e66 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -68,7 +68,7 @@ IMergeTreeDataPart::Checksums checkDataPart( NamesAndTypesList columns_txt; { - auto buf = disk->readFile(path + "columns.txt"); + auto buf = disk->readFile(fs::path(path) / "columns.txt"); columns_txt.readText(*buf); assertEOF(*buf); } @@ -141,9 +141,9 @@ IMergeTreeDataPart::Checksums checkDataPart( /// Checksums from the rest files listed in checksums.txt. May be absent. If present, they are subsequently compared with the actual data checksums. IMergeTreeDataPart::Checksums checksums_txt; - if (require_checksums || disk->exists(path + "checksums.txt")) + if (require_checksums || disk->exists(fs::path(path) / "checksums.txt")) { - auto buf = disk->readFile(path + "checksums.txt"); + auto buf = disk->readFile(fs::path(path) / "checksums.txt"); checksums_txt.read(*buf); assertEOF(*buf); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b3165febd7c..b9daa4a1f41 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1482,8 +1482,8 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ auto disk = part->volume->getDisk(); String part_path = part->getFullRelativePath(); /// If the checksums file is not present, calculate the checksums and write them to disk. - String checksums_path = part_path + "checksums.txt"; - String tmp_checksums_path = part_path + "checksums.txt.tmp"; + String checksums_path = fs::path(part_path) / "checksums.txt"; + String tmp_checksums_path = fs::path(part_path) / "checksums.txt.tmp"; if (part->isStoredOnDisk() && !disk->exists(checksums_path)) { try From 5068b163b81950ef775ac567aea1a271eabe3985 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 5 May 2021 19:55:02 +0300 Subject: [PATCH 034/154] Fix --- src/Disks/DiskLocal.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 64c599c5b81..99f0162cf12 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -69,7 +69,14 @@ public: bool isValid() const override { return entry != fs::directory_iterator(); } - String path() const override { return dir_path / entry->path().filename(); } + String path() const override + { + if (entry->is_directory()) + return dir_path / entry->path().filename() / ""; + else + return dir_path / entry->path().filename(); + } + String name() const override { return entry->path().filename(); } From 1e4fda08e2d491d53436ced28e6c22ade89e6642 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 May 2021 08:58:48 +0300 Subject: [PATCH 035/154] Allow JOIN ON with Join engine --- src/Interpreters/ExpressionAnalyzer.cpp | 2 -- tests/queries/0_stateless/00118_storage_join.reference | 10 ++++++++++ tests/queries/0_stateless/00118_storage_join.sql | 6 ++++-- 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index e0d85fc0ca7..af4369527bc 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -813,8 +813,6 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( /// Use StorageJoin if any. JoinPtr join = tryGetStorageJoin(syntax->analyzed_join); - if (join && syntax->analyzed_join->hasOn()) - throw DB::Exception("`JOIN ON` with `Join` table engine not supported, use `USING` syntax", ErrorCodes::NOT_IMPLEMENTED); if (!join) { diff --git a/tests/queries/0_stateless/00118_storage_join.reference b/tests/queries/0_stateless/00118_storage_join.reference index 05cb74361be..cd7e2d67523 100644 --- a/tests/queries/0_stateless/00118_storage_join.reference +++ b/tests/queries/0_stateless/00118_storage_join.reference @@ -34,3 +34,13 @@ 1 12 abc 0 45 +0 0 +1 1 abc +2 2 def +3 3 +4 4 +5 5 +6 6 ghi +7 7 +8 8 +9 9 diff --git a/tests/queries/0_stateless/00118_storage_join.sql b/tests/queries/0_stateless/00118_storage_join.sql index 95b090d4837..d5496687728 100644 --- a/tests/queries/0_stateless/00118_storage_join.sql +++ b/tests/queries/0_stateless/00118_storage_join.sql @@ -11,7 +11,9 @@ SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT SELECT k, js1.s, t2.s FROM (SELECT number AS k, number as s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k; SELECT k, js1.s, t2.s FROM (SELECT toUInt64(number / 3) AS k, sum(number) as s FROM numbers(10) GROUP BY toUInt64(number / 3) WITH TOTALS) js1 ANY LEFT JOIN t2 USING k; --- JOIN ON not supported for storage join -SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; -- { serverError 48 } +SELECT k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; + +-- geting qualified key columns from Join table still doen't work +SELECT t2.k FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; -- { serverError 8 } DROP TABLE t2; From 140bf7e2eaa8901f4ba6abbc13e44c5b88713361 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 7 May 2021 15:29:26 +0300 Subject: [PATCH 036/154] Less Poco --- src/Common/Config/ConfigReloader.cpp | 4 +++- .../MySQL/DatabaseConnectionMySQL.cpp | 2 +- src/Disks/DiskLocal.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 2 +- src/Disks/S3/registerDiskS3.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 24 +++++++++---------- 6 files changed, 19 insertions(+), 17 deletions(-) diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index afff08e82bb..ad0ce7cc30e 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -5,7 +5,9 @@ #include #include #include "ConfigProcessor.h" +#include +namespace fs = std::filesystem; namespace DB { @@ -167,7 +169,7 @@ struct ConfigReloader::FileWithTimestamp void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path_to_add) { - if (!path_to_add.empty() && Poco::File(path_to_add).exists()) + if (!path_to_add.empty() && fs::exists(path_to_add)) files.emplace(path_to_add, Poco::File(path_to_add).getLastModified().epochTime()); } diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 2b6c6d243a7..7b1bb7a0494 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -408,7 +408,7 @@ void DatabaseConnectionMySQL::detachTablePermanently(ContextPtr, const String & { std::lock_guard lock{mutex}; - Poco::File remove_flag(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); + Poco::File remove_flag(fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix)); if (remove_or_detach_tables.count(table_name)) throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " is dropped", diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 99f0162cf12..85023dba464 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -309,7 +309,7 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - Poco::File(fs::path(disk_path) / from_path).copyTo(fs::path(to_disk->getPath()) / to_path); /// Use more optimal way. + fs::copy(fs::path(disk_path) / from_path, fs::path(to_disk->getPath()) / to_path); /// Use more optimal way. else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 442f015b3ec..faff486eeae 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1505,7 +1505,7 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); - Poco::File(fs::path(metadata_path) / path).moveTo(fs::path(metadata_path) / detached_path); + fs::rename(fs::path(metadata_path) / path, fs::path(metadata_path) / detached_path); } } diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 639bb46c033..56726cfcca1 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -172,7 +172,7 @@ void registerDiskS3(DiskFactory & factory) throw Exception("S3 path must ends with '/', but '" + uri.key + "' doesn't.", ErrorCodes::BAD_ARGUMENTS); String metadata_path = config.getString(config_prefix + ".metadata_path", context->getPath() + "disks/" + name + "/"); - Poco::File (metadata_path).createDirectories(); + fs::create_directories(metadata_path); std::shared_ptr s3disk = std::make_shared( name, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 383e504aad8..1deed68f239 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -209,8 +209,8 @@ MergeTreeData::MergeTreeData( for (const auto & [path, disk] : getRelativeDataPathsWithDisks()) { disk->createDirectories(path); - disk->createDirectories(path + MergeTreeData::DETACHED_DIR_NAME); - auto current_version_file_path = path + MergeTreeData::FORMAT_VERSION_FILE_NAME; + disk->createDirectories(fs::path(path) / MergeTreeData::DETACHED_DIR_NAME); + String current_version_file_path = fs::path(path) / MergeTreeData::FORMAT_VERSION_FILE_NAME; if (disk->exists(current_version_file_path)) { if (!version_file.first.empty()) @@ -224,7 +224,7 @@ MergeTreeData::MergeTreeData( /// If not choose any if (version_file.first.empty()) - version_file = {relative_data_path + MergeTreeData::FORMAT_VERSION_FILE_NAME, getStoragePolicy()->getAnyDisk()}; + version_file = {fs::path(relative_data_path) / MergeTreeData::FORMAT_VERSION_FILE_NAME, getStoragePolicy()->getAnyDisk()}; bool version_file_exists = version_file.second->exists(version_file.first); @@ -3854,10 +3854,10 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( const String & with_name, ContextPtr local_context) { - String clickhouse_path = fs::absolute(local_context->getPath()); - String default_shadow_path = clickhouse_path + "shadow/"; + String clickhouse_path = fs::canonical(local_context->getPath()); + String default_shadow_path = fs::path(clickhouse_path) / "shadow/"; fs::create_directories(default_shadow_path); - auto increment = Increment(default_shadow_path + "increment.txt").get(true); + auto increment = Increment(fs::path(default_shadow_path) / "increment.txt").get(true); const String shadow_path = "shadow/"; @@ -3865,7 +3865,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( const auto data_parts = getDataParts(); String backup_name = (!with_name.empty() ? escapeForFileName(with_name) : toString(increment)); - String backup_path = shadow_path + backup_name + "/"; + String backup_path = fs::path(shadow_path) / backup_name / ""; for (const auto & disk : getStoragePolicy()->getDisks()) disk->onFreeze(backup_path); @@ -3882,20 +3882,20 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( part->volume->getDisk()->createDirectories(backup_path); - String backup_part_path = backup_path + relative_data_path + part->relative_path; + String backup_part_path = fs::path(backup_path) / relative_data_path / part->relative_path; if (auto part_in_memory = asInMemoryPart(part)) - part_in_memory->flushToDisk(backup_path + relative_data_path, part->relative_path, metadata_snapshot); + part_in_memory->flushToDisk(fs::path(backup_path) / relative_data_path, part->relative_path, metadata_snapshot); else localBackup(part->volume->getDisk(), part->getFullRelativePath(), backup_part_path); - part->volume->getDisk()->removeFileIfExists(backup_part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); + part->volume->getDisk()->removeFileIfExists(fs::path(backup_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); part->is_frozen.store(true, std::memory_order_relaxed); result.push_back(PartitionCommandResultInfo{ .partition_id = part->info.partition_id, .part_name = part->name, - .backup_path = part->volume->getDisk()->getPath() + backup_path, - .part_backup_path = part->volume->getDisk()->getPath() + backup_part_path, + .backup_path = fs::path(part->volume->getDisk()->getPath()) / backup_path, + .part_backup_path = fs::path(part->volume->getDisk()->getPath()) / backup_part_path, .backup_name = backup_name, }); ++parts_processed; From a2cfbd74ec22cf283c2d392699b877d8cb3d74ac Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 7 May 2021 22:00:36 +0300 Subject: [PATCH 037/154] Fix fs::copy to work the same as Poco::copy --- src/Disks/DiskLocal.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 85023dba464..7c192f70cd2 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -309,7 +309,15 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - fs::copy(fs::path(disk_path) / from_path, fs::path(to_disk->getPath()) / to_path); /// Use more optimal way. + { + fs::path from = fs::path(disk_path) / from_path; + if (from_path.ends_with('/')) + from = (fs::path(disk_path) / from_path.substr(0, from_path.size() - 1)).parent_path(); + else if (fs::is_directory(from)) + from = from.parent_path(); + + fs::copy(from, fs::path(to_disk->getPath()) / to_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way. + } else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } From 35f999bf0406a46c423ec5d87ed5359c373bf585 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 May 2021 00:53:44 +0300 Subject: [PATCH 038/154] Poco::createFile to fs::createFile --- src/Common/ErrorCodes.cpp | 2 + src/Common/Exception.cpp | 1 + src/Common/createFile.cpp | 52 +++++++++++++++++++ src/Common/createFile.h | 7 +++ src/Databases/DatabaseOnDisk.cpp | 36 ++++++------- .../MySQL/DatabaseConnectionMySQL.cpp | 18 +++---- .../PostgreSQL/DatabasePostgreSQL.cpp | 6 +-- src/Disks/DiskLocal.cpp | 3 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 11 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 20 +++++-- src/Storages/MergeTree/localBackup.cpp | 15 +++--- .../examples/remove_symlink_directory.cpp | 18 +++---- 12 files changed, 131 insertions(+), 58 deletions(-) create mode 100644 src/Common/createFile.cpp create mode 100644 src/Common/createFile.h diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ad0463db889..40be6a64336 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -550,6 +550,8 @@ M(580, CANNOT_SET_ROUNDING_MODE) \ M(581, TOO_LARGE_DISTRIBUTED_DEPTH) \ \ + M(996, OPERATION_NOT_PERMITTED) \ + M(997, CANNOT_CREATE_FILE) \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index dca19eea7f2..f17eaea8d7f 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -251,6 +251,7 @@ static std::string getExtraExceptionInfo(const std::exception & e) String msg; try { + /// TODO: this has to be adjusted for std::filesystem if (const auto * file_exception = dynamic_cast(&e)) { if (file_exception->code() == ENOSPC) diff --git a/src/Common/createFile.cpp b/src/Common/createFile.cpp new file mode 100644 index 00000000000..fc637f4d4f2 --- /dev/null +++ b/src/Common/createFile.cpp @@ -0,0 +1,52 @@ +#include "createFile.h" +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ +extern const int FILE_ALREADY_EXISTS; +extern const int OPERATION_NOT_PERMITTED; +extern const int NOT_ENOUGH_SPACE; +extern const int CANNOT_CREATE_FILE; +} +} + +namespace std::filesystem +{ +[[noreturn]] void handleLastError(const std::string & path) +{ + switch (errno) + { + case EEXIST: + throw DB::Exception(DB::ErrorCodes::FILE_ALREADY_EXISTS, "File {} already exist", path); + case EPERM: + throw DB::Exception(DB::ErrorCodes::OPERATION_NOT_PERMITTED, "Not enough permissions to create file {}", path); + case ENOSPC: + throw DB::Exception(DB::ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space to create file {}", path); + case ENAMETOOLONG: + throw DB::Exception(DB::ErrorCodes::CANNOT_CREATE_FILE, "File name {} is too long"); + default: + throw DB::Exception(DB::ErrorCodes::CANNOT_CREATE_FILE, "Cannot create file {}. Error: {}", path, strerror(errno)); + } +} + +/// Copy from Poco::createFile +bool createFile(const path & path) +{ + int n = open(path.c_str(), O_WRONLY | O_CREAT | O_EXCL, S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP | S_IROTH | S_IWOTH); + if (n != -1) + { + close(n); + return true; + } + if (n == -1 && errno == EEXIST) + return false; + handleLastError(path); +} +} diff --git a/src/Common/createFile.h b/src/Common/createFile.h new file mode 100644 index 00000000000..40ec74e6288 --- /dev/null +++ b/src/Common/createFile.h @@ -0,0 +1,7 @@ +#pragma once +#include + +namespace std::filesystem +{ +bool createFile(const path & path); +} diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 44ec5d27250..95cd47e3796 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -14,14 +14,14 @@ #include #include #include - #include -#include - #include #include #include #include +#include +#include +#include namespace fs = std::filesystem; @@ -321,10 +321,10 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr, const String & table_nam { auto table = detachTable(table_name); - Poco::File detached_permanently_flag(getObjectMetadataPath(table_name) + detached_suffix); + fs::path detached_permanently_flag(getObjectMetadataPath(table_name) + detached_suffix); try { - detached_permanently_flag.createFile(); + fs::createFile(detached_permanently_flag); } catch (Exception & e) { @@ -572,40 +572,40 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat /// Metadata files to load: name and flag for .tmp_drop files std::set> metadata_files; - Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) + fs::directory_iterator dir_end; + for (fs::directory_iterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) { + String file_name = dir_it->path().filename(); /// For '.svn', '.gitignore' directory and similar. - if (dir_it.name().at(0) == '.') + if (file_name.at(0) == '.') continue; /// There are .sql.bak files - skip them. - if (endsWith(dir_it.name(), ".sql.bak")) + if (endsWith(file_name, ".sql.bak")) continue; /// Permanently detached table flag - if (endsWith(dir_it.name(), ".sql.detached")) + if (endsWith(file_name, ".sql.detached")) continue; - if (endsWith(dir_it.name(), ".sql.tmp_drop")) + if (endsWith(file_name, ".sql.tmp_drop")) { /// There are files that we tried to delete previously - metadata_files.emplace(dir_it.name(), false); + metadata_files.emplace(file_name, false); } - else if (endsWith(dir_it.name(), ".sql.tmp")) + else if (endsWith(file_name, ".sql.tmp")) { /// There are files .sql.tmp - delete - LOG_INFO(log, "Removing file {}", dir_it->path()); + LOG_INFO(log, "Removing file {}", dir_it->path().string()); fs::remove(dir_it->path()); } - else if (endsWith(dir_it.name(), ".sql")) + else if (endsWith(file_name, ".sql")) { /// The required files have names like `table_name.sql` - metadata_files.emplace(dir_it.name(), true); + metadata_files.emplace(file_name, true); } else - throw Exception("Incorrect file extension: " + dir_it.name() + " in metadata directory " + getMetadataPath(), - ErrorCodes::INCORRECT_FILE_NAME); + throw Exception(ErrorCodes::INCORRECT_FILE_NAME, "Incorrect file extension: {} in metadata directory {}", file_name, getMetadataPath()); } /// Read and parse metadata in parallel diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 7b1bb7a0494..07da307c507 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -24,7 +24,7 @@ # include # include # include -# include +# include namespace fs = std::filesystem; @@ -408,27 +408,25 @@ void DatabaseConnectionMySQL::detachTablePermanently(ContextPtr, const String & { std::lock_guard lock{mutex}; - Poco::File remove_flag(fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix)); + fs::path remove_flag = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); if (remove_or_detach_tables.count(table_name)) - throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " is dropped", - ErrorCodes::TABLE_IS_DROPPED); + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", backQuoteIfNeed(database_name), backQuoteIfNeed(table_name)); - if (remove_flag.exists()) - throw Exception("The remove flag file already exists but the " + backQuoteIfNeed(database_name) + - "." + backQuoteIfNeed(table_name) + " does not exists remove tables, it is bug.", ErrorCodes::LOGICAL_ERROR); + if (fs::exists(remove_flag)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The remove flag file already exists but the {}.{} does not exists remove tables, it is bug.", + backQuoteIfNeed(database_name), backQuoteIfNeed(table_name)); auto table_iter = local_tables_cache.find(table_name); if (table_iter == local_tables_cache.end()) - throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", - ErrorCodes::UNKNOWN_TABLE); + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuoteIfNeed(database_name), backQuoteIfNeed(table_name)); remove_or_detach_tables.emplace(table_name); try { table_iter->second.second->drop(); - remove_flag.createFile(); + fs::createFile(remove_flag); } catch (...) { diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 9a60e579d9e..37e678a1c20 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include namespace fs = std::filesystem; @@ -253,10 +254,7 @@ void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool / throw Exception(fmt::format("Table {}.{} is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED); fs::path mark_table_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); - - FILE * file = fopen(mark_table_removed.string().data(), "a+"); - if (file == nullptr) - throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Cannot create file {}", mark_table_removed.string()); + fs::createFile(mark_table_removed); if (cache_tables) cached_tables.erase(table_name); diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 7c192f70cd2..31a8d217909 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -293,7 +294,7 @@ void DiskLocal::truncateFile(const String & path, size_t size) void DiskLocal::createFile(const String & path) { - Poco::File(fs::path(disk_path) / path).createFile(); + fs::createFile(fs::path(disk_path) / path); } void DiskLocal::setReadOnly(const String & path) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 437718954d4..02560ca3e48 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1089,11 +1089,14 @@ void IMergeTreeDataPart::remove(bool keep_s3) const { volume->getDisk()->moveDirectory(from, to); } - catch (const Poco::FileNotFoundException &) + catch (const fs::filesystem_error & e) { - LOG_ERROR(storage.log, "Directory {} (part to remove) doesn't exist or one of nested files has gone. Most likely this is due to manual removing. This should be discouraged. Ignoring.", fullPath(volume->getDisk(), to)); - - return; + if (e.code() == std::errc::no_such_file_or_directory) + { + LOG_ERROR(storage.log, "Directory {} (part to remove) doesn't exist or one of nested files has gone. Most likely this is due to manual removing. This should be discouraged. Ignoring.", fullPath(volume->getDisk(), to)); + return; + } + throw; } if (checksums.empty()) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1deed68f239..59297d0bc7c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1085,9 +1085,14 @@ void MergeTreeData::clearOldTemporaryDirectories(ssize_t custom_directories_life disk->removeRecursive(it->path()); } } - catch (const Poco::FileNotFoundException &) + catch (const fs::filesystem_error & e) { - /// If the file is already deleted, do nothing. + if (e.code() == std::errc::no_such_file_or_directory) + { + /// If the file is already deleted, do nothing. + } + else + throw; } } } @@ -1373,10 +1378,15 @@ void MergeTreeData::dropAllData() { disk->removeRecursive(path); } - catch (const Poco::FileNotFoundException &) + catch (const fs::filesystem_error & e) { - /// If the file is already deleted, log the error message and do nothing. - tryLogCurrentException(__PRETTY_FUNCTION__); + if (e.code() == std::errc::no_such_file_or_directory) + { + /// If the file is already deleted, log the error message and do nothing. + tryLogCurrentException(__PRETTY_FUNCTION__); + } + else + throw; } } diff --git a/src/Storages/MergeTree/localBackup.cpp b/src/Storages/MergeTree/localBackup.cpp index 7d7dacaeaf1..9f9f894c535 100644 --- a/src/Storages/MergeTree/localBackup.cpp +++ b/src/Storages/MergeTree/localBackup.cpp @@ -74,13 +74,16 @@ void localBackup(const DiskPtr & disk, const String & source_path, const String continue; } - catch (const Poco::FileNotFoundException &) + catch (const fs::filesystem_error & e) { - ++try_no; - if (try_no == max_tries) - throw; - - continue; + if (e.code() == std::errc::no_such_file_or_directory) + { + ++try_no; + if (try_no == max_tries) + throw; + continue; + } + throw; } break; diff --git a/src/Storages/examples/remove_symlink_directory.cpp b/src/Storages/examples/remove_symlink_directory.cpp index ae5fa72fa66..05fdc18be2f 100644 --- a/src/Storages/examples/remove_symlink_directory.cpp +++ b/src/Storages/examples/remove_symlink_directory.cpp @@ -1,9 +1,10 @@ #include #include -#include -#include #include +#include +#include +namespace fs = std::filesystem; namespace DB { @@ -16,18 +17,15 @@ namespace DB int main(int, char **) try { - Poco::File dir("./test_dir/"); - dir.createDirectories(); - - Poco::File("./test_dir/file").createFile(); + fs::path dir("./test_dir/"); + fs::create_directories(dir); + fs::createFile("./test_dir/file"); if (0 != symlink("./test_dir", "./test_link")) DB::throwFromErrnoWithPath("Cannot create symlink", "./test_link", DB::ErrorCodes::SYSTEM_ERROR); - Poco::File link("./test_link"); - link.renameTo("./test_link2"); - - Poco::File("./test_link2").remove(true); + fs::rename("./test_link", "./test_link2"); + fs::remove_all("./test_link2"); return 0; } catch (...) From 02288359c5f3cbb117a230dc3abc64afffaaf872 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 May 2021 13:59:55 +0300 Subject: [PATCH 039/154] Less manual concatenation of paths --- src/Access/AllowedClientHosts.h | 6 +- src/Common/ZooKeeper/ZooKeeper.cpp | 10 +- src/Databases/DatabaseAtomic.cpp | 4 +- src/Databases/DatabaseReplicatedWorker.cpp | 19 +- src/Storages/Distributed/DirectoryMonitor.cpp | 4 +- .../DistributedBlockOutputStream.cpp | 6 +- src/Storages/HDFS/StorageHDFS.cpp | 5 +- src/Storages/MergeTree/MergeTreeData.cpp | 32 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergeTreePartsMover.cpp | 12 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 52 +-- .../ReplicatedMergeTreeRestartingThread.cpp | 12 +- src/Storages/MergeTree/localBackup.cpp | 2 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 361 +++++++++--------- src/Storages/StorageS3.cpp | 4 +- .../get_current_inserts_in_replicated.cpp | 10 +- 17 files changed, 280 insertions(+), 265 deletions(-) diff --git a/src/Access/AllowedClientHosts.h b/src/Access/AllowedClientHosts.h index a6895b120e0..7b21fd7e236 100644 --- a/src/Access/AllowedClientHosts.h +++ b/src/Access/AllowedClientHosts.h @@ -7,7 +7,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -198,9 +200,9 @@ inline String AllowedClientHosts::IPSubnet::toString() const if (isMaskAllBitsOne()) return prefix.toString(); else if (IPAddress{prefix_length, mask.family()} == mask) - return prefix.toString() + "/" + std::to_string(prefix_length); + return fs::path(prefix.toString()) / std::to_string(prefix_length); else - return prefix.toString() + "/" + mask.toString(); + return fs::path(prefix.toString()) / mask.toString(); } inline bool AllowedClientHosts::IPSubnet::isMaskAllBitsOne() const diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 9f59da233fc..74c35f4f0e9 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -4,6 +4,7 @@ #include "TestKeeper.h" #include +#include #include #include @@ -17,6 +18,7 @@ #define ZOOKEEPER_CONNECTION_TIMEOUT_MS 1000 +namespace fs = std::filesystem; namespace DB { @@ -593,7 +595,7 @@ void ZooKeeper::removeChildren(const std::string & path) Coordination::Requests ops; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { - ops.emplace_back(makeRemoveRequest(path + "/" + children.back(), -1)); + ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1)); children.pop_back(); } multi(ops); @@ -609,9 +611,9 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path, const String & Coordination::Requests ops; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { - removeChildrenRecursive(path + "/" + children.back()); + removeChildrenRecursive(fs::path(path) / children.back()); if (likely(keep_child_node.empty() || keep_child_node != children.back())) - ops.emplace_back(makeRemoveRequest(path + "/" + children.back(), -1)); + ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1)); children.pop_back(); } multi(ops); @@ -629,7 +631,7 @@ void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, const Strin Strings batch; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { - String child_path = path + "/" + children.back(); + String child_path = fs::path(path) / children.back(); tryRemoveChildrenRecursive(child_path); if (likely(keep_child_node.empty() || keep_child_node != children.back())) { diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index fa4e2eed382..0dae31ac906 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -37,8 +37,8 @@ public: DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, ContextPtr context_) : DatabaseOrdinary(name_, std::move(metadata_path_), "store/", logger_name, context_) - , path_to_table_symlinks(getContext()->getPath() + "data/" + escapeForFileName(name_) + "/") - , path_to_metadata_symlink(getContext()->getPath() + "metadata/" + escapeForFileName(name_)) + , path_to_table_symlinks(fs::path(getContext()->getPath()) / "data" / escapeForFileName(name_) / "") + , path_to_metadata_symlink(fs::path(getContext()->getPath()) / "metadata" / escapeForFileName(name_)) , db_uuid(uuid) { assert(db_uuid != UUIDHelpers::Nil); diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 9ae4d026bf0..760300d6750 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -1,6 +1,9 @@ #include #include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -156,7 +159,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na } } - UInt32 our_log_ptr = parse(current_zookeeper->get(database->replica_path + "/log_ptr")); + UInt32 our_log_ptr = parse(current_zookeeper->get(fs::path(database->replica_path) / "log_ptr")); UInt32 entry_num = DatabaseReplicatedTask::getLogEntryNumber(entry_name); if (entry_num <= our_log_ptr) @@ -165,13 +168,13 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na return {}; } - String entry_path = queue_dir + "/" + entry_name; + String entry_path = fs::path(queue_dir) / entry_name; auto task = std::make_unique(entry_name, entry_path, database); String initiator_name; zkutil::EventPtr wait_committed_or_failed = std::make_shared(); - String try_node_path = entry_path + "/try"; + String try_node_path = fs::path(entry_path) / "try"; if (zookeeper->tryGet(try_node_path, initiator_name, nullptr, wait_committed_or_failed)) { task->is_initial_query = initiator_name == task->host_id_str; @@ -203,7 +206,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) throw Coordination::Exception(code, try_node_path); - if (!zookeeper->exists(entry_path + "/committed")) + if (!zookeeper->exists(fs::path(entry_path) / "committed")) { out_reason = fmt::format("Entry {} was forcefully cancelled due to timeout", entry_name); return {}; @@ -212,7 +215,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na } } - if (!zookeeper->exists(entry_path + "/committed")) + if (!zookeeper->exists(fs::path(entry_path) / "committed")) { out_reason = fmt::format("Entry {} hasn't been committed", entry_name); return {}; @@ -220,8 +223,8 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na if (task->is_initial_query) { - assert(!zookeeper->exists(entry_path + "/try")); - assert(zookeeper->exists(entry_path + "/committed") == (zookeeper->get(task->getFinishedNodePath()) == ExecutionStatus(0).serializeText())); + assert(!zookeeper->exists(fs::path(entry_path) / "try")); + assert(zookeeper->exists(fs::path(entry_path) / "committed") == (zookeeper->get(task->getFinishedNodePath()) == ExecutionStatus(0).serializeText())); out_reason = fmt::format("Entry {} has been executed as initial query", entry_name); return {}; } @@ -257,7 +260,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na bool DatabaseReplicatedDDLWorker::canRemoveQueueEntry(const String & entry_name, const Coordination::Stat &) { UInt32 entry_number = DDLTaskBase::getLogEntryNumber(entry_name); - UInt32 max_log_ptr = parse(getAndSetZooKeeper()->get(database->zookeeper_path + "/max_log_ptr")); + UInt32 max_log_ptr = parse(getAndSetZooKeeper()->get(fs::path(database->zookeeper_path) / "max_log_ptr")); return entry_number + logs_to_keep < max_log_ptr; } diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index ce19d3ec89f..bae81a56b30 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -293,7 +293,7 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( , pool(std::move(pool_)) , disk(disk_) , relative_path(relative_path_) - , path(disk->getPath() + relative_path + '/') + , path(fs::path(disk->getPath()) / relative_path / "") , should_batch_inserts(storage.getContext()->getSettingsRef().distributed_directory_monitor_batch_inserts) , dir_fsync(storage.getDistributedSettingsRef().fsync_directories) , min_batched_block_size_rows(storage.getContext()->getSettingsRef().min_insert_block_size_rows) @@ -1023,7 +1023,7 @@ void StorageDistributedDirectoryMonitor::updatePath(const std::string & new_rela { std::lock_guard status_lock(status_mutex); relative_path = new_relative_path; - path = disk->getPath() + relative_path + '/'; + path = fs::path(disk->getPath()) / relative_path / ""; } current_batch_file_path = path + "current_batch.txt"; diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 2f0187fc115..ea694fb0cfe 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -717,7 +717,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: } // Create hardlink here to reuse increment number - const std::string block_file_path(path + '/' + file_name); + const std::string block_file_path(fs::path(path) / file_name); createHardLink(first_file_tmp_path, block_file_path); auto dir_sync_guard = make_directory_sync_guard(*it); } @@ -726,10 +726,10 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: /// Make hardlinks for (; it != dir_names.end(); ++it) { - const std::string path(disk_path + data_path + *it); + const std::string path(fs::path(disk_path) / (data_path + *it)); fs::create_directory(path); - const std::string block_file_path(path + '/' + toString(storage.file_names_increment.get()) + ".bin"); + const std::string block_file_path(fs::path(path) / (toString(storage.file_names_increment.get()) + ".bin")); createHardLink(first_file_tmp_path, block_file_path); auto dir_sync_guard = make_directory_sync_guard(*it); } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index c08e487f179..4f1aec29e6b 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -18,7 +18,6 @@ #include #include #include - #include #include #include @@ -26,7 +25,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -250,7 +251,7 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c { if (re2::RE2::FullMatch(file_name, matcher)) { - Strings result_part = LSWithRegexpMatching(full_path + "/", fs, suffix_with_globs.substr(next_slash)); + Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash)); /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 59297d0bc7c..1849b2efebd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -852,8 +852,8 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) auto part = createPart(part_name, part_info, single_disk_volume, part_name); bool broken = false; - String part_path = relative_data_path + "/" + part_name; - String marker_path = part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; + String part_path = fs::path(relative_data_path) / part_name; + String marker_path = fs::path(part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; if (part_disk_ptr->exists(marker_path)) { LOG_WARNING(log, "Detaching stale part {}{}, which should have been deleted after a move. That can only happen after unclean restart of ClickHouse after move of a part having an operation blocking that stale copy of part.", getFullPathOnDisk(part_disk_ptr), part_name); @@ -941,7 +941,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) else has_adaptive_parts.store(true, std::memory_order_relaxed); - part->modification_time = part_disk_ptr->getLastModified(relative_data_path + part_name).epochTime(); + part->modification_time = part_disk_ptr->getLastModified(fs::path(relative_data_path) / part_name).epochTime(); /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later part->setState(DataPartState::Committed); @@ -1409,8 +1409,8 @@ void MergeTreeData::dropIfEmpty() for (const auto & [path, disk] : getRelativeDataPathsWithDisks()) { /// Non recursive, exception is thrown if there are more files. - disk->removeFileIfExists(path + MergeTreeData::FORMAT_VERSION_FILE_NAME); - disk->removeDirectory(path + MergeTreeData::DETACHED_DIR_NAME); + disk->removeFileIfExists(fs::path(path) / MergeTreeData::FORMAT_VERSION_FILE_NAME); + disk->removeDirectory(fs::path(path) / MergeTreeData::DETACHED_DIR_NAME); disk->removeDirectory(path); } } @@ -1854,7 +1854,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart( const VolumePtr & volume, const String & relative_path) const { MergeTreeDataPartType type; - auto full_path = relative_data_path + relative_path + "/"; + auto full_path = fs::path(relative_data_path) / relative_path / ""; auto mrk_ext = MergeTreeIndexGranularityInfo::getMarksExtensionFromFilesystem(volume->getDisk(), full_path); if (mrk_ext) @@ -1907,7 +1907,7 @@ void MergeTreeData::changeSettings( { auto disk = new_storage_policy->getDiskByName(disk_name); disk->createDirectories(relative_data_path); - disk->createDirectories(relative_data_path + MergeTreeData::DETACHED_DIR_NAME); + disk->createDirectories(fs::path(relative_data_path) / MergeTreeData::DETACHED_DIR_NAME); } /// FIXME how would that be done while reloading configuration??? @@ -1936,7 +1936,7 @@ void MergeTreeData::PartsTemporaryRename::addPart(const String & old_name, const old_and_new_names.push_back({old_name, new_name}); for (const auto & [path, disk] : storage.getRelativeDataPathsWithDisks()) { - for (auto it = disk->iterateDirectory(path + source_dir); it->isValid(); it->next()) + for (auto it = disk->iterateDirectory(fs::path(path) / source_dir); it->isValid(); it->next()) { if (it->name() == old_name) { @@ -1958,8 +1958,8 @@ void MergeTreeData::PartsTemporaryRename::tryRenameAll() if (old_name.empty() || new_name.empty()) throw DB::Exception("Empty part name. Most likely it's a bug.", ErrorCodes::INCORRECT_FILE_NAME); const auto & [path, disk] = old_part_name_to_path_and_disk[old_name]; - const auto full_path = path + source_dir; /// for old_name - disk->moveFile(full_path + old_name, full_path + new_name); + const auto full_path = fs::path(path) / source_dir; /// for old_name + disk->moveFile(fs::path(full_path) / old_name, fs::path(full_path) / new_name); } catch (...) { @@ -1983,8 +1983,8 @@ MergeTreeData::PartsTemporaryRename::~PartsTemporaryRename() try { const auto & [path, disk] = old_part_name_to_path_and_disk[old_name]; - const auto full_path = path + source_dir; /// for old_name - disk->moveFile(full_path + new_name, full_path + old_name); + const String full_path = fs::path(path) / source_dir; /// for old_name + disk->moveFile(fs::path(full_path) / new_name, fs::path(full_path) / old_name); } catch (...) { @@ -2672,7 +2672,7 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy) addPartContributionToDataVolume(part_copy); auto disk = original_active_part->volume->getDisk(); - String marker_path = original_active_part->getFullRelativePath() + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; + String marker_path = fs::path(original_active_part->getFullRelativePath()) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; try { disk->createFile(marker_path); @@ -3747,12 +3747,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( const auto & src_relative_data_path = src_part_in_memory->storage.relative_data_path; auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); src_part_in_memory->flushToDisk(src_relative_data_path, flushed_part_path, metadata_snapshot); - src_part_path = src_relative_data_path + flushed_part_path + "/"; + src_part_path = fs::path(src_relative_data_path) / flushed_part_path / ""; } LOG_DEBUG(log, "Cloning part {} to {}", fullPath(disk, src_part_path), fullPath(disk, dst_part_path)); localBackup(disk, src_part_path, dst_part_path); - disk->removeFileIfExists(dst_part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); + disk->removeFileIfExists(fs::path(dst_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); auto dst_data_part = createPart(dst_part_name, dst_part_info, single_disk_volume, tmp_dst_part_name); @@ -3934,7 +3934,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezeAll( PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn matcher, const String & backup_name, ContextPtr) { - auto backup_path = std::filesystem::path("shadow") / escapeForFileName(backup_name) / relative_data_path; + auto backup_path = fs::path("shadow") / escapeForFileName(backup_name) / relative_data_path; LOG_DEBUG(log, "Unfreezing parts by path {}", backup_path.generic_string()); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ac078eee6bd..8debf961b89 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -133,7 +133,7 @@ using RelativeSize = boost::rational; static std::string toString(const RelativeSize & x) { - return ASTSampleRatio::toString(x.numerator()) + "/" + ASTSampleRatio::toString(x.denominator()); + return fs::path(ASTSampleRatio::toString(x.numerator())) / ASTSampleRatio::toString(x.denominator()); } /// Converts sample size to an approximate number of rows (ex. `SAMPLE 1000000`) to relative value (ex. `SAMPLE 0.1`). diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index f9e3883d5e2..5b77ac9ec4a 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -206,18 +206,18 @@ MergeTreeData::DataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEnt /// Try to fetch part from S3 without copy and fallback to default copy /// if it's not possible moving_part.part->assertOnDisk(); - String path_to_clone = data->getRelativeDataPath() + directory_to_move + "/"; + String path_to_clone = fs::path(data->getRelativeDataPath()) / directory_to_move / ""; String relative_path = part->relative_path; if (disk->exists(path_to_clone + relative_path)) { LOG_WARNING(log, "Path " + fullPath(disk, path_to_clone + relative_path) + " already exists. Will remove it and clone again."); - disk->removeRecursive(path_to_clone + relative_path + "/"); + disk->removeRecursive(fs::path(path_to_clone) / relative_path / ""); } disk->createDirectories(path_to_clone); - bool is_fetched = data->tryToFetchIfShared(*part, disk, path_to_clone + "/" + part->name); + bool is_fetched = data->tryToFetchIfShared(*part, disk, fs::path(path_to_clone) / part->name); if (!is_fetched) - part->volume->getDisk()->copy(data->getRelativeDataPath() + relative_path + "/", disk, path_to_clone); - part->volume->getDisk()->removeFileIfExists(path_to_clone + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); + part->volume->getDisk()->copy(fs::path(data->getRelativeDataPath()) / relative_path / "", disk, path_to_clone); + part->volume->getDisk()->removeFileIfExists(fs::path(path_to_clone) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); } else { @@ -226,7 +226,7 @@ MergeTreeData::DataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEnt auto single_disk_volume = std::make_shared("volume_" + part->name, moving_part.reserved_space->getDisk(), 0); MergeTreeData::MutableDataPartPtr cloned_part = - data->createPart(part->name, single_disk_volume, directory_to_move + '/' + part->name); + data->createPart(part->name, single_disk_volume, fs::path(directory_to_move) / part->name); LOG_TRACE(log, "Part {} was cloned to {}", part->name, cloned_part->getFullPath()); cloned_part->loadColumnsChecksumsIndexes(true, true); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index ad41bbe1a08..f24e0b7b87d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -63,7 +63,7 @@ bool ReplicatedMergeTreeQueue::isVirtualPart(const MergeTreeData::DataPartPtr & bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) { - auto queue_path = replica_path + "/queue"; + String queue_path = fs::path(replica_path) / "queue"; LOG_DEBUG(log, "Loading queue from {}", queue_path); bool updated = false; @@ -75,7 +75,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) /// Reset batch size on initialization to recover from possible errors of too large batch size. current_multi_batch_size = 1; - String log_pointer_str = zookeeper->get(replica_path + "/log_pointer"); + String log_pointer_str = zookeeper->get(fs::path(replica_path) / "log_pointer"); log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); std::unordered_set already_loaded_paths; @@ -102,7 +102,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) futures.reserve(children.size()); for (const String & child : children) - futures.emplace_back(child, zookeeper->asyncGet(queue_path + "/" + child)); + futures.emplace_back(child, zookeeper->asyncGet(fs::path(queue_path) / child)); for (auto & future : futures) { @@ -117,7 +117,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) updated = true; } - zookeeper->tryGet(replica_path + "/mutation_pointer", mutation_pointer); + zookeeper->tryGet(fs::path(replica_path) / "mutation_pointer", mutation_pointer); } updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, {}); @@ -401,7 +401,7 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep if (!need_remove_from_zk) return; - auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name); + auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / entry->znode_name); if (code != Coordination::Error::ZOK) LOG_ERROR(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, Coordination::errorMessage(code)); @@ -457,7 +457,7 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri notifySubscribers(queue_size); - zookeeper->tryRemove(replica_path + "/queue/" + found->znode_name); + zookeeper->tryRemove(fs::path(replica_path) / "queue" / found->znode_name); updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed); return true; @@ -476,14 +476,14 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper if (pull_log_blocker.isCancelled()) throw Exception("Log pulling is cancelled", ErrorCodes::ABORTED); - String index_str = zookeeper->get(replica_path + "/log_pointer"); + String index_str = zookeeper->get(fs::path(replica_path) / "log_pointer"); UInt64 index; /// The version of "/log" is modified when new entries to merge/mutate/drop appear. Coordination::Stat stat; - zookeeper->get(zookeeper_path + "/log", &stat); + zookeeper->get(fs::path(zookeeper_path) / "log", &stat); - Strings log_entries = zookeeper->getChildrenWatch(zookeeper_path + "/log", nullptr, watch_callback); + Strings log_entries = zookeeper->getChildrenWatch(fs::path(zookeeper_path) / "log", nullptr, watch_callback); /// We update mutations after we have loaded the list of log entries, but before we insert them /// in the queue. @@ -496,7 +496,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper /// If we do not already have a pointer to the log, put a pointer to the first entry in it. index = log_entries.empty() ? 0 : parse(std::min_element(log_entries.begin(), log_entries.end())->substr(strlen("log-"))); - zookeeper->set(replica_path + "/log_pointer", toString(index)); + zookeeper->set(fs::path(replica_path) / "log_pointer", toString(index)); } else { @@ -543,7 +543,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper futures.reserve(end - begin); for (auto it = begin; it != end; ++it) - futures.emplace_back(*it, zookeeper->asyncGet(zookeeper_path + "/log/" + *it)); + futures.emplace_back(*it, zookeeper->asyncGet(fs::path(zookeeper_path) / "log" / *it)); /// Simultaneously add all new entries to the queue and move the pointer to the log. @@ -560,7 +560,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper copied_entries.emplace_back(LogEntry::parse(res.data, res.stat)); ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/queue/queue-", res.data, zkutil::CreateMode::PersistentSequential)); + fs::path(replica_path) / "queue/queue-", res.data, zkutil::CreateMode::PersistentSequential)); const auto & entry = *copied_entries.back(); if (entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART) @@ -575,11 +575,11 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper } ops.emplace_back(zkutil::makeSetRequest( - replica_path + "/log_pointer", toString(last_entry_index + 1), -1)); + fs::path(replica_path) / "log_pointer", toString(last_entry_index + 1), -1)); if (min_unprocessed_insert_time_changed) ops.emplace_back(zkutil::makeSetRequest( - replica_path + "/min_unprocessed_insert_time", toString(*min_unprocessed_insert_time_changed), -1)); + fs::path(replica_path) / "min_unprocessed_insert_time", toString(*min_unprocessed_insert_time_changed), -1)); auto responses = zookeeper->multi(ops); @@ -657,7 +657,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C { std::lock_guard lock(update_mutations_mutex); - Strings entries_in_zk = zookeeper->getChildrenWatch(zookeeper_path + "/mutations", nullptr, watch_callback); + Strings entries_in_zk = zookeeper->getChildrenWatch(fs::path(zookeeper_path) / "mutations", nullptr, watch_callback); StringSet entries_in_zk_set(entries_in_zk.begin(), entries_in_zk.end()); /// Compare with the local state, delete obsolete entries and determine which new entries to load. @@ -714,7 +714,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C std::vector> futures; for (const String & entry : entries_to_load) - futures.emplace_back(zookeeper->asyncGet(zookeeper_path + "/mutations/" + entry)); + futures.emplace_back(zookeeper->asyncGet(fs::path(zookeeper_path) / "mutations" / entry)); std::vector new_mutations; for (size_t i = 0; i < entries_to_load.size(); ++i) @@ -798,7 +798,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( { std::lock_guard lock(update_mutations_mutex); - auto rc = zookeeper->tryRemove(zookeeper_path + "/mutations/" + mutation_id); + auto rc = zookeeper->tryRemove(fs::path(zookeeper_path) / "mutations" / mutation_id); if (rc == Coordination::Error::ZOK) LOG_DEBUG(log, "Removed mutation {} from ZooKeeper.", mutation_id); @@ -935,12 +935,12 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( { if ((*it)->currently_executing) to_wait.push_back(*it); - auto code = zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name); + auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / (*it)->znode_name); /// FIXME it's probably unsafe to remove entries non-atomically /// when this method called directly from alter query (not from replication queue task), /// because entries will be lost if ALTER fails. if (code != Coordination::Error::ZOK) - LOG_INFO(log, "Couldn't remove {}: {}", replica_path + "/queue/" + (*it)->znode_name, Coordination::errorMessage(code)); + LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / (*it)->znode_name).string(), Coordination::errorMessage(code)); updateStateOnQueueEntryRemoval( *it, /* is_successful = */ false, @@ -1593,7 +1593,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep if (!finished.empty()) { - zookeeper->set(replica_path + "/mutation_pointer", finished.back()->znode_name); + zookeeper->set(fs::path(replica_path) / "mutation_pointer", finished.back()->znode_name); std::lock_guard lock(state_mutex); @@ -1796,22 +1796,22 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( } /// Load current quorum status. - auto quorum_status_future = zookeeper->asyncTryGet(queue.zookeeper_path + "/quorum/status"); + auto quorum_status_future = zookeeper->asyncTryGet(fs::path(queue.zookeeper_path) / "quorum" / "status"); /// Load current inserts std::unordered_set lock_holder_paths; - for (const String & entry : zookeeper->getChildren(queue.zookeeper_path + "/temp")) + for (const String & entry : zookeeper->getChildren(fs::path(queue.zookeeper_path) / "temp")) { if (startsWith(entry, "abandonable_lock-")) - lock_holder_paths.insert(queue.zookeeper_path + "/temp/" + entry); + lock_holder_paths.insert(fs::path(queue.zookeeper_path) / "temp" / entry); } if (!lock_holder_paths.empty()) { - Strings partitions = zookeeper->getChildren(queue.zookeeper_path + "/block_numbers"); + Strings partitions = zookeeper->getChildren(fs::path(queue.zookeeper_path) / "block_numbers"); std::vector> lock_futures; for (const String & partition : partitions) - lock_futures.push_back(zookeeper->asyncGetChildren(queue.zookeeper_path + "/block_numbers/" + partition)); + lock_futures.push_back(zookeeper->asyncGetChildren(fs::path(queue.zookeeper_path) / "block_numbers" / partition)); struct BlockInfoInZooKeeper { @@ -1832,7 +1832,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( if (startsWith(entry, "block-")) { Int64 block_number = parse(entry.substr(strlen("block-"))); - String zk_path = queue.zookeeper_path + "/block_numbers/" + partitions[i] + "/" + entry; + String zk_path = fs::path(queue.zookeeper_path) / "block_numbers" / partitions[i] / entry; block_infos.emplace_back( BlockInfoInZooKeeper{partitions[i], block_number, zk_path, zookeeper->asyncTryGet(zk_path)}); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index ca6ea3103d1..df57499ae90 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -230,7 +230,7 @@ void ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart() auto zookeeper = storage.getZooKeeper(); String quorum_str; - if (zookeeper->tryGet(storage.zookeeper_path + "/quorum/status", quorum_str)) + if (zookeeper->tryGet(fs::path(storage.zookeeper_path) / "quorum" / "status", quorum_str)) { ReplicatedMergeTreeQuorumEntry quorum_entry(quorum_str); @@ -243,12 +243,12 @@ void ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart() } Strings part_names; - String parallel_quorum_parts_path = storage.zookeeper_path + "/quorum/parallel"; + String parallel_quorum_parts_path = fs::path(storage.zookeeper_path) / "quorum" / "parallel"; if (zookeeper->tryGetChildren(parallel_quorum_parts_path, part_names) == Coordination::Error::ZOK) { for (auto & part_name : part_names) { - if (zookeeper->tryGet(parallel_quorum_parts_path + "/" + part_name, quorum_str)) + if (zookeeper->tryGet(fs::path(parallel_quorum_parts_path) / part_name, quorum_str)) { ReplicatedMergeTreeQuorumEntry quorum_entry(quorum_str); if (!quorum_entry.replicas.count(storage.replica_name) @@ -270,7 +270,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() /// How other replicas can access this one. ReplicatedMergeTreeAddress address = storage.getReplicatedMergeTreeAddress(); - String is_active_path = storage.replica_path + "/is_active"; + String is_active_path = fs::path(storage.replica_path) / "is_active"; /** If the node is marked as active, but the mark is made in the same instance, delete it. * This is possible only when session in ZooKeeper expires. @@ -294,7 +294,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() /// Simultaneously declare that this replica is active, and update the host. Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral)); - ops.emplace_back(zkutil::makeSetRequest(storage.replica_path + "/host", address.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(storage.replica_path) / "host", address.toString(), -1)); try { @@ -303,7 +303,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() catch (const Coordination::Exception & e) { String existing_replica_host; - zookeeper->tryGet(storage.replica_path + "/host", existing_replica_host); + zookeeper->tryGet(fs::path(storage.replica_path) / "host", existing_replica_host); if (existing_replica_host.empty()) existing_replica_host = "without host node"; diff --git a/src/Storages/MergeTree/localBackup.cpp b/src/Storages/MergeTree/localBackup.cpp index 9f9f894c535..2e02481e065 100644 --- a/src/Storages/MergeTree/localBackup.cpp +++ b/src/Storages/MergeTree/localBackup.cpp @@ -29,7 +29,7 @@ static void localBackupImpl(const DiskPtr & disk, const String & source_path, co for (auto it = disk->iterateDirectory(source_path); it->isValid(); it->next()) { auto source = it->path(); - auto destination = destination_path + "/" + it->name(); + auto destination = fs::path(destination_path) / it->name(); if (!disk->isDirectory(source)) { diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index be052f91b39..46b95908e2c 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -99,7 +99,7 @@ std::vector listFilesWithRegexpMatching(const std::string & path_fo if (re2::RE2::FullMatch(file_name, matcher)) { /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. - Strings result_part = listFilesWithRegexpMatching(full_path + "/", suffix_with_globs.substr(next_slash)); + Strings result_part = listFilesWithRegexpMatching(fs::path(full_path) / "", suffix_with_globs.substr(next_slash)); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } } @@ -200,7 +200,7 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu if (args.format_name == "Distributed") throw Exception("Distributed format is allowed only with explicit file path", ErrorCodes::INCORRECT_FILE_NAME); - String table_dir_path = base_path + relative_table_dir_path + "/"; + String table_dir_path = fs::path(base_path) / relative_table_dir_path / ""; fs::create_directories(table_dir_path); paths = {getTablePath(table_dir_path, format_name)}; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 864c31ec05d..38d0f75bfe5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -66,9 +66,12 @@ #include #include #include +#include #include +namespace fs = std::filesystem; + namespace ProfileEvents { extern const Event ReplicatedPartMerges; @@ -848,12 +851,12 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper for (const auto & child : children) if (child != "dropped") - zookeeper->tryRemoveRecursive(zookeeper_path + "/" + child); + zookeeper->tryRemoveRecursive(fs::path(zookeeper_path) / child); Coordination::Requests ops; Coordination::Responses responses; ops.emplace_back(zkutil::makeRemoveRequest(metadata_drop_lock->getPath(), -1)); - ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/dropped", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(zookeeper_path) / "dropped", -1)); ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path, -1)); code = zookeeper->tryMulti(ops, responses); @@ -892,12 +895,12 @@ void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr ReplicatedMergeTreeTableMetadata old_metadata(*this, metadata_snapshot); Coordination::Stat metadata_stat; - String metadata_str = zookeeper->get(zookeeper_prefix + "/metadata", &metadata_stat); + String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata", &metadata_stat); auto metadata_from_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str); old_metadata.checkEquals(metadata_from_zk, metadata_snapshot->getColumns(), getContext()); Coordination::Stat columns_stat; - auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_prefix + "/columns", &columns_stat)); + auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns", &columns_stat)); const ColumnsDescription & old_columns = metadata_snapshot->getColumns(); if (columns_from_zk != old_columns) @@ -1039,7 +1042,7 @@ static time_t tryGetPartCreateTime(zkutil::ZooKeeperPtr & zookeeper, const Strin /// We get creation time of part, if it still exists (was not merged, for example). Coordination::Stat stat; String unused; - if (zookeeper->tryGet(replica_path + "/parts/" + part_name, unused, &stat)) + if (zookeeper->tryGet(fs::path(replica_path) / "parts" / part_name, unused, &stat)) res = stat.ctime / 1000; return res; @@ -1050,7 +1053,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) { auto zookeeper = getZooKeeper(); - Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts"); + Strings expected_parts_vec = zookeeper->getChildren(fs::path(replica_path) / "parts"); /// Parts in ZK. NameSet expected_parts(expected_parts_vec.begin(), expected_parts_vec.end()); @@ -1153,7 +1156,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) exists_futures.reserve(parts_to_fetch.size()); for (const String & part_name : parts_to_fetch) { - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; exists_futures.emplace_back(zookeeper->asyncExists(part_path)); } @@ -1182,7 +1185,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) /// We assume that this occurs before the queue is loaded (queue.initialize). ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + fs::path(replica_path) / "queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); enqueue_futures.emplace_back(zookeeper->asyncMulti(ops)); } @@ -1208,13 +1211,13 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: auto local_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums( part->getColumns(), part->checksums); - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); bool has_been_already_added = false; for (const String & replica : replicas) { - String current_part_path = zookeeper_path + "/replicas/" + replica + "/parts/" + part_name; + String current_part_path = fs::path(zookeeper_path) / "replicas" / replica / "parts" / part_name; String part_zk_str; if (!zookeeper->tryGet(current_part_path, part_zk_str)) @@ -1235,9 +1238,9 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: String checksums_str; /// Let's check that the node's version with the columns did not change while we were reading the checksums. /// This ensures that the columns and the checksum refer to the same - if (!zookeeper->tryGet(current_part_path + "/columns", columns_str, &columns_stat_before) || - !zookeeper->tryGet(current_part_path + "/checksums", checksums_str) || - !zookeeper->exists(current_part_path + "/columns", &columns_stat_after) || + if (!zookeeper->tryGet(fs::path(current_part_path) / "columns", columns_str, &columns_stat_before) || + !zookeeper->tryGet(fs::path(current_part_path) / "checksums", checksums_str) || + !zookeeper->exists(fs::path(current_part_path) / "columns", &columns_stat_after) || columns_stat_before.version != columns_stat_after.version) { LOG_INFO(log, "Not checking checksums of part {} with replica {} because part changed while we were reading its checksums", part_name, replica); @@ -1271,7 +1274,7 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: if (!has_been_already_added) { const auto storage_settings_ptr = getSettings(); - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; //ops.emplace_back(zkutil::makeCheckRequest( // zookeeper_path + "/columns", expected_columns_version)); @@ -1286,14 +1289,15 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: ops.emplace_back(zkutil::makeCreateRequest( part_path, "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - part_path + "/columns", part->getColumns().toString(), zkutil::CreateMode::Persistent)); + fs::path(part_path) / "columns", part->getColumns().toString(), zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - part_path + "/checksums", getChecksumsForZooKeeper(part->checksums), zkutil::CreateMode::Persistent)); + fs::path(part_path) / "checksums", getChecksumsForZooKeeper(part->checksums), zkutil::CreateMode::Persistent)); } } else { - LOG_WARNING(log, "checkPartAndAddToZooKeeper: node {} already exists. Will not commit any nodes.", replica_path + "/parts/" + part_name); + LOG_WARNING(log, "checkPartAndAddToZooKeeper: node {} already exists. Will not commit any nodes.", + (fs::path(replica_path) / "parts" / part_name).string()); } } @@ -1361,7 +1365,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo const String part_new_name = actual_part_info.getPartName(); for (const DiskPtr & disk : getStoragePolicy()->getDisks()) - for (const auto it = disk->iterateDirectory(relative_data_path + "detached/"); it->isValid(); it->next()) + for (const auto it = disk->iterateDirectory(fs::path(relative_data_path) / "detached/"); it->isValid(); it->next()) { MergeTreePartInfo part_info; @@ -1370,7 +1374,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo continue; const String part_old_name = part_info.getPartName(); - const String part_path = "detached/" + part_old_name; + const String part_path = fs::path("detached") / part_old_name; const VolumePtr volume = std::make_shared("volume_" + part_old_name, disk); @@ -1425,7 +1429,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) existing_part = getActiveContainingPart(entry.new_part_name); /// Even if the part is local, it (in exceptional cases) may not be in ZooKeeper. Let's check that it is there. - if (existing_part && getZooKeeper()->exists(replica_path + "/parts/" + existing_part->name)) + if (existing_part && getZooKeeper()->exists(fs::path(replica_path) / "parts" / existing_part->name)) { if (!is_get_or_attach || entry.source_replica != replica_name) LOG_DEBUG(log, "Skipping action for part {} because part {} already exists.", @@ -1460,7 +1464,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) /// Perhaps we don't need this part, because during write with quorum, the quorum has failed /// (see below about `/quorum/failed_parts`). - if (entry.quorum && getZooKeeper()->exists(zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name)) + if (entry.quorum && getZooKeeper()->exists(fs::path(zookeeper_path) / "quorum" / "failed_parts" / entry.new_part_name)) { LOG_DEBUG(log, "Skipping action for part {} because quorum for that part was failed.", entry.new_part_name); return true; /// NOTE Deletion from `virtual_parts` is not done, but it is only necessary for merge. @@ -1916,14 +1920,14 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) auto zookeeper = getZooKeeper(); - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); Coordination::Requests ops; for (const auto & path_part : replicas) { Coordination::Stat stat; - String path = zookeeper_path + "/replicas/" + path_part + "/host"; + String path = fs::path(zookeeper_path) / "replicas" / path_part / "host"; zookeeper->get(path, &stat); ops.emplace_back(zkutil::makeCheckRequest(path, stat.version)); } @@ -1937,8 +1941,8 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) if (replica.empty()) { Coordination::Stat quorum_stat; - const String quorum_unparallel_path = zookeeper_path + "/quorum/status"; - const String quorum_parallel_path = zookeeper_path + "/quorum/parallel/" + entry.new_part_name; + const String quorum_unparallel_path = fs::path(zookeeper_path) / "quorum" / "status"; + const String quorum_parallel_path = fs::path(zookeeper_path) / "quorum" / "parallel" / entry.new_part_name; String quorum_str, quorum_path; ReplicatedMergeTreeQuorumEntry quorum_entry; @@ -1962,13 +1966,13 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) ErrorCodes::LOGICAL_ERROR); ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name, + fs::path(zookeeper_path) / "quorum" / "failed_parts" / entry.new_part_name, "", zkutil::CreateMode::Persistent)); /// Deleting from `blocks`. - if (!entry.block_id.empty() && zookeeper->exists(zookeeper_path + "/blocks/" + entry.block_id)) - ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/blocks/" + entry.block_id, -1)); + if (!entry.block_id.empty() && zookeeper->exists(fs::path(zookeeper_path) / "blocks" / entry.block_id)) + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(zookeeper_path) / "blocks" / entry.block_id, -1)); Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); @@ -1981,7 +1985,8 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) } else if (code == Coordination::Error::ZBADVERSION || code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { - LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part {} as failed. Code: {}", entry.new_part_name, Coordination::errorMessage(code)); + LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part {} as failed. Code: {}", + entry.new_part_name, Coordination::errorMessage(code)); } else throw Coordination::Exception(code); @@ -2003,7 +2008,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) try { String part_name = entry.actual_new_part_name.empty() ? entry.new_part_name : entry.actual_new_part_name; - if (!fetchPart(part_name, metadata_snapshot, zookeeper_path + "/replicas/" + replica, false, entry.quorum)) + if (!fetchPart(part_name, metadata_snapshot, fs::path(zookeeper_path) / "replicas" / replica, false, entry.quorum)) return false; } catch (Exception & e) @@ -2087,7 +2092,7 @@ bool StorageReplicatedMergeTree::executeFetchShared( try { - if (!fetchExistsPart(new_part_name, metadata_snapshot, zookeeper_path + "/replicas/" + source_replica, disk, path)) + if (!fetchExistsPart(new_part_name, metadata_snapshot, fs::path(zookeeper_path) / "replicas" / source_replica, disk, path)) return false; } catch (Exception & e) @@ -2406,8 +2411,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } else if (!part_desc->replica.empty()) { - String source_replica_path = zookeeper_path + "/replicas/" + part_desc->replica; - ReplicatedMergeTreeAddress address(getZooKeeper()->get(source_replica_path + "/host")); + String source_replica_path = fs::path(zookeeper_path) / "replicas" / part_desc->replica; + ReplicatedMergeTreeAddress address(getZooKeeper()->get(fs::path(source_replica_path) / "host")); auto timeouts = getFetchPartHTTPTimeouts(getContext()); auto credentials = getContext()->getInterserverCredentials(); @@ -2486,18 +2491,18 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coordination::Stat source_is_lost_stat, zkutil::ZooKeeperPtr & zookeeper) { - String source_path = zookeeper_path + "/replicas/" + source_replica; + String source_path = fs::path(zookeeper_path) / "replicas" / source_replica; /** TODO: it will be deleted! (It is only to support old version of CH server). * In current code, the replica is created in single transaction. * If the reference/master replica is not yet fully created, let's wait. */ - while (!zookeeper->exists(source_path + "/columns")) + while (!zookeeper->exists(fs::path(source_path) / "columns")) { LOG_INFO(log, "Waiting for replica {} to be fully created", source_path); zkutil::EventPtr event = std::make_shared(); - if (zookeeper->exists(source_path + "/columns", nullptr, event)) + if (zookeeper->exists(fs::path(source_path) / "columns", nullptr, event)) { LOG_WARNING(log, "Oops, a watch has leaked"); break; @@ -2514,29 +2519,29 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo while (true) { Coordination::Stat log_pointer_stat; - String raw_log_pointer = zookeeper->get(source_path + "/log_pointer", &log_pointer_stat); + String raw_log_pointer = zookeeper->get(fs::path(source_path) / "log_pointer", &log_pointer_stat); Coordination::Requests ops; - ops.push_back(zkutil::makeSetRequest(replica_path + "/log_pointer", raw_log_pointer, -1)); + ops.push_back(zkutil::makeSetRequest(fs::path(replica_path) / "log_pointer", raw_log_pointer, -1)); /// For support old versions CH. if (source_is_lost_stat.version == -1) { /// We check that it was not suddenly upgraded to new version. /// Otherwise it can be upgraded and instantly become lost, but we cannot notice that. - ops.push_back(zkutil::makeCreateRequest(source_path + "/is_lost", "0", zkutil::CreateMode::Persistent)); - ops.push_back(zkutil::makeRemoveRequest(source_path + "/is_lost", -1)); + ops.push_back(zkutil::makeCreateRequest(fs::path(source_path) / "is_lost", "0", zkutil::CreateMode::Persistent)); + ops.push_back(zkutil::makeRemoveRequest(fs::path(source_path) / "is_lost", -1)); } else /// The replica we clone should not suddenly become lost. - ops.push_back(zkutil::makeCheckRequest(source_path + "/is_lost", source_is_lost_stat.version)); + ops.push_back(zkutil::makeCheckRequest(fs::path(source_path) / "is_lost", source_is_lost_stat.version)); Coordination::Responses responses; /// Let's remember the queue of the reference/master replica. - source_queue_names = zookeeper->getChildren(source_path + "/queue"); + source_queue_names = zookeeper->getChildren(fs::path(source_path) / "queue"); /// Check that our log pointer didn't changed while we read queue entries - ops.push_back(zkutil::makeCheckRequest(source_path + "/log_pointer", log_pointer_stat.version)); + ops.push_back(zkutil::makeCheckRequest(fs::path(source_path) / "log_pointer", log_pointer_stat.version)); auto rc = zookeeper->tryMulti(ops, responses); @@ -2576,19 +2581,19 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo for (const String & entry_name : source_queue_names) { String entry; - if (!zookeeper->tryGet(source_path + "/queue/" + entry_name, entry)) + if (!zookeeper->tryGet(fs::path(source_path) / "queue" / entry_name, entry)) continue; source_queue.push_back(entry); } /// Add to the queue jobs to receive all the active parts that the reference/master replica has. - Strings source_replica_parts = zookeeper->getChildren(source_path + "/parts"); + Strings source_replica_parts = zookeeper->getChildren(fs::path(source_path) / "parts"); ActiveDataPartSet active_parts_set(format_version, source_replica_parts); Strings active_parts = active_parts_set.getParts(); /// Remove local parts if source replica does not have them, because such parts will never be fetched by other replicas. - Strings local_parts_in_zk = zookeeper->getChildren(replica_path + "/parts"); + Strings local_parts_in_zk = zookeeper->getChildren(fs::path(replica_path) / "parts"); Strings parts_to_remove_from_zk; for (const auto & part : local_parts_in_zk) { @@ -2632,7 +2637,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo log_entry.new_part_name = name; log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, name); - zookeeper->create(replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential); + zookeeper->create(fs::path(replica_path) / "queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential); } LOG_DEBUG(log, "Queued {} parts to be fetched", active_parts.size()); @@ -2640,7 +2645,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo /// Add content of the reference/master replica queue to the queue. for (const String & entry : source_queue) { - zookeeper->create(replica_path + "/queue/queue-", entry, zkutil::CreateMode::PersistentSequential); + zookeeper->create(fs::path(replica_path) / "queue/queue-", entry, zkutil::CreateMode::PersistentSequential); } LOG_DEBUG(log, "Copied {} queue entries", source_queue.size()); @@ -2652,7 +2657,7 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke Coordination::Stat is_lost_stat; bool is_new_replica = true; String res; - if (zookeeper->tryGet(replica_path + "/is_lost", res, &is_lost_stat)) + if (zookeeper->tryGet(fs::path(replica_path) / "is_lost", res, &is_lost_stat)) { if (res == "0") return; @@ -2664,14 +2669,14 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke /// Replica was created by old version of CH, so me must create "/is_lost". /// Note that in old version of CH there was no "lost" replicas possible. /// TODO is_lost node should always exist since v18.12, maybe we can replace `tryGet` with `get` and remove old code? - zookeeper->create(replica_path + "/is_lost", "0", zkutil::CreateMode::Persistent); + zookeeper->create(fs::path(replica_path) / "is_lost", "0", zkutil::CreateMode::Persistent); return; } /// is_lost is "1": it means that we are in repair mode. /// Try choose source replica to clone. /// Source replica must not be lost and should have minimal queue size and maximal log pointer. - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); std::vector futures; for (const String & source_replica_name : replicas) { @@ -2679,20 +2684,20 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke if (source_replica_name == replica_name) continue; - String source_replica_path = zookeeper_path + "/replicas/" + source_replica_name; + String source_replica_path = fs::path(zookeeper_path) / "replicas" / source_replica_name; /// Obviously the following get operations are not atomic, but it's ok to choose good enough replica, not the best one. /// NOTE: We may count some entries twice if log_pointer is moved. - futures.emplace_back(zookeeper->asyncTryGet(source_replica_path + "/is_lost")); - futures.emplace_back(zookeeper->asyncTryGet(source_replica_path + "/log_pointer")); - futures.emplace_back(zookeeper->asyncTryGet(source_replica_path + "/queue")); + futures.emplace_back(zookeeper->asyncTryGet(fs::path(source_replica_path) / "is_lost")); + futures.emplace_back(zookeeper->asyncTryGet(fs::path(source_replica_path) / "log_pointer")); + futures.emplace_back(zookeeper->asyncTryGet(fs::path(source_replica_path) / "queue")); } /// Wait for results before getting log entries for (auto & future : futures) future.wait(); - Strings log_entries = zookeeper->getChildren(zookeeper_path + "/log"); + Strings log_entries = zookeeper->getChildren(fs::path(zookeeper_path) / "log"); size_t max_log_entry = 0; if (!log_entries.empty()) { @@ -2763,14 +2768,14 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke LOG_WARNING(log, "Will mimic {}", source_replica); /// Clear obsolete queue that we no longer need. - zookeeper->removeChildren(replica_path + "/queue"); + zookeeper->removeChildren(fs::path(replica_path) / "queue"); /// Will do repair from the selected replica. cloneReplica(source_replica, source_is_lost_stat, zookeeper); /// If repair fails to whatever reason, the exception is thrown, is_lost will remain "1" and the replica will be repaired later. /// If replica is repaired successfully, we remove is_lost flag. - zookeeper->set(replica_path + "/is_lost", "0"); + zookeeper->set(fs::path(replica_path) / "is_lost", "0"); } @@ -3097,7 +3102,7 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c std::vector> exists_futures; exists_futures.reserve(parts.size()); for (const auto & part : parts) - exists_futures.emplace_back(zookeeper->asyncExists(replica_path + "/parts/" + part->name)); + exists_futures.emplace_back(zookeeper->asyncExists(fs::path(replica_path) / "parts" / part->name)); bool all_in_zk = true; for (size_t i = 0; i < parts.size(); ++i) @@ -3137,11 +3142,11 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c Coordination::Responses responses; ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/log/log-", entry.toString(), + fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); ops.emplace_back(zkutil::makeSetRequest( - zookeeper_path + "/log", "", log_version)); /// Check and update version. + fs::path(zookeeper_path) / "log", "", log_version)); /// Check and update version. Coordination::Error code = zookeeper->tryMulti(ops, responses); @@ -3177,7 +3182,7 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c auto zookeeper = getZooKeeper(); /// If there is no information about part in ZK, we will not mutate it. - if (!zookeeper->exists(replica_path + "/parts/" + part.name)) + if (!zookeeper->exists(fs::path(replica_path) / "parts" / part.name)) { if (part.modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(nullptr)) { @@ -3207,11 +3212,11 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c Coordination::Responses responses; ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/log/log-", entry.toString(), + fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); ops.emplace_back(zkutil::makeSetRequest( - zookeeper_path + "/log", "", log_version)); /// Check and update version. + fs::path(zookeeper_path) / "log", "", log_version)); /// Check and update version. Coordination::Error code = zookeeper->tryMulti(ops, responses); @@ -3232,12 +3237,12 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, Coordination::Requests & ops, bool has_children) { - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; if (has_children) { - ops.emplace_back(zkutil::makeRemoveRequest(part_path + "/checksums", -1)); - ops.emplace_back(zkutil::makeRemoveRequest(part_path + "/columns", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(part_path) / "checksums", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(part_path) / "columns", -1)); } ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1)); } @@ -3245,7 +3250,7 @@ void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_nam void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name) { auto zookeeper = getZooKeeper(); - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; Coordination::Stat stat; /// Part doesn't exist, nothing to remove @@ -3262,7 +3267,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n { auto zookeeper = getZooKeeper(); - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; Coordination::Requests ops; @@ -3281,7 +3286,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n log_entry->new_part_name = part_name; ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/queue/queue-", log_entry->toString(), + fs::path(replica_path) / "queue/queue-", log_entry->toString(), zkutil::CreateMode::PersistentSequential)); auto results = zookeeper->multi(ops); @@ -3306,7 +3311,7 @@ void StorageReplicatedMergeTree::enterLeaderElection() { leader_election = std::make_shared( getContext()->getSchedulePool(), - zookeeper_path + "/leader_election", + fs::path(zookeeper_path) / "leader_election", *current_zookeeper, /// current_zookeeper lives for the lifetime of leader_election, /// since before changing `current_zookeeper`, `leader_election` object is destroyed in `partialShutdown` method. callback, @@ -3361,13 +3366,13 @@ ConnectionTimeouts StorageReplicatedMergeTree::getFetchPartHTTPTimeouts(ContextP bool StorageReplicatedMergeTree::checkReplicaHavePart(const String & replica, const String & part_name) { auto zookeeper = getZooKeeper(); - return zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name); + return zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "parts" / part_name); } String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active) { auto zookeeper = getZooKeeper(); - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); /// Select replicas in uniformly random order. std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); @@ -3383,7 +3388,7 @@ String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_nam LOG_TRACE(log, "Candidate replica: {}", replica); if (checkReplicaHavePart(replica, part_name) && - (!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))) + (!active || zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active"))) return replica; /// Obviously, replica could become inactive or even vanish after return from this method. @@ -3395,7 +3400,7 @@ String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_nam String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(LogEntry & entry, bool active) { auto zookeeper = getZooKeeper(); - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); /// Select replicas in uniformly random order. std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); @@ -3405,11 +3410,11 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(LogEntry & entr if (replica == replica_name) continue; - if (active && !zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (active && !zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active")) continue; String largest_part_found; - Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts"); + Strings parts = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas" / replica / "parts"); for (const String & part_on_replica : parts) { if (part_on_replica == entry.new_part_name @@ -3450,7 +3455,7 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart( const String & part_name, bool active, String & found_part_name) { auto zookeeper = getZooKeeper(); - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); /// Select replicas in uniformly random order. std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); @@ -3463,10 +3468,10 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart( if (replica == replica_name) continue; - if (active && !zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (active && !zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active")) continue; - Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts"); + Strings parts = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas" / replica / "parts"); for (const String & part_on_replica : parts) { if (part_on_replica == part_name @@ -3494,11 +3499,11 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name, bool is_ auto zookeeper = getZooKeeper(); /// Information on which replicas a part has been added, if the quorum has not yet been reached. - String quorum_status_path = zookeeper_path + "/quorum/status"; + String quorum_status_path = fs::path(zookeeper_path) / "quorum" / "status"; if (is_parallel) - quorum_status_path = zookeeper_path + "/quorum/parallel/" + part_name; + quorum_status_path = fs::path(zookeeper_path) / "quorum" / "parallel" / part_name; /// The name of the previous part for which the quorum was reached. - const String quorum_last_part_path = zookeeper_path + "/quorum/last_part"; + const String quorum_last_part_path = fs::path(zookeeper_path) / "quorum" / "last_part"; String value; Coordination::Stat stat; @@ -3600,7 +3605,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) auto zookeeper = getZooKeeper(); /// The name of the previous part for which the quorum was reached. - const String quorum_last_part_path = zookeeper_path + "/quorum/last_part"; + const String quorum_last_part_path = fs::path(zookeeper_path) / "quorum" / "last_part"; /// Delete information from "last_part" node. @@ -3650,7 +3655,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) bool StorageReplicatedMergeTree::partIsInsertingWithParallelQuorum(const MergeTreePartInfo & part_info) const { auto zookeeper = getZooKeeper(); - return zookeeper->exists(zookeeper_path + "/quorum/parallel/" + part_info.getPartName()); + return zookeeper->exists(fs::path(zookeeper_path) / "quorum" / "parallel" / part_info.getPartName()); } @@ -3658,7 +3663,7 @@ bool StorageReplicatedMergeTree::partIsLastQuorumPart(const MergeTreePartInfo & { auto zookeeper = getZooKeeper(); - const String parts_with_quorum_path = zookeeper_path + "/quorum/last_part"; + const String parts_with_quorum_path = fs::path(zookeeper_path) / "quorum" / "last_part"; String parts_with_quorum_str = zookeeper->get(parts_with_quorum_path); @@ -3742,13 +3747,13 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora source_part_checksums.computeTotalChecksums(source_part->checksums); MinimalisticDataPartChecksums desired_checksums; - String part_path = source_replica_path + "/parts/" + part_name; + String part_path = fs::path(source_replica_path) / "parts" / part_name; String part_znode = zookeeper->get(part_path); if (!part_znode.empty()) desired_checksums = ReplicatedMergeTreePartHeader::fromString(part_znode).getChecksums(); else { - String desired_checksums_str = zookeeper->get(part_path + "/checksums"); + String desired_checksums_str = zookeeper->get(fs::path(part_path) / "checksums"); desired_checksums = MinimalisticDataPartChecksums::deserializeFrom(desired_checksums_str); } @@ -3777,7 +3782,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora } else { - address.fromString(zookeeper->get(source_replica_path + "/host")); + address.fromString(zookeeper->get(fs::path(source_replica_path) / "host")); timeouts = getFetchPartHTTPTimeouts(getContext()); credentials = getContext()->getInterserverCredentials(); @@ -3824,16 +3829,16 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora if (quorum) { /// Check if this quorum insert is parallel or not - if (zookeeper->exists(zookeeper_path + "/quorum/parallel/" + part_name)) + if (zookeeper->exists(fs::path(zookeeper_path) / "quorum" / "parallel" / part_name)) updateQuorum(part_name, true); - else if (zookeeper->exists(zookeeper_path + "/quorum/status")) + else if (zookeeper->exists(fs::path(zookeeper_path) / "quorum" / "status")) updateQuorum(part_name, false); } /// merged parts that are still inserted with quorum. if it only contains one block, it hasn't been merged before if (part_info.level != 0 || part_info.mutation != 0) { - Strings quorum_parts = zookeeper->getChildren(zookeeper_path + "/quorum/parallel"); + Strings quorum_parts = zookeeper->getChildren(fs::path(zookeeper_path) / "quorum" / "parallel"); for (const String & quorum_part : quorum_parts) { auto quorum_part_info = MergeTreePartInfo::fromPartName(quorum_part, format_version); @@ -3856,7 +3861,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora { // The fetched part is valuable and should not be cleaned like a temp part. part->is_temp = false; - part->renameTo("detached/" + part_name, true); + part->renameTo(fs::path("detached") / part_name, true); } } catch (const Exception & e) @@ -3934,7 +3939,7 @@ bool StorageReplicatedMergeTree::fetchExistsPart(const String & part_name, const std::function get_part; - ReplicatedMergeTreeAddress address(zookeeper->get(source_replica_path + "/host")); + ReplicatedMergeTreeAddress address(zookeeper->get(fs::path(source_replica_path) / "host")); auto timeouts = ConnectionTimeouts::getHTTPTimeouts(getContext()); auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); @@ -4092,7 +4097,7 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg auto zookeeper = getZooKeeper(); - const String quorum_status_path = zookeeper_path + "/quorum/status"; + const String quorum_status_path = fs::path(zookeeper_path) / "quorum" / "status"; String value; Coordination::Stat stat; @@ -4108,7 +4113,7 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg } String added_parts_str; - if (zookeeper->tryGet(zookeeper_path + "/quorum/last_part", added_parts_str)) + if (zookeeper->tryGet(fs::path(zookeeper_path) / "quorum" / "last_part", added_parts_str)) { if (!added_parts_str.empty()) { @@ -4423,8 +4428,8 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer /// If metadata nodes have changed, we will update table structure locally. Coordination::Requests requests; - requests.emplace_back(zkutil::makeSetRequest(replica_path + "/columns", entry.columns_str, -1)); - requests.emplace_back(zkutil::makeSetRequest(replica_path + "/metadata", entry.metadata_str, -1)); + requests.emplace_back(zkutil::makeSetRequest(fs::path(replica_path) / "columns", entry.columns_str, -1)); + requests.emplace_back(zkutil::makeSetRequest(fs::path(replica_path) / "metadata", entry.metadata_str, -1)); zookeeper->multi(requests); @@ -4440,7 +4445,7 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer } /// This transaction may not happen, but it's OK, because on the next retry we will eventually create/update this node - zookeeper->createOrUpdate(replica_path + "/metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent); + zookeeper->createOrUpdate(fs::path(replica_path) / "metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent); return true; } @@ -4486,7 +4491,7 @@ PartitionBlockNumbersHolder StorageReplicatedMergeTree::allocateBlockNumbersInAf { /// TODO: Implement optimal block number aqcuisition algorithm in multiple (but not all) partitions EphemeralLocksInAllPartitions lock_holder( - zookeeper_path + "/block_numbers", "block-", zookeeper_path + "/temp", *zookeeper); + fs::path(zookeeper_path) / "block_numbers", "block-", fs::path(zookeeper_path) / "temp", *zookeeper); PartitionBlockNumbersHolder::BlockNumbersType block_numbers; for (const auto & lock : lock_holder.getLocks()) @@ -4582,10 +4587,10 @@ void StorageReplicatedMergeTree::alter( size_t mutation_path_idx = std::numeric_limits::max(); String new_metadata_str = future_metadata_in_zk.toString(); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/metadata", new_metadata_str, metadata_version)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "metadata", new_metadata_str, metadata_version)); String new_columns_str = future_metadata.columns.toString(); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/columns", new_columns_str, -1)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "columns", new_columns_str, -1)); if (ast_to_str(current_metadata->settings_changes) != ast_to_str(future_metadata.settings_changes)) { @@ -4613,12 +4618,12 @@ void StorageReplicatedMergeTree::alter( alter_path_idx = ops.size(); ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/log/log-", alter_entry->toString(), zkutil::CreateMode::PersistentSequential)); + fs::path(zookeeper_path) / "log/log-", alter_entry->toString(), zkutil::CreateMode::PersistentSequential)); PartitionBlockNumbersHolder partition_block_numbers_holder; if (alter_entry->have_mutation) { - const String mutations_path(zookeeper_path + "/mutations"); + const String mutations_path(fs::path(zookeeper_path) / "mutations"); ReplicatedMergeTreeMutationEntry mutation_entry; mutation_entry.alter_version = new_metadata_version; @@ -4637,7 +4642,7 @@ void StorageReplicatedMergeTree::alter( ops.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version)); mutation_path_idx = ops.size(); ops.emplace_back( - zkutil::makeCreateRequest(mutations_path + "/", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential)); + zkutil::makeCreateRequest(fs::path(mutations_path) / "", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential)); } if (auto txn = query_context->getZooKeeperMetadataTransaction()) @@ -4645,7 +4650,7 @@ void StorageReplicatedMergeTree::alter( txn->moveOpsTo(ops); /// NOTE: IDatabase::alterTable(...) is called when executing ALTER_METADATA queue entry without query context, /// so we have to update metadata of DatabaseReplicated here. - String metadata_zk_path = txn->getDatabaseZooKeeperPath() + "/metadata/" + escapeForFileName(table_id.table_name); + String metadata_zk_path = fs::path(txn->getDatabaseZooKeeperPath()) / "metadata" / escapeForFileName(table_id.table_name); auto ast = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name, query_context); applyMetadataChangesToCreateQuery(ast, future_metadata); ops.emplace_back(zkutil::makeSetRequest(metadata_zk_path, getObjectDefinitionFromCreateQuery(ast), -1)); @@ -4830,7 +4835,7 @@ void StorageReplicatedMergeTree::truncate( zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); - Strings partitions = zookeeper->getChildren(zookeeper_path + "/block_numbers"); + Strings partitions = zookeeper->getChildren(fs::path(zookeeper_path) / "block_numbers"); for (String & partition_id : partitions) { @@ -4905,7 +4910,7 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, c try { auto zookeeper = getZooKeeper(); - zookeeper->set(replica_path + "/host", getReplicatedMergeTreeAddress().toString()); + zookeeper->set(fs::path(replica_path) / "host", getReplicatedMergeTreeAddress().toString()); } catch (Coordination::Exception & e) { @@ -4949,8 +4954,8 @@ StorageReplicatedMergeTree::allocateBlockNumber( deduplication_check_ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_block_id_path, -1)); } - String block_numbers_path = zookeeper_path + "/block_numbers"; - String partition_path = block_numbers_path + "/" + partition_id; + String block_numbers_path = fs::path(zookeeper_path) / "block_numbers"; + String partition_path = fs::path(block_numbers_path) / partition_id; if (!existsNodeCached(partition_path)) { @@ -4972,7 +4977,7 @@ StorageReplicatedMergeTree::allocateBlockNumber( try { lock = EphemeralLockInZooKeeper( - partition_path + "/block-", zookeeper_path + "/temp", *zookeeper, &deduplication_check_ops); + partition_path + "/block-", fs::path(zookeeper_path) / "temp", *zookeeper, &deduplication_check_ops); } catch (const zkutil::KeeperMultiException & e) { @@ -4996,11 +5001,11 @@ Strings StorageReplicatedMergeTree::waitForAllTableReplicasToProcessLogEntry( LOG_DEBUG(log, "Waiting for all replicas to process {}", entry.znode_name); auto zookeeper = getZooKeeper(); - Strings replicas = zookeeper->getChildren(table_zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(table_zookeeper_path) / "replicas"); Strings unwaited; for (const String & replica : replicas) { - if (wait_for_non_active || zookeeper->exists(table_zookeeper_path + "/replicas/" + replica + "/is_active")) + if (wait_for_non_active || zookeeper->exists(fs::path(table_zookeeper_path) / "replicas" / replica / "is_active")) { if (!waitForTableReplicaToProcessLogEntry(table_zookeeper_path, replica, entry, wait_for_non_active)) unwaited.push_back(replica); @@ -5050,7 +5055,7 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( const auto & stop_waiting = [&]() { bool stop_waiting_itself = waiting_itself && (partial_shutdown_called || is_dropped); - bool stop_waiting_non_active = !wait_for_non_active && !getZooKeeper()->exists(table_zookeeper_path + "/replicas/" + replica + "/is_active"); + bool stop_waiting_non_active = !wait_for_non_active && !getZooKeeper()->exists(fs::path(table_zookeeper_path) / "replicas" / replica / "is_active"); return stop_waiting_itself || stop_waiting_non_active; }; @@ -5072,7 +5077,7 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( { zkutil::EventPtr event = std::make_shared(); - String log_pointer = getZooKeeper()->get(table_zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event); + String log_pointer = getZooKeeper()->get(fs::path(table_zookeeper_path) / "replicas" / replica / "log_pointer", nullptr, event); if (!log_pointer.empty() && parse(log_pointer) > log_index) break; @@ -5089,9 +5094,9 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( * looking for a node with the same content. And if we do not find it - then the replica has already taken this entry in its queue. */ - String log_pointer = getZooKeeper()->get(table_zookeeper_path + "/replicas/" + replica + "/log_pointer"); + String log_pointer = getZooKeeper()->get(fs::path(table_zookeeper_path) / "replicas" / replica / "log_pointer"); - Strings log_entries = getZooKeeper()->getChildren(table_zookeeper_path + "/log"); + Strings log_entries = getZooKeeper()->getChildren(fs::path(table_zookeeper_path) / "log"); UInt64 log_index = 0; bool found = false; @@ -5103,7 +5108,7 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( continue; String log_entry_str; - bool exists = getZooKeeper()->tryGet(table_zookeeper_path + "/log/" + log_entry_name, log_entry_str); + bool exists = getZooKeeper()->tryGet(fs::path(table_zookeeper_path) / "log" / log_entry_name, log_entry_str); if (exists && entry_str == log_entry_str) { found = true; @@ -5121,7 +5126,7 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( { zkutil::EventPtr event = std::make_shared(); - String log_pointer_new = getZooKeeper()->get(table_zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event); + String log_pointer_new = getZooKeeper()->get(fs::path(table_zookeeper_path) / "replicas" / replica / "log_pointer", nullptr, event); if (!log_pointer_new.empty() && parse(log_pointer_new) > log_index) break; @@ -5146,13 +5151,13 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( * Therefore, we search by comparing the content. */ - Strings queue_entries = getZooKeeper()->getChildren(table_zookeeper_path + "/replicas/" + replica + "/queue"); + Strings queue_entries = getZooKeeper()->getChildren(fs::path(table_zookeeper_path) / "replicas" / replica / "queue"); String queue_entry_to_wait_for; for (const String & entry_name : queue_entries) { String queue_entry_str; - bool exists = getZooKeeper()->tryGet(table_zookeeper_path + "/replicas/" + replica + "/queue/" + entry_name, queue_entry_str); + bool exists = getZooKeeper()->tryGet(fs::path(table_zookeeper_path) / "replicas" / replica / "queue" / entry_name, queue_entry_str); if (exists && queue_entry_str == entry_str) { queue_entry_to_wait_for = entry_name; @@ -5170,7 +5175,7 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( LOG_DEBUG(log, "Waiting for {} to disappear from {} queue", queue_entry_to_wait_for, replica); /// Third - wait until the entry disappears from the replica queue or replica become inactive. - String path_to_wait_on = table_zookeeper_path + "/replicas/" + replica + "/queue/" + queue_entry_to_wait_for; + String path_to_wait_on = fs::path(table_zookeeper_path) / "replicas" / replica / "queue" / queue_entry_to_wait_for; return getZooKeeper()->waitForDisappear(path_to_wait_on, stop_waiting); } @@ -5212,7 +5217,7 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) { try { - auto log_entries = zookeeper->getChildren(zookeeper_path + "/log"); + auto log_entries = zookeeper->getChildren(fs::path(zookeeper_path) / "log"); if (!log_entries.empty()) { @@ -5220,14 +5225,14 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) res.log_max_index = parse(last_log_entry.substr(strlen("log-"))); } - String log_pointer_str = zookeeper->get(replica_path + "/log_pointer"); + String log_pointer_str = zookeeper->get(fs::path(replica_path) / "log_pointer"); res.log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); - auto all_replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + auto all_replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); res.total_replicas = all_replicas.size(); for (const String & replica : all_replicas) - if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active")) ++res.active_replicas; } catch (const Coordination::Exception &) @@ -5305,7 +5310,7 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t time_t max_replicas_unprocessed_insert_time = 0; bool have_replica_with_nothing_unprocessed = false; - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); for (const auto & replica : replicas) { @@ -5313,11 +5318,11 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t continue; /// Skip dead replicas. - if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (!zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active")) continue; String value; - if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/min_unprocessed_insert_time", value)) + if (!zookeeper->tryGet(fs::path(zookeeper_path) / "replicas" / replica / "min_unprocessed_insert_time", value)) continue; time_t replica_time = value.empty() ? 0 : parse(value); @@ -5424,13 +5429,13 @@ void StorageReplicatedMergeTree::fetchPartition( { /// List of replicas of source shard. - replicas = zookeeper->getChildren(from + "/replicas"); + replicas = zookeeper->getChildren(fs::path(from) / "replicas"); /// Leave only active replicas. active_replicas.reserve(replicas.size()); for (const String & replica : replicas) - if (zookeeper->exists(from + "/replicas/" + replica + "/is_active")) + if (zookeeper->exists(fs::path(from) / "replicas" / replica / "is_active")) active_replicas.push_back(replica); if (active_replicas.empty()) @@ -5447,13 +5452,13 @@ void StorageReplicatedMergeTree::fetchPartition( for (const String & replica : active_replicas) { - String current_replica_path = from + "/replicas/" + replica; + String current_replica_path = fs::path(from) / "replicas" / replica; - String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer"); + String log_pointer_str = zookeeper->get(fs::path(current_replica_path) / "log_pointer"); Int64 log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); Coordination::Stat stat; - zookeeper->get(current_replica_path + "/queue", &stat); + zookeeper->get(fs::path(current_replica_path) / "queue", &stat); size_t queue_size = stat.numChildren; if (log_pointer > max_log_pointer @@ -5471,7 +5476,7 @@ void StorageReplicatedMergeTree::fetchPartition( LOG_INFO(log, "Found {} replicas, {} of them are active. Selected {} to fetch from.", replicas.size(), active_replicas.size(), best_replica); - String best_replica_path = from + "/replicas/" + best_replica; + String best_replica_path = fs::path(from) / "replicas" / best_replica; /// Let's find out which parts are on the best replica. @@ -5490,7 +5495,7 @@ void StorageReplicatedMergeTree::fetchPartition( if (try_no >= query_context->getSettings().max_fetch_partition_retries_count) throw Exception("Too many retries to fetch parts from " + best_replica_path, ErrorCodes::TOO_MANY_RETRIES_TO_FETCH_PARTS); - Strings parts = zookeeper->getChildren(best_replica_path + "/parts"); + Strings parts = zookeeper->getChildren(fs::path(best_replica_path) / "parts"); ActiveDataPartSet active_parts_set(format_version, parts); Strings parts_to_fetch; @@ -5611,7 +5616,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte mutation_entry.source_replica = replica_name; mutation_entry.commands = commands; - const String mutations_path = zookeeper_path + "/mutations"; + const String mutations_path = fs::path(zookeeper_path) / "mutations"; const auto zookeeper = getZooKeeper(); /// Update the mutations_path node when creating the mutation and check its version to ensure that @@ -5634,7 +5639,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte Coordination::Requests requests; requests.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version)); requests.emplace_back(zkutil::makeCreateRequest( - mutations_path + "/", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential)); + fs::path(mutations_path) / "", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential)); if (auto txn = query_context->getZooKeeperMetadataTransaction()) txn->moveOpsTo(requests); @@ -5673,7 +5678,7 @@ void StorageReplicatedMergeTree::waitMutation(const String & znode_name, size_t auto zookeeper = getZooKeeper(); Strings replicas; if (mutations_sync == 2) /// wait for all replicas - replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); else if (mutations_sync == 1) /// just wait for ourself replicas.push_back(replica_name); @@ -5868,7 +5873,7 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const St exists_futures.reserve(part_names.size()); for (const String & part_name : part_names) { - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; exists_futures.emplace_back(zookeeper->asyncExists(part_path)); } @@ -5931,7 +5936,7 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper( /// if zk session will be dropped for (const String & part_name : part_names) { - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; exists_futures.emplace_back(zookeeper->asyncExists(part_path)); } @@ -5990,7 +5995,7 @@ void StorageReplicatedMergeTree::getClearBlocksInPartitionOps( Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num) { Strings blocks; - if (Coordination::Error::ZOK != zookeeper.tryGetChildren(zookeeper_path + "/blocks", blocks)) + if (Coordination::Error::ZOK != zookeeper.tryGetChildren(fs::path(zookeeper_path) / "blocks", blocks)) throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE); String partition_prefix = partition_id + "_"; @@ -5999,7 +6004,7 @@ void StorageReplicatedMergeTree::getClearBlocksInPartitionOps( { if (startsWith(block_id, partition_prefix)) { - String path = zookeeper_path + "/blocks/" + block_id; + String path = fs::path(zookeeper_path) / "blocks" / block_id; get_futures.emplace_back(path, zookeeper.asyncTryGet(path)); } } @@ -6107,7 +6112,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( else LOG_INFO(log, "Trying to attach {} with hash_hex {}", src_part->name, hash_hex); - String block_id_path = replace ? "" : (zookeeper_path + "/blocks/" + partition_id + "_replace_from_" + hash_hex); + String block_id_path = replace ? "" : (fs::path(zookeeper_path) / "blocks" / (partition_id + "_replace_from_" + hash_hex)); auto lock = allocateBlockNumber(partition_id, zookeeper, block_id_path); if (!lock) @@ -6178,8 +6183,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom( if (auto txn = query_context->getZooKeeperMetadataTransaction()) txn->moveOpsTo(ops); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); /// Just update version + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); Transaction transaction(*this); { @@ -6364,7 +6369,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta } } - ops.emplace_back(zkutil::makeCreateRequest(dest_table_storage->zookeeper_path + "/log/log-", + ops.emplace_back(zkutil::makeCreateRequest(fs::path(dest_table_storage->zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); { @@ -6413,8 +6418,8 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta Coordination::Requests ops_dest; ops_dest.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/log/log-", entry_delete.toString(), zkutil::CreateMode::PersistentSequential)); - ops_dest.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version + fs::path(zookeeper_path) / "log/log-", entry_delete.toString(), zkutil::CreateMode::PersistentSequential)); + ops_dest.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); /// Just update version op_results = zookeeper->multi(ops_dest); @@ -6453,22 +6458,22 @@ void StorageReplicatedMergeTree::getCommitPartOps( if (storage_settings_ptr->use_minimalistic_part_header_in_zookeeper) { ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/parts/" + part->name, + fs::path(replica_path) / "parts" / part->name, ReplicatedMergeTreePartHeader::fromColumnsAndChecksums(part->getColumns(), part->checksums).toString(), zkutil::CreateMode::Persistent)); } else { ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/parts/" + part->name, + fs::path(replica_path) / "parts" / part->name, "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/parts/" + part->name + "/columns", + fs::path(replica_path) / "parts" / part->name / "columns", part->getColumns().toString(), zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/parts/" + part->name + "/checksums", + fs::path(replica_path) / "parts" / part->name / "checksums", getChecksumsForZooKeeper(part->checksums), zkutil::CreateMode::Persistent)); } @@ -6619,9 +6624,9 @@ bool StorageReplicatedMergeTree::dropPart( entry.detach = detach; entry.create_time = time(nullptr); - ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/log", merge_pred.getVersion())); /// Make sure no new events were added to the log. - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version. + ops.emplace_back(zkutil::makeCheckRequest(fs::path(zookeeper_path) / "log", merge_pred.getVersion())); /// Make sure no new events were added to the log. + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); /// Just update version. Coordination::Responses responses; Coordination::Error rc = zookeeper->tryMulti(ops, responses); @@ -6676,8 +6681,8 @@ bool StorageReplicatedMergeTree::dropAllPartsInPartition( entry.create_time = time(nullptr); Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version. + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); /// Just update version. if (auto txn = query_context->getZooKeeperMetadataTransaction()) txn->moveOpsTo(ops); Coordination::Responses responses = zookeeper.multi(ops); @@ -6755,7 +6760,7 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part) String id = part.getUniqueId(); boost::replace_all(id, "/", "_"); - String zookeeper_node = zookeeper_path + "/zero_copy_s3/shared/" + part.name + "/" + id + "/" + replica_name; + String zookeeper_node = fs::path(zookeeper_path) / "zero_copy_s3" / "shared" / part.name / id / replica_name; LOG_TRACE(log, "Set zookeeper lock {}", zookeeper_node); @@ -6796,9 +6801,9 @@ bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & par String id = part.getUniqueId(); boost::replace_all(id, "/", "_"); - String zookeeper_part_node = zookeeper_path + "/zero_copy_s3/shared/" + part.name; - String zookeeper_part_uniq_node = zookeeper_part_node + "/" + id; - String zookeeper_node = zookeeper_part_uniq_node + "/" + replica_name; + String zookeeper_part_node = fs::path(zookeeper_path) / "zero_copy_s3" / "shared" / part.name; + String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id; + String zookeeper_node = fs::path(zookeeper_part_uniq_node) / replica_name; LOG_TRACE(log, "Remove zookeeper lock {}", zookeeper_node); @@ -6857,7 +6862,7 @@ String StorageReplicatedMergeTree::getSharedDataReplica( if (!zookeeper) return best_replica; - String zookeeper_part_node = zookeeper_path + "/zero_copy_s3/shared/" + part.name; + String zookeeper_part_node = fs::path(zookeeper_path) / "zero_copy_s3" / "shared" / part.name; Strings ids; zookeeper->tryGetChildren(zookeeper_part_node, ids); @@ -6865,7 +6870,7 @@ String StorageReplicatedMergeTree::getSharedDataReplica( Strings replicas; for (const auto & id : ids) { - String zookeeper_part_uniq_node = zookeeper_part_node + "/" + id; + String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id; Strings id_replicas; zookeeper->tryGetChildren(zookeeper_part_uniq_node, id_replicas); LOG_TRACE(log, "Found zookeper replicas for {}: {}", zookeeper_part_uniq_node, id_replicas.size()); @@ -6882,7 +6887,7 @@ String StorageReplicatedMergeTree::getSharedDataReplica( active_replicas.reserve(replicas.size()); for (const String & replica : replicas) - if ((replica != replica_name) && (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))) + if ((replica != replica_name) && (zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active"))) active_replicas.push_back(replica); LOG_TRACE(log, "Found zookeper active replicas for part {}: {}", part.name, active_replicas.size()); @@ -6901,13 +6906,13 @@ String StorageReplicatedMergeTree::getSharedDataReplica( for (const String & replica : active_replicas) { - String current_replica_path = zookeeper_path + "/replicas/" + replica; + String current_replica_path = fs::path(zookeeper_path) / "replicas" / replica; - String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer"); + String log_pointer_str = zookeeper->get(fs::path(current_replica_path) / "log_pointer"); Int64 log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); Coordination::Stat stat; - zookeeper->get(current_replica_path + "/queue", &stat); + zookeeper->get(fs::path(current_replica_path) / "queue", &stat); size_t queue_size = stat.numChildren; if (log_pointer > max_log_pointer @@ -6925,16 +6930,16 @@ String StorageReplicatedMergeTree::getSharedDataReplica( String StorageReplicatedMergeTree::findReplicaHavingPart( const String & part_name, const String & zookeeper_path_, zkutil::ZooKeeper::Ptr zookeeper_) { - Strings replicas = zookeeper_->getChildren(zookeeper_path_ + "/replicas"); + Strings replicas = zookeeper_->getChildren(fs::path(zookeeper_path_) / "replicas"); /// Select replicas in uniformly random order. std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); for (const String & replica : replicas) { - if (zookeeper_->exists(zookeeper_path_ + "/replicas/" + replica + "/parts/" + part_name) - && zookeeper_->exists(zookeeper_path_ + "/replicas/" + replica + "/is_active")) - return zookeeper_path_ + "/replicas/" + replica; + if (zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica / "parts" / part_name) + && zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica / "is_active")) + return fs::path(zookeeper_path_) / "replicas" / replica; } return {}; @@ -6942,23 +6947,23 @@ String StorageReplicatedMergeTree::findReplicaHavingPart( bool StorageReplicatedMergeTree::checkIfDetachedPartExists(const String & part_name) { - Poco::DirectoryIterator dir_end; + fs::directory_iterator dir_end; for (const std::string & path : getDataPaths()) - for (Poco::DirectoryIterator dir_it{path + "detached/"}; dir_it != dir_end; ++dir_it) - if (dir_it.name() == part_name) + for (fs::directory_iterator dir_it{fs::path(path) / "detached/"}; dir_it != dir_end; ++dir_it) + if (dir_it->path().filename().string() == part_name) return true; return false; } bool StorageReplicatedMergeTree::checkIfDetachedPartitionExists(const String & partition_name) { - Poco::DirectoryIterator dir_end; + fs::directory_iterator dir_end; for (const std::string & path : getDataPaths()) { - for (Poco::DirectoryIterator dir_it{path + "detached/"}; dir_it != dir_end; ++dir_it) + for (fs::directory_iterator dir_it{fs::path(path) / "detached/"}; dir_it != dir_end; ++dir_it) { MergeTreePartInfo part_info; - if (MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_info, format_version) && part_info.partition_id == partition_name) + if (MergeTreePartInfo::tryParsePartName(dir_it->path().filename(), &part_info, format_version) && part_info.partition_id == partition_name) return true; } } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2f25fb43e74..db2866ee23b 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -35,7 +35,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -196,7 +198,7 @@ bool StorageS3Source::initialize() if (current_key.empty()) return false; - file_path = bucket + "/" + current_key; + file_path = fs::path(bucket) / current_key; read_buf = wrapReadBufferWithCompressionMethod( std::make_unique(client, bucket, current_key, s3_max_single_read_retries), chooseCompressionMethod(current_key, compression_hint)); diff --git a/src/Storages/examples/get_current_inserts_in_replicated.cpp b/src/Storages/examples/get_current_inserts_in_replicated.cpp index fa998d20f66..62dcce37d4e 100644 --- a/src/Storages/examples/get_current_inserts_in_replicated.cpp +++ b/src/Storages/examples/get_current_inserts_in_replicated.cpp @@ -36,10 +36,10 @@ try Stopwatch stage; /// Load current inserts std::unordered_set lock_holder_paths; - for (const String & entry : zookeeper->getChildren(zookeeper_path + "/temp")) + for (const String & entry : zookeeper->getChildren(fs::path(zookeeper_path) / "temp")) { if (startsWith(entry, "abandonable_lock-")) - lock_holder_paths.insert(zookeeper_path + "/temp/" + entry); + lock_holder_paths.insert(fs::path(zookeeper_path) / "temp" / entry); } std::cerr << "Stage 1 (get lock holders): " << lock_holder_paths.size() << " lock holders, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; @@ -47,14 +47,14 @@ try if (!lock_holder_paths.empty()) { - Strings partitions = zookeeper->getChildren(zookeeper_path + "/block_numbers"); + Strings partitions = zookeeper->getChildren(fs::path(zookeeper_path) / "block_numbers"); std::cerr << "Stage 2 (get partitions): " << partitions.size() << " partitions, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; stage.restart(); std::vector> lock_futures; for (const String & partition : partitions) - lock_futures.push_back(zookeeper->asyncGetChildren(zookeeper_path + "/block_numbers/" + partition)); + lock_futures.push_back(zookeeper->asyncGetChildren(fs::path(zookeeper_path) / "block_numbers" / partition)); struct BlockInfo { @@ -71,7 +71,7 @@ try for (const String & entry : partition_block_numbers) { Int64 block_number = parse(entry.substr(strlen("block-"))); - String zk_path = zookeeper_path + "/block_numbers/" + partitions[i] + "/" + entry; + String zk_path = fs::path(zookeeper_path) / "block_numbers" / partitions[i] / entry; block_infos.push_back( BlockInfo{partitions[i], block_number, zk_path, zookeeper->asyncTryGet(zk_path)}); } From dc38da5e321bd4b191ebd39ee58ed790eb6f7f41 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 May 2021 14:00:31 +0300 Subject: [PATCH 040/154] Fix fs::copy to be like Poco::copyTo --- src/Disks/DiskLocal.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 31a8d217909..151d0d94597 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -311,13 +311,17 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr & to { if (isSameDiskType(*this, *to_disk)) { - fs::path from = fs::path(disk_path) / from_path; + fs::path to = fs::path(to_disk->getPath()) / to_path; + fs::path from; if (from_path.ends_with('/')) - from = (fs::path(disk_path) / from_path.substr(0, from_path.size() - 1)).parent_path(); - else if (fs::is_directory(from)) - from = from.parent_path(); + from = fs::path(disk_path) / from_path.substr(0, from_path.size() - 1); + else + from = fs::path(disk_path) / from_path; - fs::copy(from, fs::path(to_disk->getPath()) / to_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way. + if (fs::is_directory(from)) + to /= from.filename(); + + fs::copy(from, to, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way. } else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. From 2d4c8b394c85f7ccf0365e93e05f5d92f3d0c263 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 May 2021 14:34:50 +0300 Subject: [PATCH 041/154] Return Poco::moveTo --- src/Disks/S3/DiskS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index faff486eeae..442f015b3ec 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1505,7 +1505,7 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); - fs::rename(fs::path(metadata_path) / path, fs::path(metadata_path) / detached_path); + Poco::File(fs::path(metadata_path) / path).moveTo(fs::path(metadata_path) / detached_path); } } From e26cb4bee02a94a1017ad516a276a4be2d88e2a0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 9 May 2021 13:45:50 +0300 Subject: [PATCH 042/154] Fix style check, fix yandex check --- src/Common/ya.make | 1 + src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 3 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ya.make b/src/Common/ya.make index c6605e436c0..e918db8a8a6 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -93,6 +93,7 @@ SRCS( ZooKeeper/ZooKeeperNodeCache.cpp checkStackSize.cpp clearPasswordFromCommandLine.cpp + createFile.cpp createHardLink.cpp escapeForFileName.cpp filesystemHelpers.cpp diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 37e678a1c20..ba7c8c0cfc1 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -29,7 +29,6 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int TABLE_IS_DROPPED; extern const int TABLE_ALREADY_EXISTS; - extern const int CANNOT_OPEN_FILE; } static const auto suffix = ".removed"; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 56114665fd1..23fe86947ef 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2541,7 +2541,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo source_queue_names = zookeeper->getChildren(fs::path(source_path) / "queue"); /// Check that log pointer of source replica didn't changed while we read queue entries - ops.push_back(zkutil::makeCheckRequest(fs:path(source_path) / "log_pointer", log_pointer_stat.version)); + ops.push_back(zkutil::makeCheckRequest(fs::path(source_path) / "log_pointer", log_pointer_stat.version)); auto rc = zookeeper->tryMulti(ops, responses); From ab1a05a1f4b0f2176469c510dac2c125c8fac815 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 9 May 2021 14:59:49 +0300 Subject: [PATCH 043/154] Poco::Path to fs::path, less concatination --- src/Bridge/IBridgeHelper.cpp | 8 +++++--- src/Common/Config/ConfigProcessor.cpp | 12 ++++++------ src/Databases/DatabaseFactory.cpp | 3 ++- src/Databases/DatabaseLazy.cpp | 2 +- src/Databases/DatabaseOnDisk.cpp | 3 +-- src/Databases/DatabaseReplicated.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 4 +++- src/Interpreters/DDLTask.h | 9 ++++++--- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- src/Storages/Distributed/DirectoryMonitor.cpp | 6 +++--- src/Storages/MergeTree/LeaderElection.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 13 files changed, 36 insertions(+), 29 deletions(-) diff --git a/src/Bridge/IBridgeHelper.cpp b/src/Bridge/IBridgeHelper.cpp index 16886bae4fe..93ca359c578 100644 --- a/src/Bridge/IBridgeHelper.cpp +++ b/src/Bridge/IBridgeHelper.cpp @@ -5,7 +5,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -87,10 +89,10 @@ std::unique_ptr IBridgeHelper::startBridgeCommand() const const auto & config = getConfig(); /// Path to executable folder - Poco::Path path{config.getString("application.dir", "/usr/bin")}; + fs::path path(config.getString("application.dir", "/usr/bin")); std::vector cmd_args; - path.setFileName(serviceFileName()); + path /= serviceFileName(); cmd_args.push_back("--http-port"); cmd_args.push_back(std::to_string(config.getUInt(configPrefix() + ".port", getDefaultPort()))); @@ -126,7 +128,7 @@ std::unique_ptr IBridgeHelper::startBridgeCommand() const LOG_TRACE(getLog(), "Starting {}", serviceAlias()); - return ShellCommand::executeDirect(path.toString(), cmd_args, ShellCommandDestructorStrategy(true)); + return ShellCommand::executeDirect(path.string(), cmd_args, ShellCommandDestructorStrategy(true)); } } diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 36f6a18a85f..d2b1d751422 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -58,7 +58,7 @@ static std::string numberFromHost(const std::string & s) bool ConfigProcessor::isPreprocessedFile(const std::string & path) { - return endsWith(Poco::Path(path).getBaseName(), PREPROCESSED_SUFFIX); + return endsWith(fs::path(path).stem(), PREPROCESSED_SUFFIX); } @@ -412,15 +412,15 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string & { Files files; - Poco::Path merge_dir_path(config_path); + fs::path merge_dir_path(config_path); std::set merge_dirs; /// Add path_to_config/config_name.d dir - merge_dir_path.setExtension("d"); - merge_dirs.insert(merge_dir_path.toString()); + merge_dir_path = merge_dir_path.parent_path() / (merge_dir_path.stem().string() + ".d"); + merge_dirs.insert(merge_dir_path); /// Add path_to_config/conf.d dir - merge_dir_path.setBaseName("conf"); - merge_dirs.insert(merge_dir_path.toString()); + merge_dir_path = merge_dir_path.parent_path() / "conf.d"; + merge_dirs.insert(merge_dir_path); for (const std::string & merge_dir_name : merge_dirs) { diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 3efd92694a6..d179adb18cf 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -61,7 +61,8 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m try { /// Creates store/xxx/ for Atomic - Poco::File(Poco::Path(metadata_path).makeParent()).createDirectories(); + fs::create_directories(fs::path(metadata_path).parent_path()); + /// Before 20.7 it's possible that .sql metadata file does not exist for some old database. /// In this case Ordinary database is created on server startup if the corresponding metadata directory exists. /// So we should remove metadata directory if database creation failed. diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 416a94db929..a0d5f5adbc3 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -229,7 +229,7 @@ StoragePtr DatabaseLazy::loadTable(const String & table_name) const LOG_DEBUG(log, "Load table {} to cache.", backQuote(table_name)); - const String table_metadata_path = getMetadataPath() + "/" + escapeForFileName(table_name) + ".sql"; + const String table_metadata_path = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + ".sql"); try { diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 95cd47e3796..75788cb1a02 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -671,8 +671,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata( auto & create = ast->as(); if (!create.table.empty() && create.uuid != UUIDHelpers::Nil) { - String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName(); - table_name = unescapeForFileName(table_name); + String table_name = unescapeForFileName(fs::path(metadata_file_path).stem()); if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER && logger) LOG_WARNING( diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index b7214917ce8..cce02694635 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -212,7 +212,7 @@ void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(bool force_attach) createDatabaseNodesInZooKeeper(current_zookeeper); } - replica_path = zookeeper_path + "/replicas/" + getFullReplicaName(); + replica_path = fs::path(zookeeper_path) / "replicas" / getFullReplicaName(); String replica_host_id; if (current_zookeeper->tryGet(replica_path, replica_host_id)) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 442f015b3ec..b15b7703557 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1537,7 +1537,9 @@ String DiskS3::revisionToString(UInt64 revision) String DiskS3::pathToDetached(const String & source_path) { - return Poco::Path(source_path).parent().append(Poco::Path("detached")).toString() + '/'; + if (source_path.ends_with('/')) + return fs::path(source_path).parent_path().parent_path() / "detached/"; + return fs::path(source_path).parent_path() / "detached/"; } void DiskS3::onFreeze(const String & path) diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 874a29f051d..9e314eb84c6 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace Poco { @@ -14,6 +15,8 @@ namespace zkutil class ZooKeeper; } +namespace fs = std::filesystem; + namespace DB { @@ -100,9 +103,9 @@ struct DDLTaskBase virtual ContextPtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper); - inline String getActiveNodePath() const { return entry_path + "/active/" + host_id_str; } - inline String getFinishedNodePath() const { return entry_path + "/finished/" + host_id_str; } - inline String getShardNodePath() const { return entry_path + "/shards/" + getShardID(); } + inline String getActiveNodePath() const { return fs::path(entry_path) / "active" / host_id_str; } + inline String getFinishedNodePath() const { return fs::path(entry_path) / "finished" / host_id_str; } + inline String getShardNodePath() const { return fs::path(entry_path) / "shards" / getShardID(); } static String getLogEntryName(UInt32 log_entry_number); static UInt32 getLogEntryNumber(const String & log_entry_name); diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 8a4b3b07692..3eb82048dec 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -542,7 +542,7 @@ void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query) else if (!query.replica_zk_path.empty()) { getContext()->checkAccess(AccessType::SYSTEM_DROP_REPLICA); - auto remote_replica_path = query.replica_zk_path + "/replicas/" + query.replica; + String remote_replica_path = fs::path(query.replica_zk_path) / "replicas" / query.replica; /// This check is actually redundant, but it may prevent from some user mistakes for (auto & elem : DatabaseCatalog::instance().getDatabases()) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index bae81a56b30..e8835132f8f 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -958,13 +958,13 @@ void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_p const auto last_path_separator_pos = file_path.rfind('/'); const auto & base_path = file_path.substr(0, last_path_separator_pos + 1); const auto & file_name = file_path.substr(last_path_separator_pos + 1); - const auto & broken_path = base_path + "broken/"; - const auto & broken_file_path = broken_path + file_name; + const String & broken_path = fs::path(base_path) / "broken/"; + const String & broken_file_path = fs::path(broken_path) / file_name; fs::create_directory(broken_path); auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); - auto broken_dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path + "/broken/"); + auto broken_dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, fs::path(relative_path) / "broken/"); { std::lock_guard status_lock(status_mutex); diff --git a/src/Storages/MergeTree/LeaderElection.h b/src/Storages/MergeTree/LeaderElection.h index 7cdfd8e566d..2810385e9e5 100644 --- a/src/Storages/MergeTree/LeaderElection.h +++ b/src/Storages/MergeTree/LeaderElection.h @@ -83,7 +83,7 @@ private: void createNode() { shutdown_called = false; - node = EphemeralNodeHolder::createSequential(path + "/leader_election-", zookeeper, identifier); + node = EphemeralNodeHolder::createSequential(fs::path(path) / "leader_election-", zookeeper, identifier); std::string node_path = node->getPath(); node_name = node_path.substr(node_path.find_last_of('/') + 1); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1849b2efebd..c9ad3f10bd8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3232,7 +3232,7 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr for (auto & [old_name, new_name] : renamed_parts.old_and_new_names) { const auto & [path, disk] = renamed_parts.old_part_name_to_path_and_disk[old_name]; - disk->removeRecursive(path + "detached/" + new_name + "/"); + disk->removeRecursive(fs::path(path) / "detached" / new_name / ""); LOG_DEBUG(log, "Dropped detached part {}", old_name); old_name.clear(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 23fe86947ef..64f800dc05a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -254,7 +254,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , zookeeper_name(extractZooKeeperName(zookeeper_path_)) , zookeeper_path(extractZooKeeperPath(zookeeper_path_)) , replica_name(replica_name_) - , replica_path(zookeeper_path + "/replicas/" + replica_name_) + , replica_path(fs::path(zookeeper_path) / "replicas" / replica_name_) , reader(*this) , writer(*this) , merger_mutator(*this, getContext()->getSettingsRef().background_pool_size) @@ -473,14 +473,14 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( /// Mutation maybe killed or whole replica was deleted. /// Wait event will unblock at this moment. Coordination::Stat exists_stat; - if (!getZooKeeper()->exists(zookeeper_path + "/mutations/" + mutation_id, &exists_stat, wait_event)) + if (!getZooKeeper()->exists(fs::path(zookeeper_path) / "mutations" / mutation_id, &exists_stat, wait_event)) { throw Exception(ErrorCodes::UNFINISHED, "Mutation {} was killed, manually removed or table was dropped", mutation_id); } auto zookeeper = getZooKeeper(); /// Replica could be inactive. - if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (!zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active")) { LOG_WARNING(log, "Replica {} is not active during mutation. Mutation will be done asynchronously when replica becomes active.", replica); @@ -488,7 +488,7 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( break; } - String mutation_pointer = zookeeper_path + "/replicas/" + replica + "/mutation_pointer"; + String mutation_pointer = fs::path(zookeeper_path) / "replicas" / replica / "mutation_pointer"; std::string mutation_pointer_value; /// Replica could be removed if (!zookeeper->tryGet(mutation_pointer, mutation_pointer_value, nullptr, wait_event)) @@ -512,7 +512,7 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( /// It maybe already removed from zk, but local in-memory mutations /// state was not updated. - if (!getZooKeeper()->exists(zookeeper_path + "/mutations/" + mutation_id)) + if (!getZooKeeper()->exists(fs::path(zookeeper_path) / "mutations" / mutation_id)) { throw Exception(ErrorCodes::UNFINISHED, "Mutation {} was killed, manually removed or table was dropped", mutation_id); } From f03f591e559a25da6e849fa8455b191fa4250b29 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 10 May 2021 16:39:32 +0300 Subject: [PATCH 044/154] Upd comments for StoreageJoin --- src/Interpreters/HashJoin.cpp | 2 +- src/Storages/StorageJoin.cpp | 4 +++- tests/queries/0_stateless/00118_storage_join.sql | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index cc500f11ce9..d9cbffbc2b7 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -717,7 +717,7 @@ public: for (const auto & src_column : block_with_columns_to_add) { - /// Column names `src_column.name` and `qualified_name` can be differ for StorageJoin, + /// Column names `src_column.name` and `qualified_name` can differ for StorageJoin, /// because it uses not qualified right block column names auto qualified_name = join.getTableJoin().renamedRightColumnName(src_column.name); /// Don't insert column if it's in left block diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index d9970bab22c..dda2cc39d6b 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -92,7 +92,9 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join) /// TODO: check key columns - /// Some HACK to remove wrong names qualifiers: table.column -> column. + /// Set names qualifiers: table.column -> column + /// It's required because storage join stores non-qualified names + /// Qualifies will be added by join implementation (HashJoin) analyzed_join->setRightKeys(key_names); HashJoinPtr join_clone = std::make_shared(analyzed_join, metadata_snapshot->getSampleBlock().sortColumns()); diff --git a/tests/queries/0_stateless/00118_storage_join.sql b/tests/queries/0_stateless/00118_storage_join.sql index d5496687728..0381d3a8e1a 100644 --- a/tests/queries/0_stateless/00118_storage_join.sql +++ b/tests/queries/0_stateless/00118_storage_join.sql @@ -13,7 +13,7 @@ SELECT k, js1.s, t2.s FROM (SELECT toUInt64(number / 3) AS k, sum(number) as s F SELECT k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; --- geting qualified key columns from Join table still doen't work +-- getting qualified key columns from Join table still doen't work SELECT t2.k FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; -- { serverError 8 } DROP TABLE t2; From cb89a2be9783de646c95ee8c424ce6a664c7dffc Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 10 May 2021 17:56:12 +0300 Subject: [PATCH 045/154] StorageJoin clashing column name with JOIN ON --- src/Interpreters/HashJoin.cpp | 6 ++++-- src/Interpreters/TableJoin.cpp | 5 ++++- .../0_stateless/00118_storage_join.reference | 20 +++++++++++++++++++ .../0_stateless/00118_storage_join.sql | 6 +++--- 4 files changed, 31 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index d9cbffbc2b7..4cbf0886d6b 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1096,7 +1096,8 @@ void HashJoin::joinBlockImpl( const auto & col = block.getByName(left_name); bool is_nullable = nullable_right_side || right_key.type->isNullable(); - block.insert(correctNullability({col.column, col.type, right_key.name}, is_nullable)); + auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name); + block.insert(correctNullability({col.column, col.type, right_col_name}, is_nullable)); } } else if (has_required_right_keys) @@ -1121,7 +1122,8 @@ void HashJoin::joinBlockImpl( bool is_nullable = nullable_right_side || right_key.type->isNullable(); ColumnPtr thin_column = filterWithBlanks(col.column, filter); - block.insert(correctNullability({thin_column, col.type, right_key.name}, is_nullable, null_map_filter)); + auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name); + block.insert(correctNullability({thin_column, col.type, right_col_name}, is_nullable, null_map_filter)); if constexpr (need_replication) right_keys_to_replicate.push_back(block.getPositionByName(right_key.name)); diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index a088e4d0239..122e2cd6479 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -156,9 +156,12 @@ NameSet TableJoin::requiredRightKeys() const { NameSet required; for (const auto & name : key_names_right) + { + auto rename = renamedRightColumnName(name); for (const auto & column : columns_added_by_join) - if (name == column.name) + if (rename == column.name) required.insert(name); + } return required; } diff --git a/tests/queries/0_stateless/00118_storage_join.reference b/tests/queries/0_stateless/00118_storage_join.reference index cd7e2d67523..56920b290e6 100644 --- a/tests/queries/0_stateless/00118_storage_join.reference +++ b/tests/queries/0_stateless/00118_storage_join.reference @@ -28,6 +28,16 @@ 7 7 8 8 9 9 +0 0 0 +1 1 1 abc +2 2 2 def +3 0 3 +4 0 4 +5 0 5 +6 6 6 ghi +7 0 7 +8 0 8 +9 0 9 0 3 3 9 2 21 def @@ -44,3 +54,13 @@ 7 7 8 8 9 9 +0 0 0 +1 1 1 abc +2 2 2 def +3 0 3 +4 0 4 +5 0 5 +6 6 6 ghi +7 0 7 +8 0 8 +9 0 9 diff --git a/tests/queries/0_stateless/00118_storage_join.sql b/tests/queries/0_stateless/00118_storage_join.sql index 0381d3a8e1a..47896d3316c 100644 --- a/tests/queries/0_stateless/00118_storage_join.sql +++ b/tests/queries/0_stateless/00118_storage_join.sql @@ -9,11 +9,11 @@ INSERT INTO t2 VALUES (6, 'ghi'); SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k; SELECT k, js1.s, t2.s FROM (SELECT number AS k, number as s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k; +SELECT k, t2.k, js1.s, t2.s FROM (SELECT number AS k, number as s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k; + SELECT k, js1.s, t2.s FROM (SELECT toUInt64(number / 3) AS k, sum(number) as s FROM numbers(10) GROUP BY toUInt64(number / 3) WITH TOTALS) js1 ANY LEFT JOIN t2 USING k; SELECT k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; - --- getting qualified key columns from Join table still doen't work -SELECT t2.k FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; -- { serverError 8 } +SELECT k, t2.k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; DROP TABLE t2; From 3439492700dcd0b974e29ec867d84acf6efb75da Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 11 May 2021 19:14:58 +0300 Subject: [PATCH 046/154] fs::last_write_time --- src/Common/Config/ConfigReloader.cpp | 5 ++++- src/Interpreters/ExternalLoaderXMLConfigRepository.cpp | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index ad0ce7cc30e..b2aeb899e4d 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -170,7 +170,10 @@ struct ConfigReloader::FileWithTimestamp void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path_to_add) { if (!path_to_add.empty() && fs::exists(path_to_add)) - files.emplace(path_to_add, Poco::File(path_to_add).getLastModified().epochTime()); + { + fs::file_time_type fs_time = fs::last_write_time(path_to_add); + files.emplace(path_to_add, fs::file_time_type::clock::to_time_t(fs_time)); + } } bool ConfigReloader::FilesChangesTracker::isDifferOrNewerThan(const FilesChangesTracker & rhs) diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index e1052817407..e4f3447c6df 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -21,7 +21,9 @@ ExternalLoaderXMLConfigRepository::ExternalLoaderXMLConfigRepository( Poco::Timestamp ExternalLoaderXMLConfigRepository::getUpdateTime(const std::string & definition_entity_name) { - return Poco::File(definition_entity_name).getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(definition_entity_name); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } std::set ExternalLoaderXMLConfigRepository::getAllLoadablesDefinitionNames() From 81797393f1d03f299ca827b47a92e16942945750 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 12 May 2021 09:19:11 +0300 Subject: [PATCH 047/154] fs::last_write_time, fs::space --- src/Common/Config/ConfigProcessor.cpp | 6 +++--- src/Common/filesystemHelpers.cpp | 6 +----- src/Databases/DatabaseOnDisk.cpp | 8 +++++--- src/Interpreters/DatabaseCatalog.cpp | 3 ++- 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index d2b1d751422..d0b85cc7506 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -642,9 +642,9 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, } preprocessed_path = (fs::path(preprocessed_dir) / fs::path(new_path)).string(); - auto preprocessed_path_parent = Poco::Path(preprocessed_path).makeParent(); - if (!preprocessed_path_parent.toString().empty()) - Poco::File(preprocessed_path_parent).createDirectories(); + auto preprocessed_path_parent = fs::path(preprocessed_path).parent_path(); + if (!preprocessed_path_parent.string().empty()) + fs::create_directories(preprocessed_path_parent); } DOMWriter().writeNode(preprocessed_path, loaded_config.preprocessed_xml); LOG_DEBUG(log, "Saved preprocessed configuration to '{}'.", preprocessed_path); diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 835668dfa1c..b146392e1a6 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -40,12 +40,8 @@ struct statvfs getStatVFS(const String & path) bool enoughSpaceInDirectory(const std::string & path [[maybe_unused]], size_t data_size [[maybe_unused]]) { -#if POCO_VERSION >= 0x01090000 - auto free_space = Poco::File(path).freeSpace(); + auto free_space = fs::space(path).free; return data_size <= free_space; -#else - return true; -#endif } std::unique_ptr createTemporaryFile(const std::string & path) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 75788cb1a02..2ed6a0d9b6d 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -540,10 +540,12 @@ String DatabaseOnDisk::getObjectMetadataPath(const String & object_name) const time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_name) const { String table_metadata_path = getObjectMetadataPath(object_name); - Poco::File meta_file(table_metadata_path); - if (meta_file.exists()) - return meta_file.getLastModified().epochTime(); + if (fs::exists(table_metadata_path)) + { + fs::file_time_type fs_time = fs::last_write_time(table_metadata_path); + return fs::file_time_type::clock::to_time_t(fs_time); + } else return static_cast(0); } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 6d6b8107f0c..45a803934b5 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -782,7 +782,8 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr } addUUIDMapping(table_id.uuid); - drop_time = Poco::File(dropped_metadata_path).getLastModified().epochTime(); + fs::file_time_type fs_time = fs::last_write_time(dropped_metadata_path); + drop_time = fs::file_time_type::clock::to_time_t(fs_time); } std::lock_guard lock(tables_marked_dropped_mutex); From 17432b1a4177397d173e7a5d41b42596628a02e1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 12 May 2021 17:32:22 +0300 Subject: [PATCH 048/154] Poco::moveTo to fs version --- src/Disks/S3/DiskS3.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index b15b7703557..4fbfc9141a6 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1505,7 +1505,14 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); - Poco::File(fs::path(metadata_path) / path).moveTo(fs::path(metadata_path) / detached_path); + fs::path from_path = fs::path(metadata_path) / path; + fs::path to_path = fs::path(metadata_path) / detached_path; + if (path.ends_with('/')) + to_path /= from_path.parent_path().filename(); + else + to_path /= from_path.filename(); + fs::copy(from_path, to_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); + fs::remove_all(from_path); } } From e466924898fe94d724712ad9e0def8a3365910bd Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 13 May 2021 02:01:35 +0300 Subject: [PATCH 049/154] fs::premissions --- src/Disks/DiskLocal.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 151d0d94597..bf0a8bac12f 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -299,7 +299,9 @@ void DiskLocal::createFile(const String & path) void DiskLocal::setReadOnly(const String & path) { - Poco::File(fs::path(disk_path) / path).setReadOnly(true); + fs::permissions(fs::path(disk_path) / path, + fs::perms::owner_read | fs::perms::group_read | fs::perms::others_read, + fs::perm_options::replace); } bool inline isSameDiskType(const IDisk & one, const IDisk & another) From b0a54989afdc7a987766584ca327119672dbf7ad Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Thu, 13 May 2021 10:52:50 +0200 Subject: [PATCH 050/154] Link edited --- docs/en/sql-reference/statements/optimize.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 0eacc7e57b4..12d2173a9d8 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -31,7 +31,7 @@ When `OPTIMIZE` is used with the [ReplicatedMergeTree](../../engines/table-engin If you want to perform deduplication on custom set of columns rather than on all, you can specify list of columns explicitly or use any combination of [`*`](../../sql-reference/statements/select/index.md#asterisk), [`COLUMNS`](../../sql-reference/statements/select/index.md#columns-expression) or [`EXCEPT`](../../sql-reference/statements/select/index.md#except-modifier) expressions. The explictly written or implicitly expanded list of columns must include all columns specified in row ordering expression (both primary and sorting keys) and partitioning expression (partitioning key). !!! note "Note" - Notice that `*` behaves just like in `SELECT`: [`MATERIALIZED`](./create/table/#materialized) and [`ALIAS`](./create/table/#alias) columns are not used for expansion. + Notice that `*` behaves just like in `SELECT`: [`MATERIALIZED`](../../sql-reference/statements/create/table.md#materialized) and [`ALIAS`](../../sql-reference/statements/create/table.md#alias) columns are not used for expansion. Also, it is an error to specify empty list of columns, or write an expression that results in an empty list of columns, or deduplicate by an `ALIAS` column. **Syntax** From b123ec135401844d306ceefd5d19bcda19c33517 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 13 May 2021 12:32:52 +0300 Subject: [PATCH 051/154] Poco::FileException, Poco::parentPath --- src/Common/Config/ConfigProcessor.cpp | 10 +++++----- src/Common/Exception.cpp | 25 ++++++++----------------- src/Disks/IDisk.h | 2 +- 3 files changed, 14 insertions(+), 23 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index d0b85cc7506..f85e22d28b6 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -623,11 +623,11 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, if (!loaded_config.configuration->has("path")) { // Will use current directory - auto parent_path = Poco::Path(loaded_config.config_path).makeParent(); - preprocessed_dir = parent_path.toString(); - Poco::Path poco_new_path(new_path); - poco_new_path.setBaseName(poco_new_path.getBaseName() + PREPROCESSED_SUFFIX); - new_path = poco_new_path.toString(); + fs::path parent_path = fs::path(loaded_config.config_path).parent_path(); + preprocessed_dir = parent_path.string(); + fs::path fs_new_path(new_path); + fs_new_path = fs_new_path.parent_path() / (fs_new_path.stem().string() + PREPROCESSED_SUFFIX + fs_new_path.extension().string()); + new_path = fs_new_path.string(); } else { diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index f17eaea8d7f..4da84e92f2a 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -21,6 +21,8 @@ # include #endif +namespace fs = std::filesystem; + namespace DB { @@ -164,7 +166,7 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_ } } -static void getNoSpaceLeftInfoMessage(std::filesystem::path path, std::string & msg) +static void getNoSpaceLeftInfoMessage(std::filesystem::path path, String & msg) { path = std::filesystem::absolute(path); /// It's possible to get ENOSPC for non existent file (e.g. if there are no free inodes and creat() fails) @@ -251,23 +253,12 @@ static std::string getExtraExceptionInfo(const std::exception & e) String msg; try { - /// TODO: this has to be adjusted for std::filesystem - if (const auto * file_exception = dynamic_cast(&e)) + if (const auto * file_exception = dynamic_cast(&e)) { - if (file_exception->code() == ENOSPC) - { - /// See Poco::FileImpl::handleLastErrorImpl(...) - constexpr const char * expected_error_message = "no space left on device: "; - if (startsWith(file_exception->message(), expected_error_message)) - { - String path = file_exception->message().substr(strlen(expected_error_message)); - getNoSpaceLeftInfoMessage(path, msg); - } - else - { - msg += "\nCannot print extra info for Poco::Exception"; - } - } + if (file_exception->code() == std::errc::no_space_on_device) + getNoSpaceLeftInfoMessage(file_exception->path1(), msg); + else + msg += "\nCannot print extra info for Poco::Exception"; } else if (const auto * errno_exception = dynamic_cast(&e)) { diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 62ab54e90c2..459991d6afd 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -301,7 +301,7 @@ inline String fullPath(const DiskPtr & disk, const String & path) /// Return parent path for the specified path. inline String parentPath(const String & path) { - return Poco::Path(path).parent().toString(); + return fs::path(path).parent_path() / ""; } /// Return file name for the specified path. From db849eb3e5ec110645d1561e38cd0dc102d4def2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 14 May 2021 10:06:38 +0300 Subject: [PATCH 052/154] Remaining poco --- src/Disks/IDisk.h | 2 ++ src/Interpreters/ExternalLoaderXMLConfigRepository.cpp | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 459991d6afd..2e83bf0ec26 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -301,6 +301,8 @@ inline String fullPath(const DiskPtr & disk, const String & path) /// Return parent path for the specified path. inline String parentPath(const String & path) { + if (path.ends_with('/')) + return fs::path(path).parent_path().parent_path() / ""; return fs::path(path).parent_path() / ""; } diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index e4f3447c6df..4c6af554fae 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -40,8 +40,8 @@ std::set ExternalLoaderXMLConfigRepository::getAllLoadablesDefiniti if (pattern[0] != '/') { const auto app_config_path = main_config.getString("config-file", "config.xml"); - const auto config_dir = Poco::Path{app_config_path}.parent().toString(); - const auto absolute_path = config_dir + pattern; + const String config_dir = fs::path(app_config_path).parent_path(); + const String absolute_path = fs::path(config_dir) / pattern; Poco::Glob::glob(absolute_path, files, 0); if (!files.empty()) continue; From 018d52d71f7d54974f864356ea4671e26969126b Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 15 May 2021 09:20:26 +0300 Subject: [PATCH 053/154] Directory path --- src/Disks/IDisk.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 2e83bf0ec26..e299fffbf41 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -14,7 +14,6 @@ #include #include #include -#include #include "Poco/Util/AbstractConfiguration.h" namespace fs = std::filesystem; @@ -315,7 +314,7 @@ inline String fileName(const String & path) /// Return directory path for the specified path. inline String directoryPath(const String & path) { - return Poco::Path(path).setFileName("").toString(); + return fs::path(path).parent_path() / ""; } } From add5970b5f0fc092f8c5fbd213e2ce97aa72bf27 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 16 May 2021 03:12:13 +0300 Subject: [PATCH 054/154] fs::absolute --- .../Embedded/GeodataProviders/HierarchiesProvider.cpp | 9 ++++----- src/Storages/MergeTree/DataPartsExchange.cpp | 4 ++-- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp index 7257caa066f..3f35b9102a2 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp @@ -30,13 +30,12 @@ RegionsHierarchiesDataProvider::RegionsHierarchiesDataProvider(const std::string void RegionsHierarchiesDataProvider::discoverFilesWithCustomHierarchies() { std::string basename = fs::path(path).stem(); + fs::path dir_path = fs::absolute(path).parent_path(); - Poco::Path dir_path = Poco::Path(path).absolute().parent(); - - Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator dir_it(dir_path); dir_it != dir_end; ++dir_it) + fs::directory_iterator dir_end; + for (fs::directory_iterator dir_it(dir_path); dir_it != dir_end; ++dir_it) { - std::string candidate_basename = dir_it.path().getBaseName(); + std::string candidate_basename = dir_it->path().stem(); if (candidate_basename.starts_with(basename) && (candidate_basename.size() > basename.size() + 1) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 7f483fca681..69a92738777 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -652,8 +652,8 @@ void Fetcher::downloadBaseOrProjectionPartToDisk( /// File must be inside "absolute_part_path" directory. /// Otherwise malicious ClickHouse replica may force us to write to arbitrary path. - String absolute_file_path = Poco::Path(part_download_path + file_name).absolute().toString(); - if (!startsWith(absolute_file_path, Poco::Path(part_download_path).absolute().toString())) + String absolute_file_path = fs::absolute(fs::path(part_download_path) / file_name); + if (!startsWith(absolute_file_path, fs::absolute(part_download_path).string())) throw Exception("File path (" + absolute_file_path + ") doesn't appear to be inside part path (" + part_download_path + ")." " This may happen if we are trying to download part from malicious replica or logical error.", ErrorCodes::INSECURE_PATH); From 3b1bf2bae6c7a5c7be7c00e15c4f233916452150 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 16 May 2021 21:36:55 +0300 Subject: [PATCH 055/154] Poco::Path substitution --- src/Formats/FormatSchemaInfo.cpp | 55 +++++++++++++++--------- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- 2 files changed, 36 insertions(+), 23 deletions(-) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 707f9babe8d..81c11eda0b4 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include namespace DB @@ -11,6 +13,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace fs = std::filesystem; namespace { @@ -34,55 +37,65 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & String default_file_extension = getFormatSchemaDefaultFileExtension(format); - Poco::Path path; + fs::path path; if (require_message) { size_t colon_pos = format_schema.find(':'); - if ((colon_pos == String::npos) || (colon_pos == 0) || (colon_pos == format_schema.length() - 1) - || path.assign(format_schema.substr(0, colon_pos)).makeFile().getFileName().empty()) + if ((colon_pos == String::npos) || (colon_pos == 0) || (colon_pos == format_schema.length() - 1)) { throw Exception( "Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format" + (default_file_extension.empty() ? "" : ", e.g. 'schema." + default_file_extension + ":Message'") + - ". Got '" + format_schema - + "'", - ErrorCodes::BAD_ARGUMENTS); + ". Got '" + format_schema + "'", ErrorCodes::BAD_ARGUMENTS); + } + else + { + path = fs::path(format_schema.substr(0, colon_pos)); + String filename = path.has_filename() ? path.filename() : path.parent_path().filename(); + if (filename.empty()) + throw Exception( + "Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format" + + (default_file_extension.empty() ? "" : ", e.g. 'schema." + default_file_extension + ":Message'") + + ". Got '" + format_schema + "'", ErrorCodes::BAD_ARGUMENTS); } - message_name = format_schema.substr(colon_pos + 1); } else - path.assign(format_schema).makeFile().getFileName(); + { + path = fs::path(format_schema); + if (!path.has_filename()) + path = path.parent_path() / ""; + } auto default_schema_directory = [&format_schema_path]() { - static const String str = Poco::Path(format_schema_path).makeAbsolute().makeDirectory().toString(); + static const String str = fs::absolute(format_schema_path) / ""; return str; }; - if (path.getExtension().empty() && !default_file_extension.empty()) - path.setExtension(default_file_extension); + if (!path.has_extension() && !default_file_extension.empty()) + path = path.parent_path() / (path.stem().string() + '.' + default_file_extension); - if (path.isAbsolute()) + if (path.is_absolute()) { if (is_server) - throw Exception("Absolute path in the 'format_schema' setting is prohibited: " + path.toString(), ErrorCodes::BAD_ARGUMENTS); - schema_path = path.getFileName(); - schema_directory = path.makeParent().toString(); + throw Exception("Absolute path in the 'format_schema' setting is prohibited: " + path.string(), ErrorCodes::BAD_ARGUMENTS); + schema_path = path.filename(); + schema_directory = path.parent_path() / ""; } - else if (path.depth() >= 1 && path.directory(0) == "..") + else if (Poco::Path(path.string()).depth() >= 1 && Poco::Path(path.string()).directory(0) == "..") { if (is_server) throw Exception( - "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.toString(), + "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(), ErrorCodes::BAD_ARGUMENTS); - path = Poco::Path(default_schema_directory()).resolve(path).toString(); - schema_path = path.getFileName(); - schema_directory = path.makeParent().toString(); + path = Poco::Path(default_schema_directory()).resolve(Poco::Path(path.string())).toString(); + schema_path = path.filename(); + schema_directory = path.parent_path() / ""; } else { - schema_path = path.toString(); + schema_path = path; schema_directory = default_schema_directory(); } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fc752030b89..374fd473599 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3415,8 +3415,8 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const if (!containing_part.empty() && containing_part != name) // TODO maybe use PartsTemporaryRename here? - disk->moveDirectory(relative_data_path + source_dir + name, - relative_data_path + source_dir + "inactive_" + name); + disk->moveDirectory(fs::path(relative_data_path) / source_dir / name, + fs::path(relative_data_path) / source_dir / ("inactive_" + name)); else renamed_parts.addPart(name, "attaching_" + name); } From 2124113aa29ed8a25d0730deea43700e5eba380f Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 May 2021 01:06:09 +0300 Subject: [PATCH 056/154] Update programs/* --- programs/benchmark/Benchmark.cpp | 7 +++--- programs/client/Client.cpp | 9 +++++--- programs/copier/ClusterCopierApp.cpp | 12 ++++++---- programs/local/LocalServer.cpp | 12 +++++----- programs/server/Server.cpp | 34 ++++++++++++++++------------ 5 files changed, 42 insertions(+), 32 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 1d2b579db3a..498d1eecae2 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -36,7 +35,9 @@ #include #include #include +#include +namespace fs = std::filesystem; /** A tool for evaluating ClickHouse performance. * The tool emulates a case with fixed amount of simultaneously executing queries. @@ -119,8 +120,8 @@ public: int main(const std::vector &) override { - if (!json_path.empty() && Poco::File(json_path).exists()) /// Clear file with previous results - Poco::File(json_path).remove(); + if (!json_path.empty() && fs::exists(json_path)) /// Clear file with previous results + fs::remove(json_path); readQueries(); runBenchmark(); diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index aae2e0f1c59..0e4b577c40e 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -87,6 +87,8 @@ #include #include #include +#include +#include #if !defined(ARCADIA_BUILD) # include @@ -96,6 +98,7 @@ #pragma GCC optimize("-fno-var-tracking-assignments") #endif +namespace fs = std::filesystem; namespace DB { @@ -276,7 +279,7 @@ private: /// Set path for format schema files if (config().has("format_schema_path")) - context->setFormatSchemaPath(Poco::Path(config().getString("format_schema_path")).toString()); + context->setFormatSchemaPath(config().getString("format_schema_path")); /// Initialize query_id_formats if any if (config().has("query_id_formats")) @@ -633,8 +636,8 @@ private: history_file = home_path + "/.clickhouse-client-history"; } - if (!history_file.empty() && !Poco::File(history_file).exists()) - Poco::File(history_file).createFile(); + if (!history_file.empty() && !fs::exists(history_file)) + fs::createFile(history_file); LineReader::Patterns query_extenders = {"\\"}; LineReader::Patterns query_delimiters = {";", "\\G"}; diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index d3fff616b65..79bfb5b7411 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -5,7 +5,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -26,7 +28,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); if (config().has("move-fault-probability")) move_fault_probability = std::max(std::min(config().getDouble("move-fault-probability"), 1.0), 0.0); - base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current(); + base_dir = (config().has("base-dir")) ? config().getString("base-dir") : fs::current_path().string(); if (config().has("experimental-use-sample-offset")) @@ -38,18 +40,18 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id; - process_path = Poco::Path(base_dir + "/clickhouse-copier_" + process_id).absolute().toString(); - Poco::File(process_path).createDirectories(); + process_path = fs::absolute(fs::path(base_dir) / ("clickhouse-copier_" + process_id)); + fs::create_directories(process_path); /// Override variables for BaseDaemon if (config().has("log-level")) config().setString("logger.level", config().getString("log-level")); if (config().has("base-dir") || !config().has("logger.log")) - config().setString("logger.log", process_path + "/log.log"); + config().setString("logger.log", fs::path(process_path) / "log.log"); if (config().has("base-dir") || !config().has("logger.errorlog")) - config().setString("logger.errorlog", process_path + "/log.err.log"); + config().setString("logger.errorlog", fs::path(process_path) / "log.err.log"); Base::initialize(self); } diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 043cc596e2b..c444f89e08a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -42,9 +42,9 @@ #include #include #include - #include +namespace fs = std::filesystem; namespace DB { @@ -72,11 +72,11 @@ void LocalServer::initialize(Poco::Util::Application & self) Poco::Util::Application::initialize(self); /// Load config files if exists - if (config().has("config-file") || Poco::File("config.xml").exists()) + if (config().has("config-file") || fs::exists("config.xml")) { const auto config_path = config().getString("config-file", "config.xml"); ConfigProcessor config_processor(config_path, false, true); - config_processor.setConfigPath(Poco::Path(config_path).makeParent().toString()); + config_processor.setConfigPath(fs::path(config_path).parent_path()); auto loaded_config = config_processor.loadConfig(); config_processor.savePreprocessedConfig(loaded_config, loaded_config.configuration->getString("path", ".")); config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); @@ -287,8 +287,8 @@ try status.emplace(path + "status", StatusFile::write_full_info); LOG_DEBUG(log, "Loading metadata from {}", path); - Poco::File(path + "data/").createDirectories(); - Poco::File(path + "metadata/").createDirectories(); + fs::create_directories(fs::path(path) / "data/"); + fs::create_directories(fs::path(path) / "metadata/"); loadMetadataSystem(global_context); attachSystemTables(global_context); loadMetadata(global_context); @@ -479,7 +479,7 @@ void LocalServer::setupUsers() { ConfigurationPtr users_config; - if (config().has("users_config") || config().has("config-file") || Poco::File("config.xml").exists()) + if (config().has("users_config") || config().has("config-file") || fs::exists("config.xml")) { const auto users_config_path = config().getString("users_config", config().getString("config-file", "config.xml")); ConfigProcessor config_processor(users_config_path); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index a36287a8051..8668f09ca06 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -74,6 +74,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) @@ -117,6 +118,8 @@ namespace CurrentMetrics extern const Metric MaxDDLEntryID; } +namespace fs = std::filesystem; + #if USE_JEMALLOC static bool jemallocOptionEnabled(const char *name) { @@ -183,7 +186,7 @@ void setupTmpPath(Poco::Logger * log, const std::string & path) { LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); - Poco::File(path).createDirectories(); + fs::create_directories(path); /// Clearing old temporary files. Poco::DirectoryIterator dir_end; @@ -678,37 +681,38 @@ int Server::main(const std::vector & /*args*/) * Examples: do repair of local data; clone all replicated tables from replica. */ { - Poco::File(path + "flags/").createDirectories(); - global_context->setFlagsPath(path + "flags/"); + auto flags_path = fs::path(path) / "flags/"; + fs::create_directories(flags_path); + global_context->setFlagsPath(flags_path); } /** Directory with user provided files that are usable by 'file' table function. */ { - std::string user_files_path = config().getString("user_files_path", path + "user_files/"); + std::string user_files_path = config().getString("user_files_path", fs::path(path) / "user_files/"); global_context->setUserFilesPath(user_files_path); - Poco::File(user_files_path).createDirectories(); + fs::create_directories(user_files_path); } { - std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", path + "dictionaries_lib/"); + std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", fs::path(path) / "dictionaries_lib/"); global_context->setDictionariesLibPath(dictionaries_lib_path); - Poco::File(dictionaries_lib_path).createDirectories(); + fs::create_directories(dictionaries_lib_path); } /// top_level_domains_lists { - const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/") + "/"; - TLDListsHolder::getInstance().parseConfig(top_level_domains_path, config()); + const std::string & top_level_domains_path = config().getString("top_level_domains_path", fs::path(path) / "top_level_domains/"); + TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config()); } { - Poco::File(path + "data/").createDirectories(); - Poco::File(path + "metadata/").createDirectories(); + fs::create_directories(fs::path(path) / "data/"); + fs::create_directories(fs::path(path) / "metadata/"); /// Directory with metadata of tables, which was marked as dropped by Atomic database - Poco::File(path + "metadata_dropped/").createDirectories(); + fs::create_directories(fs::path(path) / "metadata_dropped/"); } if (config().has("interserver_http_port") && config().has("interserver_https_port")) @@ -891,9 +895,9 @@ int Server::main(const std::vector & /*args*/) #endif /// Set path for format schema files - auto format_schema_path = Poco::File(config().getString("format_schema_path", path + "format_schemas/")); - global_context->setFormatSchemaPath(format_schema_path.path()); - format_schema_path.createDirectories(); + fs::path format_schema_path(config().getString("format_schema_path", fs::path(path) / "format_schemas/")); + global_context->setFormatSchemaPath(format_schema_path); + fs::create_directories(format_schema_path); /// Check sanity of MergeTreeSettings on server startup global_context->getMergeTreeSettings().sanityCheck(settings); From 649dd23b8b7a5c97b39bf8f79a791d880436d540 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 May 2021 09:48:02 +0300 Subject: [PATCH 057/154] Poco::resolve --- src/Formats/FormatSchemaInfo.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 81c11eda0b4..ae1eb422d27 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -89,7 +89,11 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & throw Exception( "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(), ErrorCodes::BAD_ARGUMENTS); - path = Poco::Path(default_schema_directory()).resolve(Poco::Path(path.string())).toString(); + fs::path default_schema_directory_path(default_schema_directory()); + if (default_schema_directory_path.is_absolute()) + path = default_schema_directory_path; + else + path /= default_schema_directory_path; schema_path = path.filename(); schema_directory = path.parent_path() / ""; } From e27acc26be115f954dce2f402b048b0d6a6d9841 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 17 May 2021 17:26:36 +0300 Subject: [PATCH 058/154] better dropPart and dropPartition --- src/Storages/MergeTree/MergeTreeData.cpp | 19 +++++-- src/Storages/MergeTree/MergeTreeData.h | 8 ++- src/Storages/StorageMergeTree.cpp | 45 ++++++--------- src/Storages/StorageMergeTree.h | 5 +- src/Storages/StorageReplicatedMergeTree.cpp | 62 +++++++++++---------- src/Storages/StorageReplicatedMergeTree.h | 9 ++- 6 files changed, 78 insertions(+), 70 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f17fb077b85..5acb6065e58 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1386,7 +1386,7 @@ void MergeTreeData::clearEmptyParts() for (const auto & part : parts) { if (part->rows_count == 0) - dropPart(part->name); + dropPartNoWaitNoThrow(part->name); } } @@ -2883,9 +2883,8 @@ void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition) getContext()->checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size); } -void MergeTreeData::checkPartCanBeDropped(const ASTPtr & part_ast) +void MergeTreeData::checkPartCanBeDropped(const String & part_name) { - String part_name = part_ast->as().value.safeGet(); auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Committed}); if (!part) throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in committed state", part_name); @@ -3015,12 +3014,20 @@ Pipe MergeTreeData::alterPartition( switch (command.type) { case PartitionCommand::DROP_PARTITION: + { if (command.part) - checkPartCanBeDropped(command.partition); + { + auto part_name = command.partition->as().value.safeGet(); + checkPartCanBeDropped(part_name); + dropPart(part_name, command.detach, query_context); + } else + { checkPartitionCanBeDropped(command.partition); - dropPartition(command.partition, command.detach, command.part, query_context); - break; + dropPartition(command.partition, command.detach, query_context); + } + } + break; case PartitionCommand::DROP_DETACHED_PARTITION: dropDetached(command.partition, command.part, query_context); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index cf15016404f..87613f82689 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -610,7 +610,7 @@ public: void checkPartitionCanBeDropped(const ASTPtr & partition) override; - void checkPartCanBeDropped(const ASTPtr & part); + void checkPartCanBeDropped(const String & part_name); Pipe alterPartition( const StorageMetadataPtr & metadata_snapshot, @@ -985,8 +985,10 @@ protected: bool canReplacePartition(const DataPartPtr & src_part) const; /// Tries to drop part in background without any waits or throwing exceptions in case of errors. - virtual void dropPart(const String & name) = 0; - virtual void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr context) = 0; + virtual void dropPartNoWaitNoThrow(const String & part_name) = 0; + + virtual void dropPart(const String & part_name, bool detach, ContextPtr context) = 0; + virtual void dropPartition(const ASTPtr & partition, bool detach, ContextPtr context) = 0; virtual PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr context) = 0; virtual void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr context) = 0; virtual void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr context) = 0; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 1369c0151bd..754a9b7b115 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1240,41 +1240,32 @@ MergeTreeDataPartPtr StorageMergeTree::outdatePart(const String & part_name, boo } } -void StorageMergeTree::dropPart(const String & name) +void StorageMergeTree::dropPartNoWaitNoThrow(const String & part_name) { - auto part = outdatePart(name, false); - /// Nothing to do, part was removed in some different way - if (!part) - return; + if (auto part = outdatePart(part_name, false)) + dropPartsImpl({part}, false); - dropPartsImpl({part}, false); + /// Else nothing to do, part was removed in some different way } -void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr local_context) +void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPtr /*query_context*/) { - DataPartsVector parts_to_remove; + if (auto part = outdatePart(part_name, true)) + dropPartsImpl({part}, detach); - if (drop_part) - { - auto part = outdatePart(partition->as().value.safeGet(), true); - /// Nothing to do, part was removed in some different way - if (!part) - return; + /// Else nothing to do, part was removed in some different way +} - parts_to_remove.push_back(part); - } - else - { - /// Asks to complete merges and does not allow them to start. - /// This protects against "revival" of data for a removed partition after completion of merge. - auto merge_blocker = stopMergesAndWait(); - String partition_id = getPartitionIDFromQuery(partition, local_context); - parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); - - /// TODO should we throw an exception if parts_to_remove is empty? - removePartsFromWorkingSet(parts_to_remove, true); - } +void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, ContextPtr local_context) +{ + /// Asks to complete merges and does not allow them to start. + /// This protects against "revival" of data for a removed partition after completion of merge. + auto merge_blocker = stopMergesAndWait(); + String partition_id = getPartitionIDFromQuery(partition, local_context); + auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + /// TODO should we throw an exception if parts_to_remove is empty? + removePartsFromWorkingSet(parts_to_remove, true); dropPartsImpl(std::move(parts_to_remove), detach); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 1259e33e25f..3cb98589c2e 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -211,8 +211,9 @@ private: void clearOldMutations(bool truncate = false); // Partition helpers - void dropPart(const String & name) override; - void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr context) override; + void dropPartNoWaitNoThrow(const String & part_name) override; + void dropPart(const String & part_name, bool detach, ContextPtr context) override; + void dropPartition(const ASTPtr & partition, bool detach, ContextPtr context) override; void dropPartsImpl(DataPartsVector && parts_to_remove, bool detach); PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr context) override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index dbaaccc4bda..03415966798 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4927,7 +4927,7 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St return true; } -void StorageReplicatedMergeTree::dropPart(const String & name) +void StorageReplicatedMergeTree::dropPartNoWaitNoThrow(const String & part_name) { assertNotReadonly(); if (!is_leader) @@ -4936,48 +4936,52 @@ void StorageReplicatedMergeTree::dropPart(const String & name) zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); LogEntry entry; - dropPart(zookeeper, name, entry, false, false); + dropPartImpl(zookeeper, part_name, entry, false, false); } -void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr query_context) +void StorageReplicatedMergeTree::dropPart(const String & part_name, bool detach, ContextPtr query_context) { assertNotReadonly(); if (!is_leader) - throw Exception("DROP PART|PARTITION cannot be done on this replica because it is not a leader", ErrorCodes::NOT_A_LEADER); + throw Exception("DROP PART cannot be done on this replica because it is not a leader", ErrorCodes::NOT_A_LEADER); zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); - LogEntry entry; - bool did_drop; - if (drop_part) - { - String part_name = partition->as().value.safeGet(); - did_drop = dropPart(zookeeper, part_name, entry, detach, true); - } - else - { - String partition_id = getPartitionIDFromQuery(partition, query_context); - did_drop = dropAllPartsInPartition(*zookeeper, partition_id, entry, query_context, detach); - } + bool did_drop = dropPartImpl(zookeeper, part_name, entry, detach, true); if (did_drop) { /// If necessary, wait until the operation is performed on itself or on all replicas. - if (query_context->getSettingsRef().replication_alter_partitions_sync != 0) - { - if (query_context->getSettingsRef().replication_alter_partitions_sync == 1) - waitForReplicaToProcessLogEntry(replica_name, entry); - else - waitForAllReplicasToProcessLogEntry(entry); - } + if (query_context->getSettingsRef().replication_alter_partitions_sync == 1) + waitForReplicaToProcessLogEntry(replica_name, entry); + else if (query_context->getSettingsRef().replication_alter_partitions_sync == 2) + waitForAllReplicasToProcessLogEntry(entry); + } +} + +void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool detach, ContextPtr query_context) +{ + assertNotReadonly(); + if (!is_leader) + throw Exception("DROP PARTITION cannot be done on this replica because it is not a leader", ErrorCodes::NOT_A_LEADER); + + zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); + LogEntry entry; + + String partition_id = getPartitionIDFromQuery(partition, query_context); + bool did_drop = dropAllPartsInPartition(*zookeeper, partition_id, entry, query_context, detach); + + if (did_drop) + { + /// If necessary, wait until the operation is performed on itself or on all replicas. + if (query_context->getSettingsRef().replication_alter_partitions_sync == 1) + waitForReplicaToProcessLogEntry(replica_name, entry); + else if (query_context->getSettingsRef().replication_alter_partitions_sync == 2) + waitForAllReplicasToProcessLogEntry(entry); } - if (!drop_part) - { - String partition_id = getPartitionIDFromQuery(partition, query_context); - cleanLastPartNode(partition_id); - } + cleanLastPartNode(partition_id); } @@ -6712,7 +6716,7 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI return true; } -bool StorageReplicatedMergeTree::dropPart( +bool StorageReplicatedMergeTree::dropPartImpl( zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop) { LOG_TRACE(log, "Will try to insert a log entry to DROP_RANGE for part: " + part_name); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 9c749911964..9d0727ba0e7 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -630,13 +630,14 @@ private: /// Info about how other replicas can access this one. ReplicatedMergeTreeAddress getReplicatedMergeTreeAddress() const; - void dropPart(const String & name) override; - bool dropPart(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop); bool dropAllPartsInPartition( zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, ContextPtr query_context, bool detach); + void dropPartNoWaitNoThrow(const String & part_name) override; + void dropPart(const String & part_name, bool detach, ContextPtr query_context) override; + // Partition helpers - void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, ContextPtr query_context) override; + void dropPartition(const ASTPtr & partition, bool detach, ContextPtr query_context) override; PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr query_context) override; void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr query_context) override; void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) override; @@ -647,6 +648,8 @@ private: bool fetch_part, ContextPtr query_context) override; + bool dropPartImpl(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop); + /// Check granularity of already existing replicated table in zookeeper if it exists /// return true if it's fixed bool checkFixedGranualrityInZookeeper(); From 85cc7a89239656fb53aabf85614c896771f6a425 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 May 2021 17:44:10 +0300 Subject: [PATCH 059/154] Remove last presence of Poco::Path in src --- src/Formats/FormatSchemaInfo.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index ae1eb422d27..513d9738f62 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -1,9 +1,7 @@ #include -#include #include #include #include -#include namespace DB @@ -83,7 +81,7 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (Poco::Path(path.string()).depth() >= 1 && Poco::Path(path.string()).directory(0) == "..") + else if (path.string().starts_with("..")) { if (is_server) throw Exception( From 9c279c0c28dbd45fa8943d8e58851124a107aa2c Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Mon, 17 May 2021 22:11:53 +0200 Subject: [PATCH 060/154] consider the table Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/optimize.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 12d2173a9d8..62b9d8818ed 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -49,7 +49,7 @@ OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT (c **Examples** -1. Local example on a [`MergeTree`](../../engines/table-engines/mergetree-family/mergetree.md) table engine: +Consider the table: ``` sql DROP TABLE IF EXISTS dup_example; @@ -182,4 +182,4 @@ OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY COLUMNS OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY COLUMNS('[i]') EXCEPT(insert_time_ns, insert_replica_id); ``` -Don't forget to `DROP` tables and replicas `SYSTEM DROP REPLICA` afterwards. \ No newline at end of file +Don't forget to `DROP` tables and replicas `SYSTEM DROP REPLICA` afterwards. From 8e75e0ccde6cbb4cb6c4ef3285a7248b668fa1a2 Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Mon, 17 May 2021 22:12:15 +0200 Subject: [PATCH 061/154] remove DROP Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/optimize.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 62b9d8818ed..bf7da2bc867 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -52,7 +52,6 @@ OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT (c Consider the table: ``` sql -DROP TABLE IF EXISTS dup_example; CREATE TABLE dup_example ( pk Int32, -- primary key From 808f3c879165d914fec55737c972c4b3c9a79bc7 Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Mon, 17 May 2021 22:13:18 +0200 Subject: [PATCH 062/154] remove Valid cases Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/optimize.md | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index bf7da2bc867..78889f4b94e 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -64,12 +64,7 @@ CREATE TABLE dup_example ( ORDER BY (pk, sk); ``` -The `MergeTree` engine does not have parameters. - -**Valid cases** - -In the case below all columns are taken into account, i.e. row is removed only if all values in all columns are equal to corresponding values in another row. -Here and below we need to add `FINAL` to force deduplication in case of a small set of data. +Columns for deduplication are not specified, so all of them are taken into account. Row is removed only if all values in all columns are equal to corresponding values in previous row: ``` sql OPTIMIZE TABLE dup_example FINAL DEDUPLICATE; From 4b33108daca4326cec234de8f57136770e0c5318 Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Mon, 17 May 2021 22:13:41 +0200 Subject: [PATCH 063/154] remove ticks from Note Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/optimize.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 78889f4b94e..d46cfa1e6fa 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -31,7 +31,7 @@ When `OPTIMIZE` is used with the [ReplicatedMergeTree](../../engines/table-engin If you want to perform deduplication on custom set of columns rather than on all, you can specify list of columns explicitly or use any combination of [`*`](../../sql-reference/statements/select/index.md#asterisk), [`COLUMNS`](../../sql-reference/statements/select/index.md#columns-expression) or [`EXCEPT`](../../sql-reference/statements/select/index.md#except-modifier) expressions. The explictly written or implicitly expanded list of columns must include all columns specified in row ordering expression (both primary and sorting keys) and partitioning expression (partitioning key). !!! note "Note" - Notice that `*` behaves just like in `SELECT`: [`MATERIALIZED`](../../sql-reference/statements/create/table.md#materialized) and [`ALIAS`](../../sql-reference/statements/create/table.md#alias) columns are not used for expansion. + Notice that `*` behaves just like in `SELECT`: [MATERIALIZED](../../sql-reference/statements/create/table.md#materialized) and [ALIAS](../../sql-reference/statements/create/table.md#alias) columns are not used for expansion. Also, it is an error to specify empty list of columns, or write an expression that results in an empty list of columns, or deduplicate by an `ALIAS` column. **Syntax** From 7dde67a94e06f9a8287fe599925011d77b27cda6 Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Mon, 17 May 2021 22:50:02 +0200 Subject: [PATCH 064/154] Updated examples --- docs/en/sql-reference/statements/optimize.md | 235 ++++++++++--------- 1 file changed, 130 insertions(+), 105 deletions(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index d46cfa1e6fa..47fee3d78d8 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -52,128 +52,153 @@ OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT (c Consider the table: ``` sql - -CREATE TABLE dup_example ( - pk Int32, -- primary key - sk Int32, -- secondary key +CREATE TABLE example ( + primary_key Int32, + secondary_key Int32, value UInt32, - mat UInt32 MATERIALIZED rand(), -- materialized value - alias UInt32 ALIAS 2, -- aliased value - PRIMARY KEY pk -) ENGINE=MergeTree -ORDER BY (pk, sk); + partition_key UInt32, + materialized_value UInt32 MATERIALIZED 12345, + aliased_value UInt32 ALIAS 2, + PRIMARY KEY primary_key +) ENGINE=MergeTree  +PARTITION BY partition_key +ORDER BY (primary_key, secondary_key); +``` +``` sql +INSERT INTO example (primary_key, secondary_key, value, partition_key) +VALUES (0, 0, 0, 0), (0, 0, 0, 0), (1, 1, 2, 2), (1, 1, 2, 3), (1, 1, 3, 3); +``` +``` sql +SELECT * FROM example; +``` +Result: ``` -Columns for deduplication are not specified, so all of them are taken into account. Row is removed only if all values in all columns are equal to corresponding values in previous row: +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +│ 1 │ 1 │ 3 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ + +5 rows in set. +``` + +When columns for deduplication are not specified, all of them are taken into account. Row is removed only if all values in all columns are equal to corresponding values in previous row: ``` sql -OPTIMIZE TABLE dup_example FINAL DEDUPLICATE; +OPTIMIZE TABLE example FINAL DEDUPLICATE; +``` +``` sql +SELECT * FROM example; +``` +Result: +``` +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +│ 1 │ 1 │ 3 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ + +4 rows in set. ``` -Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED`: in our case deduplicate by `pk`, `sk` and `value` columns. +When columns are specified implicitly, the table is deduplicated by all columns that are not `ALIAS` or `MATERIALIZED`. Considering the table above, these are `primary_key`, `secondary_key`, `value`, and `partition_key` columns: +```sql +OPTIMIZE TABLE example FINAL DEDUPLICATE BY *; +``` +``` sql +SELECT * FROM example; +``` +Result: +``` +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +│ 1 │ 1 │ 3 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ + +4 rows in set. +``` + +Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED` and explicitly not `value`: `primary_key`, `secondary_key`, and `partition_key` columns. ``` sql -OPTIMIZE TABLE dup_example FINAL DEDUPLICATE BY *; +OPTIMIZE TABLE example FINAL DEDUPLICATE BY * EXCEPT value; ``` - -Deduplicate explicitly by `pk`, `sk`, `value` and `mat` columns. ``` sql -OPTIMIZE TABLE dup_example FINAL DEDUPLICATE BY pk, sk, value, mat; +SELECT * FROM example; +``` +Result: +``` +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ + +3 rows in set. ``` -Deduplicate by columns matching a regex `'.*k'`: `pk` and `sk` columns. - +Deduplicate explicitly by `primary_key`, `secondary_key`, and `partition_key` columns: +```sql +OPTIMIZE TABLE example FINAL DEDUPLICATE BY primary_key, secondary_key, partition_key; +``` ``` sql -OPTIMIZE TABLE dup_example FINAL DEDUPLICATE BY COLUMNS('.*k'); +SELECT * FROM example; +``` +Result: +``` +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ + +3 rows in set. ``` -**Error cases** - -Note that **primary key** column should not be missed in any `BY` expression. These queries will face errors: - +Deduplicate by any column matching a regex: `primary_key`, `secondary_key`, and `partition_key` columns: +```sql +OPTIMIZE TABLE example FINAL DEDUPLICATE BY COLUMNS('.*_key'); +``` ``` sql -OPTIMIZE TABLE dup_example DEDUPLICATE BY * EXCEPT(pk); -OPTIMIZE TABLE dup_example DEDUPLICATE BY sk, value; +SELECT * FROM example; ``` - -Empty list cases: -``` sql -OPTIMIZE TABLE dup_example DEDUPLICATE BY * EXCEPT(pk, sk, value, mat, alias); -- server error -OPTIMIZE TABLE dup_example DEDUPLICATE BY; -- syntax error +Result: ``` +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ -2. Replicated example on a [`ReplicatedMergeTree`](../../engines/table-engines/mergetree-family/replication.md) table engine: - -```sql -DROP TABLE IF EXISTS replicated_deduplicate_by_columns_r1; -DROP TABLE IF EXISTS replicated_deduplicate_by_columns_r2; - -SET replication_alter_partitions_sync = 2; - -CREATE TABLE IF NOT EXISTS replicated_deduplicate_by_columns_r1 ( - id Int32, - value UInt32, - insert_time_ns DateTime64(9) MATERIALIZED now64(9), - insert_replica_id UInt8 MATERIALIZED randConstant() -) ENGINE=ReplicatedMergeTree('zookeeper_name_configured_in_auxiliary_zookeepers:path', 'r1') -ORDER BY id; - -CREATE TABLE IF NOT EXISTS replicated_deduplicate_by_columns_r2 ( - id Int32, - value UInt32, - insert_time_ns DateTime64(9) MATERIALIZED now64(9), - insert_replica_id UInt8 MATERIALIZED randConstant() -) ENGINE=ReplicatedMergeTree('zookeeper_name_configured_in_auxiliary_zookeepers:path', 'r2') -ORDER BY id; -``` - -For the `ReplicatedMergeTree` engine we give the path to the table and name of the replica in Zookeeper. - -Insert some data into both replicas and wait for them to sync: - -```sql -SYSTEM SYNC REPLICA replicated_deduplicate_by_columns_r2; -SYSTEM SYNC REPLICA replicated_deduplicate_by_columns_r1; -``` - -Check that we have data on replicas: - -```sql -SELECT 'r1', id, value, count(), uniqExact(insert_time_ns), uniqExact(insert_replica_id) -FROM replicated_deduplicate_by_columns_r1 -GROUP BY id, value -ORDER BY id, value; - -SELECT 'r2', id, value, count(), uniqExact(insert_time_ns), uniqExact(insert_replica_id) -FROM replicated_deduplicate_by_columns_r2 -GROUP BY id, value -ORDER BY id, value; -``` - -Remove full duplicates from replica `r1` based on all columns: - -```sql -OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE; -``` - -Remove duplicates from replica `r1` based on all columns that are not `ALIAS` or `MATERIALIZED`: - -```sql -OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY *; -- except insert_time_ns, insert_replica_id -``` - -Deduplicate replica `r1` explicitly by `id` and `value` columns: - -```sql -OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY id, value; -``` - -Deduplicate by columns matching a regex: - -```sql -OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY COLUMNS('[id, value]'); - -OPTIMIZE TABLE replicated_deduplicate_by_columns_r1 FINAL DEDUPLICATE BY COLUMNS('[i]') EXCEPT(insert_time_ns, insert_replica_id); -``` - -Don't forget to `DROP` tables and replicas `SYSTEM DROP REPLICA` afterwards. +3 rows in set. +``` \ No newline at end of file From 5f083b779dde1072dc14e4729e04db45bda06c59 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 18 May 2021 13:11:12 +0300 Subject: [PATCH 065/154] A little better, fix checks --- programs/client/Client.cpp | 2 +- src/Common/createFile.cpp | 6 ++---- src/Common/createFile.h | 6 +++--- src/Databases/DatabaseOnDisk.cpp | 2 +- src/Databases/MySQL/DatabaseConnectionMySQL.cpp | 2 +- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Disks/DiskLocal.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Server/StaticRequestHandler.cpp | 7 ++++--- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/examples/remove_symlink_directory.cpp | 2 +- 11 files changed, 17 insertions(+), 18 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 407f70b36b6..58e3efe736d 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -637,7 +637,7 @@ private: } if (!history_file.empty() && !fs::exists(history_file)) - fs::createFile(history_file); + FS::createFile(history_file); LineReader::Patterns query_extenders = {"\\"}; LineReader::Patterns query_delimiters = {";", "\\G"}; diff --git a/src/Common/createFile.cpp b/src/Common/createFile.cpp index fc637f4d4f2..83ab9c36f4b 100644 --- a/src/Common/createFile.cpp +++ b/src/Common/createFile.cpp @@ -17,7 +17,7 @@ extern const int CANNOT_CREATE_FILE; } } -namespace std::filesystem +namespace FS { [[noreturn]] void handleLastError(const std::string & path) { @@ -37,7 +37,7 @@ namespace std::filesystem } /// Copy from Poco::createFile -bool createFile(const path & path) +bool createFile(const std::string & path) { int n = open(path.c_str(), O_WRONLY | O_CREAT | O_EXCL, S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP | S_IROTH | S_IWOTH); if (n != -1) @@ -45,8 +45,6 @@ bool createFile(const path & path) close(n); return true; } - if (n == -1 && errno == EEXIST) - return false; handleLastError(path); } } diff --git a/src/Common/createFile.h b/src/Common/createFile.h index 40ec74e6288..67c7023a455 100644 --- a/src/Common/createFile.h +++ b/src/Common/createFile.h @@ -1,7 +1,7 @@ #pragma once -#include +#include -namespace std::filesystem +namespace FS { -bool createFile(const path & path); +bool createFile(const std::string & path); } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index a47a38cfa7c..8393f9e81c0 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -326,7 +326,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr, const String & table_nam fs::path detached_permanently_flag(getObjectMetadataPath(table_name) + detached_suffix); try { - fs::createFile(detached_permanently_flag); + FS::createFile(detached_permanently_flag); } catch (Exception & e) { diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index cd26aa9d02b..7047ed4ad53 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -436,7 +436,7 @@ void DatabaseConnectionMySQL::detachTablePermanently(ContextPtr, const String & try { table_iter->second.second->drop(); - fs::createFile(remove_flag); + FS::createFile(remove_flag); } catch (...) { diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 33ae26a9c40..5d75915ea91 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -258,7 +258,7 @@ void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool / throw Exception(fmt::format("Table {}.{} is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED); fs::path mark_table_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); - fs::createFile(mark_table_removed); + FS::createFile(mark_table_removed); if (cache_tables) cached_tables.erase(table_name); diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index bf0a8bac12f..b4880daaadd 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -294,7 +294,7 @@ void DiskLocal::truncateFile(const String & path, size_t size) void DiskLocal::createFile(const String & path) { - fs::createFile(fs::path(disk_path) / path); + FS::createFile(fs::path(disk_path) / path); } void DiskLocal::setReadOnly(const String & path) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 86b810d031e..8c872dcdec5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -112,7 +112,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) /// Will write file with database metadata, if needed. String database_name_escaped = escapeForFileName(database_name); - fs::path metadata_path = fs::canonical(getContext()->getPath()); + fs::path metadata_path = fs::absolute(getContext()->getPath()); fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp"); fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql"); diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index 978271c1f04..4f730b7cc30 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -141,12 +141,13 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) const auto & file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size()); fs::path user_files_absolute_path = fs::absolute(fs::path(server.context()->getUserFilesPath())); - fs::path file_path = user_files_absolute_path / file_name; + /// Fixme: it does not work with fs::path(user_files_absolute_path) / file_name + String file_path = fs::absolute(user_files_absolute_path.string() + "/" + file_name); if (!fs::exists(file_path)) - throw Exception("Invalid file name " + file_path.string() + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); + throw Exception("Invalid file name " + file_path + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); - ReadBufferFromFile in(file_path.string()); + ReadBufferFromFile in(file_path); copyData(in, out); } else if (startsWith(response_expression, config_prefix)) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 57c6d3be9ec..48f8793a506 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4388,7 +4388,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( const String & with_name, ContextPtr local_context) { - String clickhouse_path = fs::canonical(local_context->getPath()); + String clickhouse_path = fs::absolute(local_context->getPath()); String default_shadow_path = fs::path(clickhouse_path) / "shadow/"; fs::create_directories(default_shadow_path); auto increment = Increment(fs::path(default_shadow_path) / "increment.txt").get(true); diff --git a/src/Storages/examples/remove_symlink_directory.cpp b/src/Storages/examples/remove_symlink_directory.cpp index 05fdc18be2f..a90083233e7 100644 --- a/src/Storages/examples/remove_symlink_directory.cpp +++ b/src/Storages/examples/remove_symlink_directory.cpp @@ -19,7 +19,7 @@ try { fs::path dir("./test_dir/"); fs::create_directories(dir); - fs::createFile("./test_dir/file"); + FS::createFile("./test_dir/file"); if (0 != symlink("./test_dir", "./test_link")) DB::throwFromErrnoWithPath("Cannot create symlink", "./test_link", DB::ErrorCodes::SYSTEM_ERROR); From 31d1b92fdc6a18d57c7e53575014619e2e913039 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 19 May 2021 16:19:49 +0300 Subject: [PATCH 066/154] Create dictionary.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал табличную функцию dictionary. --- .../table-functions/dictionary.md | 60 +++++++++++++++++++ 1 file changed, 60 insertions(+) create mode 100644 docs/en/sql-reference/table-functions/dictionary.md diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md new file mode 100644 index 00000000000..e9d78a6fab5 --- /dev/null +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -0,0 +1,60 @@ +--- +toc_priority: 54 +toc_title: dictionary function +--- + +# dictionary {#dictionary-function} + +Displays the [dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. + +**Syntax** + +``` sql +dictionary('dict') +``` + +**Arguments** + +- `dict` — A dictionary name. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +A ClickHouse table. + +**Example** + +Input table: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +Create a dictionary: + +``` sql +CREATE DICTIONARY table_function_dictionary_test_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_function_dictionary_source_table')) +LAYOUT(DIRECT()); +``` + +Query: + +``` sql +SELECT * FROM dictionary('table_function_dictionary_test_dictionary'); +``` + +Result: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +**See Also** + +- [Dictionary engine](../../engines/table-engines/special/dictionary.md#dictionary) From b5421209e3707067554e84f298d21d6030b51dba Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 19 May 2021 18:20:52 +0300 Subject: [PATCH 067/154] Fix link MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил ссылку. --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index e9d78a6fab5..608dba01bea 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -5,7 +5,7 @@ toc_title: dictionary function # dictionary {#dictionary-function} -Displays the [dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. +Displays the [dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. **Syntax** From 096d117f6801385fe4dc38f42e343206916dae28 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 22 May 2021 00:12:46 +0300 Subject: [PATCH 068/154] Review fixes --- src/Common/Config/ConfigProcessor.cpp | 11 +++++------ src/Common/ErrorCodes.cpp | 1 - src/Common/createFile.cpp | 4 ++-- src/Databases/DatabaseOnDisk.cpp | 2 -- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/loadMetadata.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 7 files changed, 12 insertions(+), 16 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 396334c6a8b..598f64e0d99 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -416,16 +416,15 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string & std::set merge_dirs; /// Add path_to_config/config_name.d dir - merge_dir_path = merge_dir_path.parent_path() / (merge_dir_path.stem().string() + ".d"); + merge_dir_path.replace_extension("d"); merge_dirs.insert(merge_dir_path); /// Add path_to_config/conf.d dir - merge_dir_path = merge_dir_path.parent_path() / "conf.d"; + merge_dir_path.replace_filename("conf.d"); merge_dirs.insert(merge_dir_path); for (const std::string & merge_dir_name : merge_dirs) { - fs::path merge_dir(merge_dir_name); - if (!fs::exists(merge_dir) || !is_directory(merge_dir)) + if (!fs::exists(merge_dir_name) || !fs::is_directory(merge_dir_name)) continue; for (fs::directory_iterator it(merge_dir_name); it != fs::directory_iterator(); ++it) @@ -635,7 +634,7 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, fs::path parent_path = fs::path(loaded_config.config_path).parent_path(); preprocessed_dir = parent_path.string(); fs::path fs_new_path(new_path); - fs_new_path = fs_new_path.parent_path() / (fs_new_path.stem().string() + PREPROCESSED_SUFFIX + fs_new_path.extension().string()); + fs_new_path.replace_filename(fs_new_path.stem().string() + PREPROCESSED_SUFFIX + fs_new_path.extension().string()); new_path = fs_new_path.string(); } else @@ -652,7 +651,7 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, preprocessed_path = (fs::path(preprocessed_dir) / fs::path(new_path)).string(); auto preprocessed_path_parent = fs::path(preprocessed_path).parent_path(); - if (!preprocessed_path_parent.string().empty()) + if (!preprocessed_path_parent.empty()) fs::create_directories(preprocessed_path_parent); } DOMWriter().writeNode(preprocessed_path, loaded_config.preprocessed_xml); diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index d3cd812ef64..dccc20d325b 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -553,7 +553,6 @@ M(583, ILLEGAL_PROJECTION) \ M(584, PROJECTION_NOT_USED) \ \ - M(996, OPERATION_NOT_PERMITTED) \ M(997, CANNOT_CREATE_FILE) \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Common/createFile.cpp b/src/Common/createFile.cpp index 83ab9c36f4b..74f8f9cf980 100644 --- a/src/Common/createFile.cpp +++ b/src/Common/createFile.cpp @@ -11,7 +11,7 @@ namespace DB namespace ErrorCodes { extern const int FILE_ALREADY_EXISTS; -extern const int OPERATION_NOT_PERMITTED; +extern const int PATH_ACCESS_DENIED; extern const int NOT_ENOUGH_SPACE; extern const int CANNOT_CREATE_FILE; } @@ -26,7 +26,7 @@ namespace FS case EEXIST: throw DB::Exception(DB::ErrorCodes::FILE_ALREADY_EXISTS, "File {} already exist", path); case EPERM: - throw DB::Exception(DB::ErrorCodes::OPERATION_NOT_PERMITTED, "Not enough permissions to create file {}", path); + throw DB::Exception(DB::ErrorCodes::PATH_ACCESS_DENIED, "Not enough permissions to create file {}", path); case ENOSPC: throw DB::Exception(DB::ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space to create file {}", path); case ENAMETOOLONG: diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 8393f9e81c0..f50adf54c3f 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -19,8 +19,6 @@ #include #include #include -#include -#include #include namespace fs = std::filesystem; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index ca473b4aac6..0794cb52a49 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -356,7 +356,7 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d /// Old ClickHouse versions did not store database.sql files fs::path database_metadata_file = fs::path(getContext()->getPath()) / "metadata" / (escapeForFileName(database_name) + ".sql"); if (fs::exists(database_metadata_file)) - fs::remove_all(database_metadata_file); + fs::remove(database_metadata_file); } return db; diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index c109c53af20..0a84cdca309 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -112,15 +112,15 @@ void loadMetadata(ContextPtr context, const String & default_database_name) if (!it->is_directory()) { /// TODO: DETACH DATABASE PERMANENTLY ? - if (endsWith(current_file, ".sql")) + if (fs::path(current_file).extension() == ".sql") { String db_name = current_file.substr(0, current_file.size() - 4); if (db_name != DatabaseCatalog::SYSTEM_DATABASE) - databases.emplace(unescapeForFileName(db_name), path + "/" + db_name); + databases.emplace(unescapeForFileName(db_name), fs::path(path) / db_name); } /// Temporary fails may be left from previous server runs. - if (endsWith(current_file, ".tmp")) + if (fs::path(current_file).extension() == ".tmp") { LOG_WARNING(log, "Removing temporary file {}", it->path().string()); try diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2a34ae3fd07..88c7edc948b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6532,7 +6532,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta ops.emplace_back(zkutil::makeCheckRequest(alter_partition_version_path, alter_partition_version_stat.version)); ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", -1)); /// Just update version, because merges assignment relies on it - ops.emplace_back(zkutil::makeSetRequest(fs:path(dest_table_storage->zookeeper_path) / "log", "", -1)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(dest_table_storage->zookeeper_path) / "log", "", -1)); ops.emplace_back(zkutil::makeCreateRequest(fs::path(dest_table_storage->zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); From 8bac10d24f3c76563a478f2c8b3492975dd65034 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 22 May 2021 21:24:13 +0300 Subject: [PATCH 069/154] Update base/* --- base/daemon/BaseDaemon.cpp | 36 +++++++++++++++++------------------- base/daemon/SentryWriter.cpp | 8 ++++---- base/loggers/Loggers.cpp | 17 +++++++++-------- 3 files changed, 30 insertions(+), 31 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 01e700ebba3..742d0958668 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -26,8 +26,6 @@ #include #include #include -#include -#include #include #include #include @@ -59,6 +57,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include @@ -70,6 +69,7 @@ #endif #include +namespace fs = std::filesystem; DB::PipeFDs signal_pipe; @@ -437,11 +437,11 @@ static void sanitizerDeathCallback() static std::string createDirectory(const std::string & file) { - auto path = Poco::Path(file).makeParent(); - if (path.toString().empty()) + fs::path path = fs::path(file).parent_path(); + if (path.empty()) return ""; - Poco::File(path).createDirectories(); - return path.toString(); + fs::create_directories(path); + return path; }; @@ -449,7 +449,7 @@ static bool tryCreateDirectories(Poco::Logger * logger, const std::string & path { try { - Poco::File(path).createDirectories(); + fs::create_directories(path); return true; } catch (...) @@ -470,7 +470,7 @@ void BaseDaemon::reloadConfiguration() */ config_path = config().getString("config-file", getDefaultConfigFileName()); DB::ConfigProcessor config_processor(config_path, false, true); - config_processor.setConfigPath(Poco::Path(config_path).makeParent().toString()); + config_processor.setConfigPath(fs::path(config_path).parent_path()); loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true); if (last_configuration != nullptr) @@ -524,18 +524,17 @@ std::string BaseDaemon::getDefaultConfigFileName() const void BaseDaemon::closeFDs() { #if defined(OS_FREEBSD) || defined(OS_DARWIN) - Poco::File proc_path{"/dev/fd"}; + fs::path proc_path{"/dev/fd"}; #else - Poco::File proc_path{"/proc/self/fd"}; + fs::path proc_path{"/proc/self/fd"}; #endif - if (proc_path.isDirectory()) /// Hooray, proc exists + if (fs::is_directory(proc_path)) /// Hooray, proc exists { - std::vector fds; /// in /proc/self/fd directory filenames are numeric file descriptors - proc_path.list(fds); - for (const auto & fd_str : fds) + fs::directory_iterator dir_end; + for (fs::directory_iterator dir_it(proc_path); dir_it != dir_end; ++dir_it) { - int fd = DB::parse(fd_str); + int fd = DB::parse(dir_it->path().filename()); if (fd > 2 && fd != signal_pipe.fds_rw[0] && fd != signal_pipe.fds_rw[1]) ::close(fd); } @@ -597,7 +596,7 @@ void BaseDaemon::initialize(Application & self) { /** When creating pid file and looking for config, will search for paths relative to the working path of the program when started. */ - std::string path = Poco::Path(config().getString("application.path")).setFileName("").toString(); + std::string path = fs::path(config().getString("application.path")).replace_filename(""); if (0 != chdir(path.c_str())) throw Poco::Exception("Cannot change directory to " + path); } @@ -645,7 +644,7 @@ void BaseDaemon::initialize(Application & self) std::string log_path = config().getString("logger.log", ""); if (!log_path.empty()) - log_path = Poco::Path(log_path).setFileName("").toString(); + log_path = fs::path(log_path).replace_filename(""); /** Redirect stdout, stderr to separate files in the log directory (or in the specified file). * Some libraries write to stderr in case of errors in debug mode, @@ -708,8 +707,7 @@ void BaseDaemon::initialize(Application & self) tryCreateDirectories(&logger(), core_path); - Poco::File cores = core_path; - if (!(cores.exists() && cores.isDirectory())) + if (!(fs::exists(core_path) && fs::is_directory(core_path))) { core_path = !log_path.empty() ? log_path : "/opt/"; tryCreateDirectories(&logger(), core_path); diff --git a/base/daemon/SentryWriter.cpp b/base/daemon/SentryWriter.cpp index 1028dc7d2dc..3719bb74c52 100644 --- a/base/daemon/SentryWriter.cpp +++ b/base/daemon/SentryWriter.cpp @@ -25,6 +25,7 @@ # include # include +namespace fs = std::filesystem; namespace { @@ -53,8 +54,7 @@ void setExtras() sentry_set_extra("physical_cpu_cores", sentry_value_new_int32(getNumberOfPhysicalCPUCores())); if (!server_data_path.empty()) - sentry_set_extra("disk_free_space", sentry_value_new_string(formatReadableSizeWithBinarySuffix( - Poco::File(server_data_path).freeSpace()).c_str())); + sentry_set_extra("disk_free_space", sentry_value_new_string(formatReadableSizeWithBinarySuffix(fs::space(server_data_path).free).c_str())); } void sentry_logger(sentry_level_e level, const char * message, va_list args, void *) @@ -110,12 +110,12 @@ void SentryWriter::initialize(Poco::Util::LayeredConfiguration & config) if (enabled) { server_data_path = config.getString("path", ""); - const std::filesystem::path & default_tmp_path = std::filesystem::path(config.getString("tmp_path", Poco::Path::temp())) / "sentry"; + const std::filesystem::path & default_tmp_path = fs::path(config.getString("tmp_path", fs::temp_directory_path())) / "sentry"; const std::string & endpoint = config.getString("send_crash_reports.endpoint"); const std::string & temp_folder_path = config.getString("send_crash_reports.tmp_path", default_tmp_path); - Poco::File(temp_folder_path).createDirectories(); + fs::create_directories(temp_folder_path); sentry_options_t * options = sentry_options_new(); /// will be freed by sentry_init or sentry_shutdown sentry_options_set_release(options, VERSION_STRING_SHORT); diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index 913deaf1eb8..11dfbdaeb71 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -6,10 +6,11 @@ #include "OwnFormattingChannel.h" #include "OwnPatternFormatter.h" #include -#include #include #include -#include +#include + +namespace fs = std::filesystem; namespace DB { @@ -20,11 +21,11 @@ namespace DB // TODO: move to libcommon static std::string createDirectory(const std::string & file) { - auto path = Poco::Path(file).makeParent(); - if (path.toString().empty()) + auto path = fs::path(file).parent_path(); + if (path.empty()) return ""; - Poco::File(path).createDirectories(); - return path.toString(); + fs::create_directories(path); + return path; }; void Loggers::setTextLog(std::shared_ptr log, int max_priority) @@ -70,7 +71,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log // Set up two channel chains. log_file = new Poco::FileChannel; - log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(log_path).absolute().toString()); + log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::absolute(log_path)); log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true")); @@ -102,7 +103,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log std::cerr << "Logging errors to " << errorlog_path << std::endl; error_log_file = new Poco::FileChannel; - error_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(errorlog_path).absolute().toString()); + error_log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::absolute(errorlog_path)); error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); error_log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true")); From 866b29fb5a0402e646bbd7e26a0a9780e6d2849e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 23 May 2021 10:56:13 +0300 Subject: [PATCH 070/154] Return list fds with Poco, more canonical --- base/daemon/BaseDaemon.cpp | 8 +++++--- src/Databases/DatabaseAtomic.cpp | 6 +++--- .../Embedded/GeodataProviders/HierarchiesProvider.cpp | 2 +- src/Formats/FormatSchemaInfo.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 5 files changed, 11 insertions(+), 9 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 742d0958668..a2118cc7bb1 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -57,6 +57,7 @@ #include #include #include +#include #include #if !defined(ARCADIA_BUILD) @@ -530,11 +531,12 @@ void BaseDaemon::closeFDs() #endif if (fs::is_directory(proc_path)) /// Hooray, proc exists { + std::vector fds; /// in /proc/self/fd directory filenames are numeric file descriptors - fs::directory_iterator dir_end; - for (fs::directory_iterator dir_it(proc_path); dir_it != dir_end; ++dir_it) + Poco::File(proc_path.string()).list(fds); + for (const auto & fd_str : fds) { - int fd = DB::parse(dir_it->path().filename()); + int fd = DB::parse(fd_str); if (fd > 2 && fd != signal_pipe.fds_rw[0] && fd != signal_pipe.fds_rw[1]) ::close(fd); } diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index b69af842978..e4b7d6e3038 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -443,9 +443,9 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & try { String link = path_to_table_symlinks + escapeForFileName(table_name); - fs::path data = fs::absolute(getContext()->getPath()).string() + actual_data_path; + fs::path data = fs::canonical(getContext()->getPath()) / actual_data_path; if (!if_data_path_exist || fs::exists(data)) - fs::create_symlink(data, link); + fs::create_directory_symlink(data, link); } catch (...) { @@ -481,7 +481,7 @@ void DatabaseAtomic::tryCreateMetadataSymlink() { try { - fs::create_symlink(metadata_path, path_to_metadata_symlink); + fs::create_directory_symlink(metadata_path, path_to_metadata_symlink); } catch (...) { diff --git a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp index 3f35b9102a2..210459da0be 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp @@ -30,7 +30,7 @@ RegionsHierarchiesDataProvider::RegionsHierarchiesDataProvider(const std::string void RegionsHierarchiesDataProvider::discoverFilesWithCustomHierarchies() { std::string basename = fs::path(path).stem(); - fs::path dir_path = fs::absolute(path).parent_path(); + fs::path dir_path = fs::canonical(path).parent_path(); fs::directory_iterator dir_end; for (fs::directory_iterator dir_it(dir_path); dir_it != dir_end; ++dir_it) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 513d9738f62..f985cb2681f 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -67,7 +67,7 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & auto default_schema_directory = [&format_schema_path]() { - static const String str = fs::absolute(format_schema_path) / ""; + static const String str = fs::canonical(format_schema_path) / ""; return str; }; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 8c872dcdec5..86b810d031e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -112,7 +112,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) /// Will write file with database metadata, if needed. String database_name_escaped = escapeForFileName(database_name); - fs::path metadata_path = fs::absolute(getContext()->getPath()); + fs::path metadata_path = fs::canonical(getContext()->getPath()); fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp"); fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql"); From 0d2c2399389fca52955ac52dd7ef452d6384cb56 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Mon, 24 May 2021 00:07:52 +0400 Subject: [PATCH 071/154] Allow recursive execution paths --- src/Access/MemoryAccessStorage.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/MemoryAccessStorage.h b/src/Access/MemoryAccessStorage.h index 92439342168..512ccff1d1b 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -51,7 +51,7 @@ private: void setAllNoLock(const std::vector> & all_entities, Notifications & notifications); void prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const; - mutable std::mutex mutex; + mutable std::recursive_mutex mutex; std::unordered_map entries_by_id; /// We want to search entries both by ID and by the pair of name and type. std::unordered_map entries_by_name_and_type[static_cast(EntityType::MAX)]; mutable std::list handlers_by_type[static_cast(EntityType::MAX)]; From e46a497f8a22509a856b7997b1b3faa4aef4e341 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Mon, 24 May 2021 00:13:05 +0400 Subject: [PATCH 072/154] Fix typo --- docs/en/development/build-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build-osx.md b/docs/en/development/build-osx.md index f34107ca3d3..a862bdeb299 100644 --- a/docs/en/development/build-osx.md +++ b/docs/en/development/build-osx.md @@ -20,7 +20,7 @@ Install the latest [Xcode](https://apps.apple.com/am/app/xcode/id497799835?mt=12 Open it at least once to accept the end-user license agreement and automatically install the required components. -Then, make sure that the latest Comman Line Tools are installed and selected in the system: +Then, make sure that the latest Command Line Tools are installed and selected in the system: ``` bash sudo rm -rf /Library/Developer/CommandLineTools From a4d1b9b07d7fb6c8b1340e5ef63b0531d963dc5a Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 24 May 2021 16:55:05 +0300 Subject: [PATCH 073/154] Fix --- src/Storages/StorageBuffer.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 6688112c9f1..e5af154d7bd 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -841,7 +841,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc size_t block_rows = block_to_write.rows(); size_t block_bytes = block_to_write.bytes(); - size_t block_allocated_bytes = block_to_write.allocatedBytes(); + size_t block_allocated_bytes_delta = block_to_write.allocatedBytes() - buffer.data.allocatedBytes(); CurrentMetrics::sub(CurrentMetrics::StorageBufferRows, block_rows); CurrentMetrics::sub(CurrentMetrics::StorageBufferBytes, block_bytes); @@ -851,7 +851,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc if (!destination_id) { total_writes.rows -= block_rows; - total_writes.bytes -= block_allocated_bytes; + total_writes.bytes -= block_allocated_bytes_delta; LOG_DEBUG(log, "Flushing buffer with {} rows (discarded), {} bytes, age {} seconds {}.", rows, bytes, time_passed, (check_thresholds ? "(bg)" : "(direct)")); return; @@ -890,7 +890,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc } total_writes.rows -= block_rows; - total_writes.bytes -= block_allocated_bytes; + total_writes.bytes -= block_allocated_bytes_delta; UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Flushing buffer with {} rows, {} bytes, age {} seconds, took {} ms {}.", rows, bytes, time_passed, milliseconds, (check_thresholds ? "(bg)" : "(direct)")); From b37f9c20e2feb82cef4754b81210f68cebef05ea Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Mon, 24 May 2021 20:07:43 +0300 Subject: [PATCH 074/154] Update dictionary.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил пример. --- docs/en/sql-reference/table-functions/dictionary.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index 608dba01bea..3e3d2a10984 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -36,8 +36,7 @@ Create a dictionary: ``` sql CREATE DICTIONARY table_function_dictionary_test_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_function_dictionary_source_table')) -LAYOUT(DIRECT()); +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_function_dictionary_source_table')) LAYOUT(DIRECT()); ``` Query: From 31107816bb42c0844df4959f01203b60ce4b517b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 24 May 2021 19:03:09 +0300 Subject: [PATCH 075/154] fs::absolute to fs::canonical --- base/loggers/Loggers.cpp | 4 ++-- programs/copier/ClusterCopierApp.cpp | 2 +- programs/install/Install.cpp | 2 +- src/Disks/DiskLocal.cpp | 13 +++++-------- src/Server/StaticRequestHandler.cpp | 4 ++-- src/Storages/MergeTree/DataPartsExchange.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageFile.cpp | 4 ++-- 8 files changed, 16 insertions(+), 19 deletions(-) diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index 11dfbdaeb71..80e62d0a6d6 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -71,7 +71,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log // Set up two channel chains. log_file = new Poco::FileChannel; - log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::absolute(log_path)); + log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::weakly_canonical(log_path)); log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true")); @@ -103,7 +103,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log std::cerr << "Logging errors to " << errorlog_path << std::endl; error_log_file = new Poco::FileChannel; - error_log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::absolute(errorlog_path)); + error_log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::weakly_canonical(errorlog_path)); error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); error_log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true")); diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 79bfb5b7411..8925ab63f99 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -40,7 +40,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id; - process_path = fs::absolute(fs::path(base_dir) / ("clickhouse-copier_" + process_id)); + process_path = fs::weakly_canonical(fs::path(base_dir) / ("clickhouse-copier_" + process_id)); fs::create_directories(process_path); /// Override variables for BaseDaemon diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 96d336673d0..1f2c1c30ae4 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -288,7 +288,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) bool is_symlink = fs::is_symlink(symlink_path); fs::path points_to; if (is_symlink) - points_to = fs::absolute(fs::read_symlink(symlink_path)); + points_to = fs::weakly_canonical(fs::read_symlink(symlink_path)); if (is_symlink && points_to == main_bin_path) { diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index b4880daaadd..51ed7fe577b 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -61,8 +61,8 @@ private: class DiskLocalDirectoryIterator : public IDiskDirectoryIterator { public: - explicit DiskLocalDirectoryIterator(const fs::path & disk_path_, const String & dir_path_) - : dir_path(dir_path_), entry(disk_path_ / dir_path_) + explicit DiskLocalDirectoryIterator(const String & disk_path_, const String & dir_path_) + : dir_path(dir_path_), entry(fs::path(disk_path_) / dir_path_) { } @@ -196,7 +196,7 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { - return std::make_unique(fs::path(disk_path), path); + return std::make_unique(disk_path, path); } void DiskLocal::moveFile(const String & from_path, const String & to_path) @@ -314,12 +314,9 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr & to if (isSameDiskType(*this, *to_disk)) { fs::path to = fs::path(to_disk->getPath()) / to_path; - fs::path from; + fs::path from = fs::path(disk_path) / from_path; if (from_path.ends_with('/')) - from = fs::path(disk_path) / from_path.substr(0, from_path.size() - 1); - else - from = fs::path(disk_path) / from_path; - + from = from.parent_path(); if (fs::is_directory(from)) to /= from.filename(); diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index 4f730b7cc30..4746ab6d1e0 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -140,9 +140,9 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) { const auto & file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size()); - fs::path user_files_absolute_path = fs::absolute(fs::path(server.context()->getUserFilesPath())); + fs::path user_files_absolute_path = fs::canonical(fs::path(server.context()->getUserFilesPath())); /// Fixme: it does not work with fs::path(user_files_absolute_path) / file_name - String file_path = fs::absolute(user_files_absolute_path.string() + "/" + file_name); + String file_path = fs::canonical(user_files_absolute_path.string() + "/" + file_name); if (!fs::exists(file_path)) throw Exception("Invalid file name " + file_path + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 69a92738777..47ed4a0762c 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -652,8 +652,8 @@ void Fetcher::downloadBaseOrProjectionPartToDisk( /// File must be inside "absolute_part_path" directory. /// Otherwise malicious ClickHouse replica may force us to write to arbitrary path. - String absolute_file_path = fs::absolute(fs::path(part_download_path) / file_name); - if (!startsWith(absolute_file_path, fs::absolute(part_download_path).string())) + String absolute_file_path = fs::weakly_canonical(fs::path(part_download_path) / file_name); + if (!startsWith(absolute_file_path, fs::weakly_canonical(part_download_path).string())) throw Exception("File path (" + absolute_file_path + ") doesn't appear to be inside part path (" + part_download_path + ")." " This may happen if we are trying to download part from malicious replica or logical error.", ErrorCodes::INSECURE_PATH); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7cac7b9ffd7..d67d67768ef 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4406,7 +4406,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( const String & with_name, ContextPtr local_context) { - String clickhouse_path = fs::absolute(local_context->getPath()); + String clickhouse_path = fs::canonical(local_context->getPath()); String default_shadow_path = fs::path(clickhouse_path) / "shadow/"; fs::create_directories(default_shadow_path); auto increment = Increment(fs::path(default_shadow_path) / "increment.txt").get(true); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 2dd3bb757f3..aac1b708567 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -129,13 +129,13 @@ void checkCreationIsAllowed(ContextPtr context_global, const std::string & db_di Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context) { - fs::path user_files_absolute_path = fs::absolute(user_files_path); + fs::path user_files_absolute_path = fs::weakly_canonical(user_files_path); fs::path fs_table_path(table_path); if (fs_table_path.is_relative()) fs_table_path = user_files_absolute_path / fs_table_path; Strings paths; - const String path = fs::absolute(fs_table_path); + const String path = fs::weakly_canonical(fs_table_path); if (path.find_first_of("*?{") == std::string::npos) paths.push_back(path); else From 34eaa48294ee9e92e405b3136235b7444eef74ca Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 25 May 2021 19:34:43 +0300 Subject: [PATCH 076/154] Part 1. --- .../QueryPlan/ReadFromMergeTree.cpp | 192 +++++++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 25 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 280 +++++++++++++++++- .../MergeTree/MergeTreeDataSelectExecutor.h | 27 +- 4 files changed, 498 insertions(+), 26 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index d7a78e6cc6b..771f95cc8fd 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -2,38 +2,54 @@ #include #include #include +#include #include #include #include +#include +#include #include #include namespace DB { +namespace ErrorCodes +{ + extern const int INDEX_NOT_USED; +} + ReadFromMergeTree::ReadFromMergeTree( + SelectQueryInfo query_info_, + const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, + ContextPtr context_, + const MergeTreeData & data_, const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, String query_id_, - Names required_columns_, - RangesInDataParts parts_, - IndexStatPtr index_stats_, + Names real_column_names_, + MergeTreeData::DataPartsVector parts_, + //IndexStatPtr index_stats_, PrewhereInfoPtr prewhere_info_, Names virt_column_names_, Settings settings_, size_t num_streams_, ReadType read_type_) : ISourceStep(DataStream{.header = MergeTreeBaseSelectProcessor::transformHeader( - metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals(), storage_.getStorageID()), + metadata_snapshot_->getSampleBlockForColumns(real_column_names_, storage_.getVirtuals(), storage_.getStorageID()), prewhere_info_, storage_.getPartitionValueType(), virt_column_names_)}) + , query_info(std::move(query_info_)) + , max_block_numbers_to_read(max_block_numbers_to_read_) + , context(std::move(context_)) + , data(data_) , storage(storage_) , metadata_snapshot(std::move(metadata_snapshot_)) , query_id(std::move(query_id_)) - , required_columns(std::move(required_columns_)) + , real_column_names(std::move(real_column_names_)) , parts(std::move(parts_)) - , index_stats(std::move(index_stats_)) + //, index_stats(std::move(index_stats_)) , prewhere_info(std::move(prewhere_info_)) , virt_column_names(std::move(virt_column_names_)) , settings(std::move(settings_)) @@ -140,8 +156,172 @@ Pipe ReadFromMergeTree::read() return pipe; } +static std::optional> filterPartsByVirtualColumns( + const MergeTreeData & data, + MergeTreeData::DataPartsVector & parts, + ASTPtr & query, + ContextPtr context) +{ + std::unordered_set part_values; + ASTPtr expression_ast; + auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); + + // Generate valid expressions for filtering + VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast); + + // If there is still something left, fill the virtual block and do the filtering. + if (expression_ast) + { + virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */); + VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context, expression_ast); + return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); + } + + return {}; +} + +static void filterPartsByPartition( + StorageMetadataPtr & metadata_snapshot, + const MergeTreeData & data, + SelectQueryInfo & query_info, + ContextPtr & context, + ContextPtr & query_context, + MergeTreeData::DataPartsVector & parts, + const std::optional> & part_values, + const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read, + Poco::Logger * log, + ReadFromMergeTree::IndexStats & index_stats) +{ + const Settings & settings = context->getSettingsRef(); + std::optional partition_pruner; + std::optional minmax_idx_condition; + DataTypes minmax_columns_types; + if (metadata_snapshot->hasPartitionKey()) + { + const auto & partition_key = metadata_snapshot->getPartitionKey(); + auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); + minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); + + minmax_idx_condition.emplace( + query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context))); + partition_pruner.emplace(metadata_snapshot->getPartitionKey(), query_info, context, false /* strict */); + + if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) + { + String msg = "Neither MinMax index by columns ("; + bool first = true; + for (const String & col : minmax_columns_names) + { + if (first) + first = false; + else + msg += ", "; + msg += col; + } + msg += ") nor partition expr is used and setting 'force_index_by_date' is set"; + + throw Exception(msg, ErrorCodes::INDEX_NOT_USED); + } + } + + MergeTreeDataSelectExecutor::PartFilterCounters part_filter_counters; + if (query_context->getSettingsRef().allow_experimental_query_deduplication) + MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( + parts, + part_values, + data.getPinnedPartUUIDs(), + minmax_idx_condition, + minmax_columns_types, + partition_pruner, + max_block_numbers_to_read, + query_context, + part_filter_counters, + log); + else + MergeTreeDataSelectExecutor::selectPartsToRead( + parts, + part_values, + minmax_idx_condition, + minmax_columns_types, + partition_pruner, + max_block_numbers_to_read, + part_filter_counters); + + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::None, + .num_parts_after = part_filter_counters.num_initial_selected_parts, + .num_granules_after = part_filter_counters.num_initial_selected_granules}); + + if (minmax_idx_condition) + { + auto description = minmax_idx_condition->getDescription(); + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::MinMax, + .condition = std::move(description.condition), + .used_keys = std::move(description.used_keys), + .num_parts_after = part_filter_counters.num_parts_after_minmax, + .num_granules_after = part_filter_counters.num_granules_after_minmax}); + LOG_DEBUG(log, "MinMax index condition: {}", minmax_idx_condition->toString()); + } + + if (partition_pruner) + { + auto description = partition_pruner->getKeyCondition().getDescription(); + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::Partition, + .condition = std::move(description.condition), + .used_keys = std::move(description.used_keys), + .num_parts_after = part_filter_counters.num_parts_after_partition_pruner, + .num_granules_after = part_filter_counters.num_granules_after_partition_pruner}); + } +} + void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) { + auto part_values = filterPartsByVirtualColumns(data, parts, query_info.query, context); + if (part_values && part_values->empty()) + { + pipeline.init(Pipe(std::make_shared(getOutputStream().header))); + return; + } + + /// If there are only virtual columns in the query, you must request at least one non-virtual one. + if (real_column_names.empty()) + { + NamesAndTypesList available_real_columns = metadata_snapshot->getColumns().getAllPhysical(); + real_column_names.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); + } + + metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); + + // Build and check if primary key is used when necessary + const auto & primary_key = metadata_snapshot->getPrimaryKey(); + Names primary_key_columns = primary_key.column_names; + KeyCondition key_condition(query_info, context, primary_key_columns, primary_key.expression); + + if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue()) + { + throw Exception( + ErrorCodes::INDEX_NOT_USED, + "Primary key ({}) is not used and setting 'force_primary_key' is set.", + fmt::join(primary_key_columns, ", ")); + } + LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); + + const auto & select = query_info.query->as(); + auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; + + filterPartsByPartition( + metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read, log, index_stats); + + bool sample_factor_column_queried = false; + for (const auto & col : virt_column_names) + if (col == "_sample_factor") + sample_factor_column_queried = true; + + auto sampling = MergeTreeDataSelectExecutor::getSampling( + select, parts, metadata_snapshot, key_condition, settings, data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context) + Pipe pipe = read(); for (const auto & processor : pipe.getProcessors()) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 479610b3edc..762d7fea0e7 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -36,7 +37,6 @@ public: }; using IndexStats = std::vector; - using IndexStatPtr = std::unique_ptr; /// Part of settings which are needed for reading. struct Settings @@ -46,6 +46,7 @@ public: size_t preferred_max_column_in_block_size_bytes; size_t min_marks_for_concurrent_read; bool use_uncompressed_cache; + bool force_primary_key; MergeTreeReaderSettings reader_settings; MergeTreeReadPool::BackoffSettings backoff_settings; @@ -67,12 +68,16 @@ public: }; ReadFromMergeTree( + SelectQueryInfo query_info_, + const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, + ContextPtr context_, + const MergeTreeData & data_, const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, String query_id_, - Names required_columns_, - RangesInDataParts parts_, - IndexStatPtr index_stats_, + Names real_column_names_, + MergeTreeData::DataPartsVector parts_, + //IndexStatPtr index_stats_, PrewhereInfoPtr prewhere_info_, Names virt_column_names_, Settings settings_, @@ -91,13 +96,17 @@ public: void describeIndexes(JSONBuilder::JSONMap & map) const override; private: + SelectQueryInfo query_info; + const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read; + ContextPtr context; + const MergeTreeData & data; const MergeTreeData & storage; StorageMetadataPtr metadata_snapshot; String query_id; - Names required_columns; - RangesInDataParts parts; - IndexStatPtr index_stats; + Names real_column_names; + MergeTreeData::DataPartsVector parts; + IndexStat index_stats; PrewhereInfoPtr prewhere_info; Names virt_column_names; Settings settings; @@ -105,6 +114,8 @@ private: size_t num_streams; ReadType read_type; + Poco::Logger * log; + Pipe read(); Pipe readFromPool(); Pipe readInOrder(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 821423ca5a5..444d074a4db 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -86,7 +86,8 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( const MergeTreeData::DataPartsVector & parts, const StorageMetadataPtr & metadata_snapshot, const KeyCondition & key_condition, - const Settings & settings) const + const Settings & settings, + Poco::Logger * log) { size_t rows_count = 0; @@ -369,6 +370,269 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( return plan; } +MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( + const ASTSelectQuery & select, + MergeTreeData::DataPartsVector & parts, + const StorageMetadataPtr & metadata_snapshot, + KeyCondition & key_condition, + const MergeTreeData & data, + Poco::Logger * log, + bool sample_factor_column_queried, + NamesAndTypesList available_real_columns, + ContextPtr context) +{ + const Settings & settings = context->getSettingsRef(); + Float64 used_sample_factor = 1; + /// Sampling. + MergeTreeDataSelectSamplingData sampling; + + RelativeSize relative_sample_size = 0; + RelativeSize relative_sample_offset = 0; + + auto select_sample_size = select.sampleSize(); + auto select_sample_offset = select.sampleOffset(); + + if (select_sample_size) + { + relative_sample_size.assign( + select_sample_size->as().ratio.numerator, + select_sample_size->as().ratio.denominator); + + if (relative_sample_size < 0) + throw Exception("Negative sample size", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + relative_sample_offset = 0; + if (select_sample_offset) + relative_sample_offset.assign( + select_sample_offset->as().ratio.numerator, + select_sample_offset->as().ratio.denominator); + + if (relative_sample_offset < 0) + throw Exception("Negative sample offset", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + /// Convert absolute value of the sampling (in form `SAMPLE 1000000` - how many rows to + /// read) into the relative `SAMPLE 0.1` (how much data to read). + size_t approx_total_rows = 0; + if (relative_sample_size > 1 || relative_sample_offset > 1) + approx_total_rows = getApproximateTotalRowsToRead(parts, metadata_snapshot, key_condition, settings, log); + + if (relative_sample_size > 1) + { + relative_sample_size = convertAbsoluteSampleSizeToRelative(select_sample_size, approx_total_rows); + LOG_DEBUG(log, "Selected relative sample size: {}", toString(relative_sample_size)); + } + + /// SAMPLE 1 is the same as the absence of SAMPLE. + if (relative_sample_size == RelativeSize(1)) + relative_sample_size = 0; + + if (relative_sample_offset > 0 && RelativeSize(0) == relative_sample_size) + throw Exception("Sampling offset is incorrect because no sampling", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + if (relative_sample_offset > 1) + { + relative_sample_offset = convertAbsoluteSampleSizeToRelative(select_sample_offset, approx_total_rows); + LOG_DEBUG(log, "Selected relative sample offset: {}", toString(relative_sample_offset)); + } + } + + /** Which range of sampling key values do I need to read? + * First, in the whole range ("universe") we select the interval + * of relative `relative_sample_size` size, offset from the beginning by `relative_sample_offset`. + * + * Example: SAMPLE 0.4 OFFSET 0.3 + * + * [------********------] + * ^ - offset + * <------> - size + * + * If the interval passes through the end of the universe, then cut its right side. + * + * Example: SAMPLE 0.4 OFFSET 0.8 + * + * [----------------****] + * ^ - offset + * <------> - size + * + * Next, if the `parallel_replicas_count`, `parallel_replica_offset` settings are set, + * then it is necessary to break the received interval into pieces of the number `parallel_replicas_count`, + * and select a piece with the number `parallel_replica_offset` (from zero). + * + * Example: SAMPLE 0.4 OFFSET 0.3, parallel_replicas_count = 2, parallel_replica_offset = 1 + * + * [----------****------] + * ^ - offset + * <------> - size + * <--><--> - pieces for different `parallel_replica_offset`, select the second one. + * + * It is very important that the intervals for different `parallel_replica_offset` cover the entire range without gaps and overlaps. + * It is also important that the entire universe can be covered using SAMPLE 0.1 OFFSET 0, ... OFFSET 0.9 and similar decimals. + */ + + /// Parallel replicas has been requested but there is no way to sample data. + /// Select all data from first replica and no data from other replicas. + if (settings.parallel_replicas_count > 1 && !data.supportsSampling() && settings.parallel_replica_offset > 0) + { + LOG_DEBUG(log, "Will use no data on this replica because parallel replicas processing has been requested" + " (the setting 'max_parallel_replicas') but the table does not support sampling and this replica is not the first."); + sampling.read_nothing = true; + return sampling; + } + + sampling.use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && data.supportsSampling()); + bool no_data = false; /// There is nothing left after sampling. + + if (sampling.use_sampling) + { + if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) + used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); + + RelativeSize size_of_universum = 0; + const auto & sampling_key = metadata_snapshot->getSamplingKey(); + DataTypePtr sampling_column_type = sampling_key.data_types[0]; + + if (sampling_key.data_types.size() == 1) + { + if (typeid_cast(sampling_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(sampling_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(sampling_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(sampling_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + } + + if (size_of_universum == RelativeSize(0)) + throw Exception( + "Invalid sampling column type in storage parameters: " + sampling_column_type->getName() + + ". Must be one unsigned integer type", + ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); + + if (settings.parallel_replicas_count > 1) + { + if (relative_sample_size == RelativeSize(0)) + relative_sample_size = 1; + + relative_sample_size /= settings.parallel_replicas_count.value; + relative_sample_offset += relative_sample_size * RelativeSize(settings.parallel_replica_offset.value); + } + + if (relative_sample_offset >= RelativeSize(1)) + no_data = true; + + /// Calculate the half-interval of `[lower, upper)` column values. + bool has_lower_limit = false; + bool has_upper_limit = false; + + RelativeSize lower_limit_rational = relative_sample_offset * size_of_universum; + RelativeSize upper_limit_rational = (relative_sample_offset + relative_sample_size) * size_of_universum; + + UInt64 lower = boost::rational_cast(lower_limit_rational); + UInt64 upper = boost::rational_cast(upper_limit_rational); + + if (lower > 0) + has_lower_limit = true; + + if (upper_limit_rational < size_of_universum) + has_upper_limit = true; + + /*std::cerr << std::fixed << std::setprecision(100) + << "relative_sample_size: " << relative_sample_size << "\n" + << "relative_sample_offset: " << relative_sample_offset << "\n" + << "lower_limit_float: " << lower_limit_rational << "\n" + << "upper_limit_float: " << upper_limit_rational << "\n" + << "lower: " << lower << "\n" + << "upper: " << upper << "\n";*/ + + if ((has_upper_limit && upper == 0) + || (has_lower_limit && has_upper_limit && lower == upper)) + no_data = true; + + if (no_data || (!has_lower_limit && !has_upper_limit)) + { + sampling.use_sampling = false; + } + else + { + /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. + + std::shared_ptr lower_function; + std::shared_ptr upper_function; + + /// If sample and final are used together no need to calculate sampling expression twice. + /// The first time it was calculated for final, because sample key is a part of the PK. + /// So, assume that we already have calculated column. + ASTPtr sampling_key_ast = metadata_snapshot->getSamplingKeyAST(); + + if (select.final()) + { + sampling_key_ast = std::make_shared(sampling_key.column_names[0]); + /// We do spoil available_real_columns here, but it is not used later. + available_real_columns.emplace_back(sampling_key.column_names[0], std::move(sampling_column_type)); + } + + if (has_lower_limit) + { + if (!key_condition.addCondition(sampling_key.column_names[0], Range::createLeftBounded(lower, true))) + throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); + + ASTPtr args = std::make_shared(); + args->children.push_back(sampling_key_ast); + args->children.push_back(std::make_shared(lower)); + + lower_function = std::make_shared(); + lower_function->name = "greaterOrEquals"; + lower_function->arguments = args; + lower_function->children.push_back(lower_function->arguments); + + sampling.filter_function = lower_function; + } + + if (has_upper_limit) + { + if (!key_condition.addCondition(sampling_key.column_names[0], Range::createRightBounded(upper, false))) + throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); + + ASTPtr args = std::make_shared(); + args->children.push_back(sampling_key_ast); + args->children.push_back(std::make_shared(upper)); + + upper_function = std::make_shared(); + upper_function->name = "less"; + upper_function->arguments = args; + upper_function->children.push_back(upper_function->arguments); + + sampling.filter_function = upper_function; + } + + if (has_lower_limit && has_upper_limit) + { + ASTPtr args = std::make_shared(); + args->children.push_back(lower_function); + args->children.push_back(upper_function); + + sampling.filter_function = std::make_shared(); + sampling.filter_function->name = "and"; + sampling.filter_function->arguments = args; + sampling.filter_function->children.push_back(sampling.filter_function->arguments); + } + + ASTPtr query = sampling.filter_function; + auto syntax_result = TreeRewriter(context).analyze(query, available_real_columns); + sampling.filter_expression = ExpressionAnalyzer(sampling.filter_function, syntax_result, context).getActionsDAG(false); + } + } + + if (no_data) + { + LOG_DEBUG(log, "Sampling yields no data."); + sampling.read_nothing = true; + } + + return sampling; +} + QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names_to_return, @@ -534,7 +798,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( partition_pruner, max_block_numbers_to_read, query_context, - part_filter_counters); + part_filter_counters, + log); else selectPartsToRead( parts, @@ -2192,7 +2457,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( void MergeTreeDataSelectExecutor::selectPartsToRead( MergeTreeData::DataPartsVector & parts, - const std::unordered_set & part_values, + const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, @@ -2204,7 +2469,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( for (const auto & part_or_projection : prev_parts) { const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get(); - if (!part_values.empty() && part_values.find(part->name) == part_values.end()) + if (part_values && part_values->find(part->name) == part_values->end()) continue; if (part->isEmpty()) @@ -2246,14 +2511,15 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, - const std::unordered_set & part_values, + const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context, - PartFilterCounters & counters) const + PartFilterCounters & counters, + Poco::Logger * log) { const Settings & settings = query_context->getSettings(); @@ -2269,7 +2535,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( for (const auto & part_or_projection : prev_parts) { const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get(); - if (!part_values.empty() && part_values.find(part->name) == part_values.end()) + if (part_values && part_values->find(part->name) == part_values->end()) continue; if (part->isEmpty()) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index c058c7e936b..231908e2ee7 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -16,6 +16,7 @@ class KeyCondition; struct MergeTreeDataSelectSamplingData { bool use_sampling = false; + bool read_nothing = false; std::shared_ptr filter_function; ActionsDAGPtr filter_expression; }; @@ -117,11 +118,12 @@ private: const String & query_id) const; /// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index. - size_t getApproximateTotalRowsToRead( + static size_t getApproximateTotalRowsToRead( const MergeTreeData::DataPartsVector & parts, const StorageMetadataPtr & metadata_snapshot, const KeyCondition & key_condition, - const Settings & settings) const; + const Settings & settings, + Poco::Logger * log); static MarkRanges markRangesFromPKRange( const MergeTreeData::DataPartPtr & part, @@ -141,6 +143,7 @@ private: size_t & granules_dropped, Poco::Logger * log); +public: struct PartFilterCounters { size_t num_initial_selected_parts = 0; @@ -155,7 +158,7 @@ private: /// as well as `max_block_number_to_read`. static void selectPartsToRead( MergeTreeData::DataPartsVector & parts, - const std::unordered_set & part_values, + const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, @@ -163,16 +166,28 @@ private: PartFilterCounters & counters); /// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded. - void selectPartsToReadWithUUIDFilter( + static void selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, - const std::unordered_set & part_values, + const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context, - PartFilterCounters & counters) const; + PartFilterCounters & counters, + Poco::Logger * log); + + static MergeTreeDataSelectSamplingData getSampling( + const ASTSelectQuery & select, + MergeTreeData::DataPartsVector & parts, + const StorageMetadataPtr & metadata_snapshot, + KeyCondition & key_condition, + const MergeTreeData & data, + Poco::Logger * log, + bool sample_factor_column_queried, + NamesAndTypesList available_real_columns, + ContextPtr context); }; } From 7d1431f6b6b38af5567b850a8627441d88adfaba Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 25 May 2021 20:25:00 +0300 Subject: [PATCH 077/154] better code near dropPartition --- src/Storages/StorageMergeTree.cpp | 27 +++++++++-------- src/Storages/StorageReplicatedMergeTree.cpp | 33 ++++++++++----------- src/Storages/StorageReplicatedMergeTree.h | 2 +- 3 files changed, 31 insertions(+), 31 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 754a9b7b115..3b13f625d2c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1242,30 +1242,33 @@ MergeTreeDataPartPtr StorageMergeTree::outdatePart(const String & part_name, boo void StorageMergeTree::dropPartNoWaitNoThrow(const String & part_name) { - if (auto part = outdatePart(part_name, false)) - dropPartsImpl({part}, false); + if (auto part = outdatePart(part_name, /*force=*/ false)) + dropPartsImpl({part}, /*detach=*/ false); /// Else nothing to do, part was removed in some different way } void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPtr /*query_context*/) { - if (auto part = outdatePart(part_name, true)) + if (auto part = outdatePart(part_name, /*force=*/ true)) dropPartsImpl({part}, detach); - - /// Else nothing to do, part was removed in some different way } void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, ContextPtr local_context) { - /// Asks to complete merges and does not allow them to start. - /// This protects against "revival" of data for a removed partition after completion of merge. - auto merge_blocker = stopMergesAndWait(); - String partition_id = getPartitionIDFromQuery(partition, local_context); - auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + DataPartsVector parts_to_remove; + /// New scope controls lifetime of merge_blocker. + { + /// Asks to complete merges and does not allow them to start. + /// This protects against "revival" of data for a removed partition after completion of merge. + auto merge_blocker = stopMergesAndWait(); + String partition_id = getPartitionIDFromQuery(partition, local_context); + parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + + /// TODO should we throw an exception if parts_to_remove is empty? + removePartsFromWorkingSet(parts_to_remove, true); + } - /// TODO should we throw an exception if parts_to_remove is empty? - removePartsFromWorkingSet(parts_to_remove, true); dropPartsImpl(std::move(parts_to_remove), detach); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 03415966798..0cfebfdedfd 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4936,7 +4936,7 @@ void StorageReplicatedMergeTree::dropPartNoWaitNoThrow(const String & part_name) zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); LogEntry entry; - dropPartImpl(zookeeper, part_name, entry, false, false); + dropPartImpl(zookeeper, part_name, entry, /*detach=*/ false, /*throw_if_noop=*/ false); } void StorageReplicatedMergeTree::dropPart(const String & part_name, bool detach, ContextPtr query_context) @@ -4948,16 +4948,13 @@ void StorageReplicatedMergeTree::dropPart(const String & part_name, bool detach, zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); LogEntry entry; - bool did_drop = dropPartImpl(zookeeper, part_name, entry, detach, true); + dropPartImpl(zookeeper, part_name, entry, detach, /*throw_if_noop=*/ true); - if (did_drop) - { - /// If necessary, wait until the operation is performed on itself or on all replicas. - if (query_context->getSettingsRef().replication_alter_partitions_sync == 1) - waitForReplicaToProcessLogEntry(replica_name, entry); - else if (query_context->getSettingsRef().replication_alter_partitions_sync == 2) - waitForAllReplicasToProcessLogEntry(entry); - } + /// If necessary, wait until the operation is performed on itself or on all replicas. + if (query_context->getSettingsRef().replication_alter_partitions_sync == 1) + waitForReplicaToProcessLogEntry(replica_name, entry); + else if (query_context->getSettingsRef().replication_alter_partitions_sync == 2) + waitForAllReplicasToProcessLogEntry(entry); } void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool detach, ContextPtr query_context) @@ -4979,9 +4976,9 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool de waitForReplicaToProcessLogEntry(replica_name, entry); else if (query_context->getSettingsRef().replication_alter_partitions_sync == 2) waitForAllReplicasToProcessLogEntry(entry); - } - cleanLastPartNode(partition_id); + cleanLastPartNode(partition_id); + } } @@ -6716,7 +6713,7 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI return true; } -bool StorageReplicatedMergeTree::dropPartImpl( +void StorageReplicatedMergeTree::dropPartImpl( zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop) { LOG_TRACE(log, "Will try to insert a log entry to DROP_RANGE for part: " + part_name); @@ -6733,7 +6730,7 @@ bool StorageReplicatedMergeTree::dropPartImpl( { if (throw_if_noop) throw Exception("Part " + part_name + " not found locally, won't try to drop it.", ErrorCodes::NO_SUCH_DATA_PART); - return false; + return; } /// There isn't a lot we can do otherwise. Can't cancel merges because it is possible that a replica already @@ -6744,7 +6741,7 @@ bool StorageReplicatedMergeTree::dropPartImpl( throw Exception("Part " + part_name + " is currently participating in a background operation (mutation/merge)" + ", try again later", ErrorCodes::PART_IS_TEMPORARILY_LOCKED); - return false; + return; } if (partIsLastQuorumPart(part->info)) @@ -6752,7 +6749,7 @@ bool StorageReplicatedMergeTree::dropPartImpl( if (throw_if_noop) throw Exception("Part " + part_name + " is last inserted part with quorum in partition. Cannot drop", ErrorCodes::NOT_IMPLEMENTED); - return false; + return; } if (partIsInsertingWithParallelQuorum(part->info)) @@ -6760,7 +6757,7 @@ bool StorageReplicatedMergeTree::dropPartImpl( if (throw_if_noop) throw Exception("Part " + part_name + " is inserting with parallel quorum. Cannot drop", ErrorCodes::NOT_IMPLEMENTED); - return false; + return; } Coordination::Requests ops; @@ -6802,7 +6799,7 @@ bool StorageReplicatedMergeTree::dropPartImpl( String log_znode_path = dynamic_cast(*responses[clear_block_ops_size + 1]).path_created; entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - return true; + return; } } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 9d0727ba0e7..b75823a14dd 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -648,7 +648,7 @@ private: bool fetch_part, ContextPtr query_context) override; - bool dropPartImpl(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop); + void dropPartImpl(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop); /// Check granularity of already existing replicated table in zookeeper if it exists /// return true if it's fixed From 940eab692478b306d6f81ca352b4f19c4eab87d9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 25 May 2021 16:45:46 +0300 Subject: [PATCH 078/154] Review fixes --- src/Common/createFile.cpp | 52 ++++++++++++------- src/Common/createFile.h | 2 + src/Disks/DiskLocal.cpp | 26 ++-------- src/Disks/S3/DiskS3.cpp | 6 +-- src/Interpreters/loadMetadata.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 6 files changed, 43 insertions(+), 47 deletions(-) diff --git a/src/Common/createFile.cpp b/src/Common/createFile.cpp index 74f8f9cf980..64e344f1f22 100644 --- a/src/Common/createFile.cpp +++ b/src/Common/createFile.cpp @@ -10,31 +10,13 @@ namespace DB { namespace ErrorCodes { -extern const int FILE_ALREADY_EXISTS; extern const int PATH_ACCESS_DENIED; -extern const int NOT_ENOUGH_SPACE; extern const int CANNOT_CREATE_FILE; } } namespace FS { -[[noreturn]] void handleLastError(const std::string & path) -{ - switch (errno) - { - case EEXIST: - throw DB::Exception(DB::ErrorCodes::FILE_ALREADY_EXISTS, "File {} already exist", path); - case EPERM: - throw DB::Exception(DB::ErrorCodes::PATH_ACCESS_DENIED, "Not enough permissions to create file {}", path); - case ENOSPC: - throw DB::Exception(DB::ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space to create file {}", path); - case ENAMETOOLONG: - throw DB::Exception(DB::ErrorCodes::CANNOT_CREATE_FILE, "File name {} is too long"); - default: - throw DB::Exception(DB::ErrorCodes::CANNOT_CREATE_FILE, "Cannot create file {}. Error: {}", path, strerror(errno)); - } -} /// Copy from Poco::createFile bool createFile(const std::string & path) @@ -45,6 +27,38 @@ bool createFile(const std::string & path) close(n); return true; } - handleLastError(path); + DB::throwFromErrnoWithPath("Cannot create file: " + path, path, DB::ErrorCodes::CANNOT_CREATE_FILE); } + +bool canRead(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + { + if (st.st_uid == geteuid()) + return (st.st_mode & S_IRUSR) != 0; + else if (st.st_gid == getegid()) + return (st.st_mode & S_IRGRP) != 0; + else + return (st.st_mode & S_IROTH) != 0 || geteuid() == 0; + } + DB::throwFromErrnoWithPath("Cannot check read access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + + +bool canWrite(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + { + if (st.st_uid == geteuid()) + return (st.st_mode & S_IWUSR) != 0; + else if (st.st_gid == getegid()) + return (st.st_mode & S_IWGRP) != 0; + else + return (st.st_mode & S_IWOTH) != 0 || geteuid() == 0; + } + DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + } diff --git a/src/Common/createFile.h b/src/Common/createFile.h index 67c7023a455..f96d141d39e 100644 --- a/src/Common/createFile.h +++ b/src/Common/createFile.h @@ -4,4 +4,6 @@ namespace FS { bool createFile(const std::string & path); +bool canRead(const std::string & path); +bool canWrite(const std::string & path); } diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 51ed7fe577b..e31731a7edc 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -208,17 +208,7 @@ void DiskLocal::replaceFile(const String & from_path, const String & to_path) { fs::path from_file = fs::path(disk_path) / from_path; fs::path to_file = fs::path(disk_path) / to_path; - if (fs::exists(to_file)) - { - fs::path tmp_file(to_file.string() + ".old"); - fs::rename(to_file, tmp_file); - fs::rename(from_file, fs::path(disk_path) / to_path); - fs::remove(tmp_file); - } - else - { - fs::rename(from_file, to_file); - } + fs::rename(from_file, to_file); } std::unique_ptr @@ -300,8 +290,8 @@ void DiskLocal::createFile(const String & path) void DiskLocal::setReadOnly(const String & path) { fs::permissions(fs::path(disk_path) / path, - fs::perms::owner_read | fs::perms::group_read | fs::perms::others_read, - fs::perm_options::replace); + fs::perms::owner_write | fs::perms::group_write | fs::perms::others_write, + fs::perm_options::remove); } bool inline isSameDiskType(const IDisk & one, const IDisk & another) @@ -399,15 +389,7 @@ void registerDiskLocal(DiskFactory & factory) throw Exception("Disk path must end with /. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); } - fs::path disk(path); - fs::perms p = fs::status(disk).permissions(); - bool is_readable = ((p & fs::perms::owner_read) != fs::perms::none) - | ((p & fs::perms::group_read) != fs::perms::none) - | ((p & fs::perms::others_read) != fs::perms::none); - bool is_writable = ((p & fs::perms::owner_write) != fs::perms::none) - | ((p & fs::perms::group_write) != fs::perms::none) - | ((p & fs::perms::others_write) != fs::perms::none); - if (!is_readable || !is_writable) + if (!FS::canRead(path) || !FS::canWrite(path)) throw Exception("There is no RW access to the disk " + name + " (" + path + ")", ErrorCodes::PATH_ACCESS_DENIED); bool has_space_ratio = config.has(config_prefix + ".keep_free_space_ratio"); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 4fbfc9141a6..7dd3381cd51 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1506,11 +1506,9 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); fs::path from_path = fs::path(metadata_path) / path; - fs::path to_path = fs::path(metadata_path) / detached_path; + fs::path to_path = fs::path(metadata_path) / detached_path / from_path.filename(); if (path.ends_with('/')) - to_path /= from_path.parent_path().filename(); - else - to_path /= from_path.filename(); + to_path /= to_path.parent_path(); fs::copy(from_path, to_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); fs::remove_all(from_path); } diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 0a84cdca309..b60cd76e85a 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -114,7 +114,7 @@ void loadMetadata(ContextPtr context, const String & default_database_name) /// TODO: DETACH DATABASE PERMANENTLY ? if (fs::path(current_file).extension() == ".sql") { - String db_name = current_file.substr(0, current_file.size() - 4); + String db_name = fs::path(current_file).stem(); if (db_name != DatabaseCatalog::SYSTEM_DATABASE) databases.emplace(unescapeForFileName(db_name), fs::path(path) / db_name); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 3a3ac90fab0..a76297f36a2 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -114,7 +114,7 @@ using RelativeSize = boost::rational; static std::string toString(const RelativeSize & x) { - return fs::path(ASTSampleRatio::toString(x.numerator())) / ASTSampleRatio::toString(x.denominator()); + return ASTSampleRatio::toString(x.numerator()) + "/" + ASTSampleRatio::toString(x.denominator()); } /// Converts sample size to an approximate number of rows (ex. `SAMPLE 1000000`) to relative value (ex. `SAMPLE 0.1`). From 3782d6bba2db803c59af464a5862ba61fca1a2ba Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Wed, 26 May 2021 13:34:28 +0200 Subject: [PATCH 079/154] Output row number deleted --- docs/en/sql-reference/statements/optimize.md | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 47fee3d78d8..8ba6db4cbf6 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -85,8 +85,6 @@ Result: │ 1 │ 1 │ 2 │ 3 │ │ 1 │ 1 │ 3 │ 3 │ └─────────────┴───────────────┴───────┴───────────────┘ - -5 rows in set. ``` When columns for deduplication are not specified, all of them are taken into account. Row is removed only if all values in all columns are equal to corresponding values in previous row: @@ -109,8 +107,6 @@ Result: │ 1 │ 1 │ 2 │ 3 │ │ 1 │ 1 │ 3 │ 3 │ └─────────────┴───────────────┴───────┴───────────────┘ - -4 rows in set. ``` When columns are specified implicitly, the table is deduplicated by all columns that are not `ALIAS` or `MATERIALIZED`. Considering the table above, these are `primary_key`, `secondary_key`, `value`, and `partition_key` columns: @@ -132,8 +128,6 @@ Result: │ 1 │ 1 │ 2 │ 3 │ │ 1 │ 1 │ 3 │ 3 │ └─────────────┴───────────────┴───────┴───────────────┘ - -4 rows in set. ``` Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED` and explicitly not `value`: `primary_key`, `secondary_key`, and `partition_key` columns. @@ -155,8 +149,6 @@ Result: ┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ │ 1 │ 1 │ 2 │ 3 │ └─────────────┴───────────────┴───────┴───────────────┘ - -3 rows in set. ``` Deduplicate explicitly by `primary_key`, `secondary_key`, and `partition_key` columns: @@ -177,8 +169,6 @@ Result: ┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ │ 1 │ 1 │ 2 │ 3 │ └─────────────┴───────────────┴───────┴───────────────┘ - -3 rows in set. ``` Deduplicate by any column matching a regex: `primary_key`, `secondary_key`, and `partition_key` columns: @@ -199,6 +189,4 @@ Result: ┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ │ 1 │ 1 │ 2 │ 3 │ └─────────────┴───────────────┴───────┴───────────────┘ - -3 rows in set. ``` \ No newline at end of file From 4d296249d59de0dc31adbf0cbcaf5e2acee98658 Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Wed, 26 May 2021 14:17:52 +0200 Subject: [PATCH 080/154] RU version edited --- docs/ru/sql-reference/statements/optimize.md | 143 ++++++++++++++++--- 1 file changed, 124 insertions(+), 19 deletions(-) diff --git a/docs/ru/sql-reference/statements/optimize.md b/docs/ru/sql-reference/statements/optimize.md index e1a9d613537..93f7fde707f 100644 --- a/docs/ru/sql-reference/statements/optimize.md +++ b/docs/ru/sql-reference/statements/optimize.md @@ -32,10 +32,22 @@ ClickHouse не оповещает клиента. Чтобы включить Список столбцов для дедупликации должен включать все столбцы, указанные в условиях сортировки (первичный ключ и ключ сортировки), а также в условиях партиционирования (ключ партиционирования). - !!! note "Примечание" - Обратите внимание, что символ подстановки `*` обрабатывается так же, как и в запросах `SELECT`: столбцы `MATERIALIZED` и `ALIAS` не включаются в результат. +!!! note "Примечание" + Обратите внимание, что символ подстановки `*` обрабатывается так же, как и в запросах `SELECT`: столбцы [MATERIALIZED](../../sql-reference/statements/create/table.md#materialized) и [ALIAS](../../sql-reference/statements/create/table.md#alias) не включаются в результат. Если указать пустой список или выражение, которое возвращает пустой список, или дедуплицировать столбец по псевдониму (`ALIAS`), то сервер вернет ошибку. +**Синтаксис** + +``` sql +OPTIMIZE TABLE table DEDUPLICATE; -- по всем столбцам +OPTIMIZE TABLE table DEDUPLICATE BY *; -- исключаются MATERIALIZED и ALIAS столбцы +OPTIMIZE TABLE table DEDUPLICATE BY colX,colY,colZ; +OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT colX; +OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT (colX, colY); +OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex'); +OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT colX; +OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT (colX, colY); +``` **Примеры** @@ -50,38 +62,131 @@ CREATE TABLE example ( materialized_value UInt32 MATERIALIZED 12345, aliased_value UInt32 ALIAS 2, PRIMARY KEY primary_key -) ENGINE=MergeTree -PARTITION BY partition_key; +) ENGINE=MergeTree  +PARTITION BY partition_key +ORDER BY (primary_key, secondary_key); +``` +``` sql +INSERT INTO example (primary_key, secondary_key, value, partition_key) +VALUES (0, 0, 0, 0), (0, 0, 0, 0), (1, 1, 2, 2), (1, 1, 2, 3), (1, 1, 3, 3); +``` +``` sql +SELECT * FROM example; +``` +Результат: ``` -Прежний способ дедупликации, когда учитываются все столбцы. Строка удаляется только в том случае, если все значения во всех столбцах равны соответствующим значениям в предыдущей строке. +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +│ 1 │ 1 │ 3 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ +``` +Если в запросе не указаны столбцы, по которым нужно дедуплицировать, то учитываются все столбцы таблицы. Строка удаляется только в том случае, если все значения во всех столбцах равны соответствующим значениям в другой строке. ``` sql OPTIMIZE TABLE example FINAL DEDUPLICATE; ``` - -Дедупликация по всем столбцам, кроме `ALIAS` и `MATERIALIZED`: `primary_key`, `secondary_key`, `value`, `partition_key` и `materialized_value`. - - ``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE BY *; +SELECT * FROM example; +``` +Результат: +``` +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +│ 1 │ 1 │ 3 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ ``` -Дедупликация по всем столбцам, кроме `ALIAS`, `MATERIALIZED` и `materialized_value`: столбцы `primary_key`, `secondary_key`, `value` и `partition_key`. - - +Если столбцы в запросе указаны через `*`, то дедупликация пройдет по всем столбцам, кроме `ALIAS` и `MATERIALIZED`. Для таблицы `example` будут учтены: `primary_key`, `secondary_key`, `value` и `partition_key`. +```sql +OPTIMIZE TABLE example FINAL DEDUPLICATE BY *; +``` ``` sql -OPTIMIZE TABLE example FINAL DEDUPLICATE BY * EXCEPT materialized_value; +SELECT * FROM example; +``` +Результат: +``` +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +│ 1 │ 1 │ 3 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ +``` + +Дедупликация по всем столбцам, кроме `ALIAS` и `MATERIALIZED` (BY `*`), и с исключением столбца `value`: `primary_key`, `secondary_key` и `partition_key`. +``` sql +OPTIMIZE TABLE example FINAL DEDUPLICATE BY * EXCEPT value; +``` +``` sql +SELECT * FROM example; +``` +Результат: +``` +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ ``` Дедупликация по столбцам `primary_key`, `secondary_key` и `partition_key`. - -``` sql +```sql OPTIMIZE TABLE example FINAL DEDUPLICATE BY primary_key, secondary_key, partition_key; ``` - -Дедупликация по любому столбцу, соответствующему регулярному выражению: столбцам `primary_key`, `secondary_key` и `partition_key`. - ``` sql +SELECT * FROM example; +``` +Результат: +``` +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ +``` + +Дедупликация по любому столбцу, который соответствует регулярному выражению `.*_key`: `primary_key`, `secondary_key` и `partition_key`. +```sql OPTIMIZE TABLE example FINAL DEDUPLICATE BY COLUMNS('.*_key'); ``` +``` sql +SELECT * FROM example; +``` +Результат: +``` +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 0 │ 0 │ 0 │ 0 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 2 │ +└─────────────┴───────────────┴───────┴───────────────┘ +┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ +│ 1 │ 1 │ 2 │ 3 │ +└─────────────┴───────────────┴───────┴───────────────┘ +``` \ No newline at end of file From e0011fa7980fdf3e9f645957a0465b1c9edac369 Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Wed, 26 May 2021 14:44:10 +0200 Subject: [PATCH 081/154] Ru version note edited --- docs/ru/sql-reference/statements/optimize.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/statements/optimize.md b/docs/ru/sql-reference/statements/optimize.md index 93f7fde707f..976c1994414 100644 --- a/docs/ru/sql-reference/statements/optimize.md +++ b/docs/ru/sql-reference/statements/optimize.md @@ -34,13 +34,13 @@ ClickHouse не оповещает клиента. Чтобы включить !!! note "Примечание" Обратите внимание, что символ подстановки `*` обрабатывается так же, как и в запросах `SELECT`: столбцы [MATERIALIZED](../../sql-reference/statements/create/table.md#materialized) и [ALIAS](../../sql-reference/statements/create/table.md#alias) не включаются в результат. - Если указать пустой список или выражение, которое возвращает пустой список, или дедуплицировать столбец по псевдониму (`ALIAS`), то сервер вернет ошибку. + Если указать пустой список или выражение, которое возвращает пустой список, то сервер вернет ошибку. Запрос вида `DEDUPLICATE BY aliased_value` также вернет ошибку. **Синтаксис** ``` sql OPTIMIZE TABLE table DEDUPLICATE; -- по всем столбцам -OPTIMIZE TABLE table DEDUPLICATE BY *; -- исключаются MATERIALIZED и ALIAS столбцы +OPTIMIZE TABLE table DEDUPLICATE BY *; -- исключаются столбцы MATERIALIZED и ALIAS OPTIMIZE TABLE table DEDUPLICATE BY colX,colY,colZ; OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT colX; OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT (colX, colY); From 4314886268d96f1473b8af742afe75e2c88f9796 Mon Sep 17 00:00:00 2001 From: Alina Terekhova Date: Wed, 26 May 2021 14:46:35 +0200 Subject: [PATCH 082/154] Minor change --- docs/ru/sql-reference/statements/optimize.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/optimize.md b/docs/ru/sql-reference/statements/optimize.md index 976c1994414..70503ec4de9 100644 --- a/docs/ru/sql-reference/statements/optimize.md +++ b/docs/ru/sql-reference/statements/optimize.md @@ -131,7 +131,7 @@ SELECT * FROM example; └─────────────┴───────────────┴───────┴───────────────┘ ``` -Дедупликация по всем столбцам, кроме `ALIAS` и `MATERIALIZED` (BY `*`), и с исключением столбца `value`: `primary_key`, `secondary_key` и `partition_key`. +Дедупликация по всем столбцам, кроме `ALIAS` и `MATERIALIZED` (`BY *`), и с исключением столбца `value`: `primary_key`, `secondary_key` и `partition_key`. ``` sql OPTIMIZE TABLE example FINAL DEDUPLICATE BY * EXCEPT value; ``` From 6bc0167970a4cbf8bef58ead5e325fae89a325e9 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Wed, 26 May 2021 17:35:05 +0300 Subject: [PATCH 083/154] Add test --- ...01881_total_bytes_storage_buffer.reference | 5 ++++ .../01881_total_bytes_storage_buffer.sql | 23 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/01881_total_bytes_storage_buffer.reference create mode 100644 tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql diff --git a/tests/queries/0_stateless/01881_total_bytes_storage_buffer.reference b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.reference new file mode 100644 index 00000000000..eaa9e9259df --- /dev/null +++ b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.reference @@ -0,0 +1,5 @@ +0 +8192 +32 +8192 +32 diff --git a/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql new file mode 100644 index 00000000000..d962a6db456 --- /dev/null +++ b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS test_buffer_table; + +CREATE TABLE test_buffer_table +( + `a` Int64 +) +ENGINE = Buffer('', '', 1, 1000000, 1000000, 1000000, 1000000, 1000000, 1000000); + +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; + +INSERT INTO test_buffer_table SELECT number FROM numbers(1000); +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; + +OPTIMIZE TABLE test_buffer_table; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; + +INSERT INTO test_buffer_table SELECT number FROM numbers(1000); +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; + +OPTIMIZE TABLE test_buffer_table; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; + +DROP TABLE test_buffer_table; From fee5351387ec8f400e7b07e3093c0a59d4348b1c Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 26 May 2021 20:12:10 +0300 Subject: [PATCH 084/154] Update docs/en/sql-reference/table-functions/dictionary.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index 3e3d2a10984..93bb5b73947 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -42,7 +42,7 @@ SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_fu Query: ``` sql -SELECT * FROM dictionary('table_function_dictionary_test_dictionary'); +SELECT * FROM dictionary('new_dictionary'); ``` Result: From 8877c3b5c56d5ba26b450961e7e95088bb40bab3 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 26 May 2021 20:12:20 +0300 Subject: [PATCH 085/154] Update docs/en/sql-reference/table-functions/dictionary.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index 93bb5b73947..4082caf78cd 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -36,7 +36,7 @@ Create a dictionary: ``` sql CREATE DICTIONARY table_function_dictionary_test_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_function_dictionary_source_table')) LAYOUT(DIRECT()); +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table')) LAYOUT(DIRECT()); ``` Query: From c8b9a464d71936c1ea5c0a910be95ead1528b3b6 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 26 May 2021 20:12:30 +0300 Subject: [PATCH 086/154] Update docs/en/sql-reference/table-functions/dictionary.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index 4082caf78cd..ae9d9af03c0 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -23,7 +23,7 @@ A ClickHouse table. **Example** -Input table: +Input table `dictionary_source_table`: ``` text ┌─id─┬─value─┐ From 2a1c5acd31b724ae457a698ce8bc4e264d0bd639 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 26 May 2021 20:12:40 +0300 Subject: [PATCH 087/154] Update docs/en/sql-reference/table-functions/dictionary.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index ae9d9af03c0..fca680e5c00 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -5,7 +5,7 @@ toc_title: dictionary function # dictionary {#dictionary-function} -Displays the [dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. +Displays the [dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. Works the same way as [Dictionary](../../engines/table-engines/special/dictionary.md) engine. **Syntax** From 620f2755231fff1f1b29b190cbd16a040cc9c470 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 26 May 2021 20:13:05 +0300 Subject: [PATCH 088/154] Update docs/en/sql-reference/table-functions/dictionary.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index fca680e5c00..675fcb5bfdd 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -35,7 +35,7 @@ Input table `dictionary_source_table`: Create a dictionary: ``` sql -CREATE DICTIONARY table_function_dictionary_test_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id +CREATE DICTIONARY new_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table')) LAYOUT(DIRECT()); ``` From 991ae97fb6797d5ab3f1a1e0bfb887fcba2e866f Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 26 May 2021 20:53:51 +0300 Subject: [PATCH 089/154] Translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../table-engines/special/dictionary.md | 4 +- .../external-dicts-dict-sources.md | 4 ++ .../table-engines/special/dictionary.md | 3 + .../external-dicts-dict-sources.md | 4 ++ .../table-functions/dictionary.md | 59 +++++++++++++++++++ 5 files changed, 73 insertions(+), 1 deletion(-) create mode 100644 docs/ru/sql-reference/table-functions/dictionary.md diff --git a/docs/en/engines/table-engines/special/dictionary.md b/docs/en/engines/table-engines/special/dictionary.md index a6d6f296673..e143f3a8ee1 100644 --- a/docs/en/engines/table-engines/special/dictionary.md +++ b/docs/en/engines/table-engines/special/dictionary.md @@ -94,4 +94,6 @@ select * from products limit 1; └───────────────┴─────────────────┘ ``` -[Original article](https://clickhouse.tech/docs/en/operations/table_engines/dictionary/) +**See Also** + +- [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index dc0b6e17198..3b9a28403f7 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -98,6 +98,10 @@ Setting fields: When dictionary with source `FILE` is created via DDL command (`CREATE DICTIONARY ...`), the source file needs to be located in `user_files` directory, to prevent DB users accessing arbitrary file on ClickHouse node. +**See Also** + +- [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) + ## Executable File {#dicts-external_dicts_dict_sources-executable} Working with executable files depends on [how the dictionary is stored in memory](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts executable file and treats its output as dictionary data. diff --git a/docs/ru/engines/table-engines/special/dictionary.md b/docs/ru/engines/table-engines/special/dictionary.md index 243fd5395c0..df47ade5a42 100644 --- a/docs/ru/engines/table-engines/special/dictionary.md +++ b/docs/ru/engines/table-engines/special/dictionary.md @@ -90,3 +90,6 @@ select * from products limit 1; └───────────────┴─────────────────┘ ``` +**Смотрите также** + +- [Функция dictionary](../../sql-reference/table-functions/dictionary.md#dictionary-function) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index a7999470330..a9f3857f183 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -97,6 +97,10 @@ SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated')) Если словарь с источником `FILE` создается с помощью DDL-команды (`CREATE DICTIONARY ...`), источник словаря должен быть расположен в каталоге `user_files`. Иначе пользователи базы данных будут иметь доступ к произвольному файлу на узле ClickHouse. +**Смотрите также** + +- [Функция dictionary](../../sql-reference/table-functions/dictionary.md#dictionary-function) + ## Исполняемый файл {#dicts-external_dicts_dict_sources-executable} Работа с исполняемым файлом зависит от [размещения словаря в памяти](external-dicts-dict-layout.md). Если тип размещения словаря `cache` и `complex_key_cache`, то ClickHouse запрашивает необходимые ключи, отправляя запрос в `STDIN` исполняемого файла. diff --git a/docs/ru/sql-reference/table-functions/dictionary.md b/docs/ru/sql-reference/table-functions/dictionary.md new file mode 100644 index 00000000000..d4909bf5d9f --- /dev/null +++ b/docs/ru/sql-reference/table-functions/dictionary.md @@ -0,0 +1,59 @@ +--- +toc_priority: 54 +toc_title: dictionary +--- + +# dictionary {#dictionary-function} + +Отображает данные [словаря](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) как таблицу ClickHouse. Работает аналогично движку [Dictionary](../../engines/table-engines/special/dictionary.md). + +**Синтаксис** + +``` sql +dictionary('dict') +``` + +**Аргументы** + +- `dict` — имя словаря. [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +Таблица ClickHouse. + +**Пример** + +Входная таблица `dictionary_source_table`: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +Создаем словарь: + +``` sql +CREATE DICTIONARY new_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table')) LAYOUT(DIRECT()); +``` + +Запрос: + +``` sql +SELECT * FROM dictionary('new_dictionary'); +``` + +Результат: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +**Смотрите также** + +- [Движок Dictionary](../../engines/table-engines/special/dictionary.md#dictionary) From a51a6ea0b78f305fbcc33703505256ed2f5b0369 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 26 May 2021 21:14:43 +0300 Subject: [PATCH 090/154] Part 2. --- .../QueryPlan/ReadFromMergeTree.cpp | 900 +++++++++++++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 34 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 265 ++++++ .../MergeTree/MergeTreeDataSelectExecutor.h | 16 +- 4 files changed, 1157 insertions(+), 58 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 771f95cc8fd..67377a54c34 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -2,15 +2,34 @@ #include #include #include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include #include #include +#include +#include #include #include +namespace ProfileEvents +{ + extern const Event SelectedParts; + extern const Event SelectedRanges; + extern const Event SelectedMarks; +} + namespace DB { @@ -19,6 +38,55 @@ namespace ErrorCodes extern const int INDEX_NOT_USED; } +namespace +{ + +/// Marks are placed whenever threshold on rows or bytes is met. +/// So we have to return the number of marks on whatever estimate is higher - by rows or by bytes. +size_t roundRowsOrBytesToMarks( + size_t rows_setting, + size_t bytes_setting, + size_t rows_granularity, + size_t bytes_granularity) +{ + size_t res = (rows_setting + rows_granularity - 1) / rows_granularity; + + if (bytes_granularity == 0) + return res; + else + return std::max(res, (bytes_setting + bytes_granularity - 1) / bytes_granularity); +} +/// Same as roundRowsOrBytesToMarks() but do not return more then max_marks +size_t minMarksForConcurrentRead( + size_t rows_setting, + size_t bytes_setting, + size_t rows_granularity, + size_t bytes_granularity, + size_t max_marks) +{ + size_t marks = 1; + + if (rows_setting + rows_granularity <= rows_setting) /// overflow + marks = max_marks; + else if (rows_setting) + marks = (rows_setting + rows_granularity - 1) / rows_granularity; + + if (bytes_granularity == 0) + return marks; + else + { + /// Overflow + if (bytes_setting + bytes_granularity <= bytes_setting) /// overflow + return max_marks; + if (bytes_setting) + return std::max(marks, (bytes_setting + bytes_granularity - 1) / bytes_granularity); + else + return marks; + } +} + +} + ReadFromMergeTree::ReadFromMergeTree( SelectQueryInfo query_info_, const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, @@ -26,15 +94,14 @@ ReadFromMergeTree::ReadFromMergeTree( const MergeTreeData & data_, const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, - String query_id_, + StorageMetadataPtr metadata_snapshot_base_, Names real_column_names_, MergeTreeData::DataPartsVector parts_, //IndexStatPtr index_stats_, PrewhereInfoPtr prewhere_info_, Names virt_column_names_, Settings settings_, - size_t num_streams_, - ReadType read_type_) + size_t num_streams_) : ISourceStep(DataStream{.header = MergeTreeBaseSelectProcessor::transformHeader( metadata_snapshot_->getSampleBlockForColumns(real_column_names_, storage_.getVirtuals(), storage_.getStorageID()), prewhere_info_, @@ -46,35 +113,35 @@ ReadFromMergeTree::ReadFromMergeTree( , data(data_) , storage(storage_) , metadata_snapshot(std::move(metadata_snapshot_)) - , query_id(std::move(query_id_)) + , metadata_snapshot_base(std::move(metadata_snapshot_base_)) , real_column_names(std::move(real_column_names_)) , parts(std::move(parts_)) - //, index_stats(std::move(index_stats_)) , prewhere_info(std::move(prewhere_info_)) , virt_column_names(std::move(virt_column_names_)) , settings(std::move(settings_)) , num_streams(num_streams_) - , read_type(read_type_) { } -Pipe ReadFromMergeTree::readFromPool() +Pipe ReadFromMergeTree::readFromPool( + RangesInDataParts parts_with_range, Names required_columns, + size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) { Pipes pipes; size_t sum_marks = 0; size_t total_rows = 0; - for (const auto & part : parts) + for (const auto & part : parts_with_range) { sum_marks += part.getMarksCount(); total_rows += part.getRowsCount(); } auto pool = std::make_shared( - num_streams, + used_max_streams, sum_marks, - settings.min_marks_for_concurrent_read, - std::move(parts), + min_marks_for_concurrent_read, + std::move(parts_with_range), storage, metadata_snapshot, prewhere_info, @@ -85,14 +152,14 @@ Pipe ReadFromMergeTree::readFromPool() false); auto * logger = &Poco::Logger::get(storage.getLogName() + " (SelectExecutor)"); - LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, num_streams); + LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, used_max_streams); - for (size_t i = 0; i < num_streams; ++i) + for (size_t i = 0; i < used_max_streams; ++i) { auto source = std::make_shared( - i, pool, settings.min_marks_for_concurrent_read, settings.max_block_size, + i, pool, min_marks_for_concurrent_read, settings.max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, - storage, metadata_snapshot, settings.use_uncompressed_cache, + storage, metadata_snapshot, use_uncompressed_cache, prewhere_info, settings.reader_settings, virt_column_names); if (i == 0) @@ -108,22 +175,22 @@ Pipe ReadFromMergeTree::readFromPool() } template -ProcessorPtr ReadFromMergeTree::createSource(const RangesInDataPart & part) +ProcessorPtr ReadFromMergeTree::createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache) { return std::make_shared( storage, metadata_snapshot, part.data_part, settings.max_block_size, settings.preferred_block_size_bytes, - settings.preferred_max_column_in_block_size_bytes, required_columns, part.ranges, settings.use_uncompressed_cache, + settings.preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info, true, settings.reader_settings, virt_column_names, part.part_index_in_query); } -Pipe ReadFromMergeTree::readInOrder() +Pipe ReadFromMergeTree::readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache) { Pipes pipes; - for (const auto & part : parts) + for (const auto & part : parts_with_range) { auto source = read_type == ReadType::InReverseOrder - ? createSource(part) - : createSource(part); + ? createSource(part, required_columns, use_uncompressed_cache) + : createSource(part, required_columns, use_uncompressed_cache); pipes.emplace_back(std::move(source)); } @@ -141,12 +208,14 @@ Pipe ReadFromMergeTree::readInOrder() return pipe; } -Pipe ReadFromMergeTree::read() +Pipe ReadFromMergeTree::read( + RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, + size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) { if (read_type == ReadType::Default && num_streams > 1) - return readFromPool(); + return readFromPool(parts_with_range, required_columns, used_max_streams, min_marks_for_concurrent_read, use_uncompressed_cache); - auto pipe = readInOrder(); + auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache); /// Use ConcatProcessor to concat sources together. /// It is needed to read in parts order (and so in PK order) if single thread is used. @@ -276,8 +345,629 @@ static void filterPartsByPartition( } } +Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( + RangesInDataParts && parts_with_ranges, + const Names & column_names) +{ + const auto & q_settings = context->getSettingsRef(); + + /// Count marks for each part. + std::vector sum_marks_in_parts(parts_with_ranges.size()); + size_t sum_marks = 0; + size_t total_rows = 0; + + const auto data_settings = data.getSettings(); + size_t adaptive_parts = 0; + for (size_t i = 0; i < parts_with_ranges.size(); ++i) + { + total_rows += parts_with_ranges[i].getRowsCount(); + sum_marks_in_parts[i] = parts_with_ranges[i].getMarksCount(); + sum_marks += sum_marks_in_parts[i]; + + if (parts_with_ranges[i].data_part->index_granularity_info.is_adaptive) + ++adaptive_parts; + } + + size_t index_granularity_bytes = 0; + if (adaptive_parts > parts_with_ranges.size() / 2) + index_granularity_bytes = data_settings->index_granularity_bytes; + + const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + q_settings.merge_tree_max_rows_to_use_cache, + q_settings.merge_tree_max_bytes_to_use_cache, + data_settings->index_granularity, + index_granularity_bytes); + + const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + q_settings.merge_tree_min_rows_for_concurrent_read, + q_settings.merge_tree_min_bytes_for_concurrent_read, + data_settings->index_granularity, + index_granularity_bytes, + sum_marks); + + bool use_uncompressed_cache = q_settings.use_uncompressed_cache; + if (sum_marks > max_marks_to_use_cache) + use_uncompressed_cache = false; + + if (0 == sum_marks) + return {}; + + size_t used_num_streams = num_streams; + if (used_num_streams > 1) + { + /// Reduce the number of num_streams if the data is small. + if (sum_marks < used_num_streams * min_marks_for_concurrent_read && parts_with_ranges.size() < used_num_streams) + used_num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts_with_ranges.size()); + } + + return read(std::move(parts_with_ranges), column_names, ReadType::Default, + used_num_streams, min_marks_for_concurrent_read, use_uncompressed_cache); +} + +static ActionsDAGPtr createProjection(const Block & header) +{ + auto projection = std::make_shared(header.getNamesAndTypesList()); + projection->removeUnusedActions(header.getNames()); + projection->projectInput(); + return projection; +} + +Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( + RangesInDataParts && parts_with_ranges, + const Names & column_names, + const ActionsDAGPtr & sorting_key_prefix_expr, + ActionsDAGPtr & out_projection, + const InputOrderInfoPtr & input_order_info) +{ + const auto & q_settings = context->getSettingsRef(); + size_t sum_marks = 0; + size_t adaptive_parts = 0; + std::vector sum_marks_in_parts(parts_with_ranges.size()); + const auto data_settings = data.getSettings(); + + for (size_t i = 0; i < parts_with_ranges.size(); ++i) + { + sum_marks_in_parts[i] = parts_with_ranges[i].getMarksCount(); + sum_marks += sum_marks_in_parts[i]; + + if (parts_with_ranges[i].data_part->index_granularity_info.is_adaptive) + ++adaptive_parts; + } + + size_t index_granularity_bytes = 0; + if (adaptive_parts > parts_with_ranges.size() / 2) + index_granularity_bytes = data_settings->index_granularity_bytes; + + const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + q_settings.merge_tree_max_rows_to_use_cache, + q_settings.merge_tree_max_bytes_to_use_cache, + data_settings->index_granularity, + index_granularity_bytes); + + const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + q_settings.merge_tree_min_rows_for_concurrent_read, + q_settings.merge_tree_min_bytes_for_concurrent_read, + data_settings->index_granularity, + index_granularity_bytes, + sum_marks); + + bool use_uncompressed_cache = settings.use_uncompressed_cache; + if (sum_marks > max_marks_to_use_cache) + use_uncompressed_cache = false; + + Pipes res; + + if (sum_marks == 0) + return {}; + + /// Let's split ranges to avoid reading much data. + auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size = settings.max_block_size] + (const auto & ranges, int direction) + { + MarkRanges new_ranges; + const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; + size_t marks_in_range = 1; + + if (direction == 1) + { + /// Split first few ranges to avoid reading much data. + bool split = false; + for (auto range : ranges) + { + while (!split && range.begin + marks_in_range < range.end) + { + new_ranges.emplace_back(range.begin, range.begin + marks_in_range); + range.begin += marks_in_range; + marks_in_range *= 2; + + if (marks_in_range > max_marks_in_range) + split = true; + } + new_ranges.emplace_back(range.begin, range.end); + } + } + else + { + /// Split all ranges to avoid reading much data, because we have to + /// store whole range in memory to reverse it. + for (auto it = ranges.rbegin(); it != ranges.rend(); ++it) + { + auto range = *it; + while (range.begin + marks_in_range < range.end) + { + new_ranges.emplace_front(range.end - marks_in_range, range.end); + range.end -= marks_in_range; + marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); + } + new_ranges.emplace_front(range.begin, range.end); + } + } + + return new_ranges; + }; + + const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; + bool need_preliminary_merge = (parts_with_ranges.size() > q_settings.read_in_order_two_level_merge_threshold); + + Pipes pipes; + + for (size_t i = 0; i < num_streams && !parts_with_ranges.empty(); ++i) + { + size_t need_marks = min_marks_per_stream; + RangesInDataParts new_parts; + + /// Loop over parts. + /// We will iteratively take part or some subrange of a part from the back + /// and assign a stream to read from it. + while (need_marks > 0 && !parts_with_ranges.empty()) + { + RangesInDataPart part = parts_with_ranges.back(); + parts_with_ranges.pop_back(); + + size_t & marks_in_part = sum_marks_in_parts.back(); + + /// We will not take too few rows from a part. + if (marks_in_part >= min_marks_for_concurrent_read && + need_marks < min_marks_for_concurrent_read) + need_marks = min_marks_for_concurrent_read; + + /// Do not leave too few rows in the part. + if (marks_in_part > need_marks && + marks_in_part - need_marks < min_marks_for_concurrent_read) + need_marks = marks_in_part; + + MarkRanges ranges_to_get_from_part; + + /// We take the whole part if it is small enough. + if (marks_in_part <= need_marks) + { + ranges_to_get_from_part = part.ranges; + + need_marks -= marks_in_part; + sum_marks_in_parts.pop_back(); + } + else + { + /// Loop through ranges in part. Take enough ranges to cover "need_marks". + while (need_marks > 0) + { + if (part.ranges.empty()) + throw Exception("Unexpected end of ranges while spreading marks among streams", ErrorCodes::LOGICAL_ERROR); + + MarkRange & range = part.ranges.front(); + + const size_t marks_in_range = range.end - range.begin; + const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); + + ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); + range.begin += marks_to_get_from_range; + marks_in_part -= marks_to_get_from_range; + need_marks -= marks_to_get_from_range; + if (range.begin == range.end) + part.ranges.pop_front(); + } + parts_with_ranges.emplace_back(part); + } + ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); + new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); + } + + auto read_type = input_order_info->direction == 1 + ? ReadFromMergeTree::ReadType::InOrder + : ReadFromMergeTree::ReadType::InReverseOrder; + + pipes.emplace_back(read(std::move(new_parts), column_names, read_type, + num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); + } + + if (need_preliminary_merge) + { + SortDescription sort_description; + for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) + sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], + input_order_info->direction, 1); + + auto sorting_key_expr = std::make_shared(sorting_key_prefix_expr); + + for (auto & pipe : pipes) + { + /// Drop temporary columns, added by 'sorting_key_prefix_expr' + out_projection = createProjection(pipe.getHeader()); + + pipe.addSimpleTransform([sorting_key_expr](const Block & header) + { + return std::make_shared(header, sorting_key_expr); + }); + + if (pipe.numOutputPorts() > 1) + { + + auto transform = std::make_shared( + pipe.getHeader(), + pipe.numOutputPorts(), + sort_description, + settings.max_block_size); + + pipe.addTransform(std::move(transform)); + } + } + } + + return Pipe::unitePipes(std::move(pipes)); +} + +static void addMergingFinal( + Pipe & pipe, + size_t num_output_streams, + const SortDescription & sort_description, + MergeTreeData::MergingParams merging_params, + Names partition_key_columns, + size_t max_block_size) +{ + const auto & header = pipe.getHeader(); + size_t num_outputs = pipe.numOutputPorts(); + + auto get_merging_processor = [&]() -> MergingTransformPtr + { + switch (merging_params.mode) + { + case MergeTreeData::MergingParams::Ordinary: + { + return std::make_shared(header, num_outputs, + sort_description, max_block_size); + } + + case MergeTreeData::MergingParams::Collapsing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.sign_column, true, max_block_size); + + case MergeTreeData::MergingParams::Summing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size); + + case MergeTreeData::MergingParams::Aggregating: + return std::make_shared(header, num_outputs, + sort_description, max_block_size); + + case MergeTreeData::MergingParams::Replacing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.version_column, max_block_size); + + case MergeTreeData::MergingParams::VersionedCollapsing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.sign_column, max_block_size); + + case MergeTreeData::MergingParams::Graphite: + throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); + } + + __builtin_unreachable(); + }; + + if (num_output_streams <= 1 || sort_description.empty()) + { + pipe.addTransform(get_merging_processor()); + return; + } + + ColumnNumbers key_columns; + key_columns.reserve(sort_description.size()); + + for (const auto & desc : sort_description) + { + if (!desc.column_name.empty()) + key_columns.push_back(header.getPositionByName(desc.column_name)); + else + key_columns.emplace_back(desc.column_number); + } + + pipe.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header, num_output_streams, key_columns); + }); + + pipe.transform([&](OutputPortRawPtrs ports) + { + Processors transforms; + std::vector output_ports; + transforms.reserve(ports.size() + num_output_streams); + output_ports.reserve(ports.size()); + + for (auto & port : ports) + { + auto copier = std::make_shared(header, num_output_streams); + connect(*port, copier->getInputPort()); + output_ports.emplace_back(copier->getOutputs().begin()); + transforms.emplace_back(std::move(copier)); + } + + for (size_t i = 0; i < num_output_streams; ++i) + { + auto merge = get_merging_processor(); + merge->setSelectorPosition(i); + auto input = merge->getInputs().begin(); + + /// Connect i-th merge with i-th input port of every copier. + for (size_t j = 0; j < ports.size(); ++j) + { + connect(*output_ports[j], *input); + ++output_ports[j]; + ++input; + } + + transforms.emplace_back(std::move(merge)); + } + + return transforms; + }); +} + + +Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( + RangesInDataParts && parts_with_range, + const Names & column_names, + ActionsDAGPtr & out_projection) +{ + const auto & q_settings = context->getSettingsRef(); + const auto data_settings = data.getSettings(); + size_t sum_marks = 0; + size_t adaptive_parts = 0; + for (const auto & part : parts_with_range) + { + for (const auto & range : part.ranges) + sum_marks += range.end - range.begin; + + if (part.data_part->index_granularity_info.is_adaptive) + ++adaptive_parts; + } + + size_t index_granularity_bytes = 0; + if (adaptive_parts >= parts_with_range.size() / 2) + index_granularity_bytes = data_settings->index_granularity_bytes; + + const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + q_settings.merge_tree_max_rows_to_use_cache, + q_settings.merge_tree_max_bytes_to_use_cache, + data_settings->index_granularity, + index_granularity_bytes); + + bool use_uncompressed_cache = settings.use_uncompressed_cache; + if (sum_marks > max_marks_to_use_cache) + use_uncompressed_cache = false; + + size_t used_num_streams = num_streams; + if (used_num_streams > q_settings.max_final_threads) + used_num_streams = q_settings.max_final_threads; + + /// If setting do_not_merge_across_partitions_select_final is true than we won't merge parts from different partitions. + /// We have all parts in parts vector, where parts with same partition are nearby. + /// So we will store iterators pointed to the beginning of each partition range (and parts.end()), + /// then we will create a pipe for each partition that will run selecting processor and merging processor + /// for the parts with this partition. In the end we will unite all the pipes. + std::vector parts_to_merge_ranges; + auto it = parts_with_range.begin(); + parts_to_merge_ranges.push_back(it); + + if (q_settings.do_not_merge_across_partitions_select_final) + { + while (it != parts_with_range.end()) + { + it = std::find_if( + it, parts_with_range.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); + parts_to_merge_ranges.push_back(it); + } + /// We divide threads for each partition equally. But we will create at least the number of partitions threads. + /// (So, the total number of threads could be more than initial num_streams. + used_num_streams /= (parts_to_merge_ranges.size() - 1); + } + else + { + /// If do_not_merge_across_partitions_select_final is false we just merge all the parts. + parts_to_merge_ranges.push_back(parts_with_range.end()); + } + + Pipes partition_pipes; + + /// If do_not_merge_across_partitions_select_final is true and num_streams > 1 + /// we will store lonely parts with level > 0 to use parallel select on them. + std::vector lonely_parts; + size_t total_rows_in_lonely_parts = 0; + size_t sum_marks_in_lonely_parts = 0; + + for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) + { + Pipe pipe; + + { + RangesInDataParts new_parts; + + /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition + /// with level > 0 then we won't postprocess this part and if num_streams > 1 we + /// can use parallel select on such parts. We save such parts in one vector and then use + /// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select. + if (used_num_streams > 1 && q_settings.do_not_merge_across_partitions_select_final && + std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && + parts_to_merge_ranges[range_index]->data_part->info.level > 0) + { + total_rows_in_lonely_parts += parts_to_merge_ranges[range_index]->getRowsCount(); + sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount(); + lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index])); + continue; + } + else + { + for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) + { + new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); + } + } + + if (new_parts.empty()) + continue; + + // ReadFromMergeTree::Settings step_settings + // { + // .max_block_size = max_block_size, + // .preferred_block_size_bytes = settings.preferred_block_size_bytes, + // .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, + // .min_marks_for_concurrent_read = 0, /// this setting is not used for reading in order + // .use_uncompressed_cache = use_uncompressed_cache, + // .reader_settings = reader_settings, + // .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), + // }; + + pipe = read(std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder, + used_num_streams, 0, use_uncompressed_cache); + + /// Drop temporary columns, added by 'sorting_key_expr' + if (!out_projection) + out_projection = createProjection(pipe.getHeader()); + } + + auto sorting_expr = std::make_shared( + metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + + pipe.addSimpleTransform([sorting_expr](const Block & header) + { + return std::make_shared(header, sorting_expr); + }); + + /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition + /// with level > 0 then we won't postprocess this part + if (q_settings.do_not_merge_across_partitions_select_final && + std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && + parts_to_merge_ranges[range_index]->data_part->info.level > 0) + { + partition_pipes.emplace_back(std::move(pipe)); + continue; + } + + Names sort_columns = metadata_snapshot->getSortingKeyColumns(); + SortDescription sort_description; + size_t sort_columns_size = sort_columns.size(); + sort_description.reserve(sort_columns_size); + + Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; + + const auto & header = pipe.getHeader(); + for (size_t i = 0; i < sort_columns_size; ++i) + sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); + + addMergingFinal( + pipe, + std::min(used_num_streams, q_settings.max_final_threads), + sort_description, data.merging_params, partition_key_columns, settings.max_block_size); + + // auto final_step = std::make_unique( + // plan->getCurrentDataStream(), + // std::min(used_num_streams, settings.max_final_threads), + // sort_description, + // data.merging_params, + // partition_key_columns, + // max_block_size); + + // final_step->setStepDescription("Merge rows for FINAL"); + // plan->addStep(std::move(final_step)); + + partition_pipes.emplace_back(std::move(pipe)); + } + + if (!lonely_parts.empty()) + { + RangesInDataParts new_parts; + + size_t num_streams_for_lonely_parts = used_num_streams * lonely_parts.size(); + + const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + q_settings.merge_tree_min_rows_for_concurrent_read, + q_settings.merge_tree_min_bytes_for_concurrent_read, + data_settings->index_granularity, + index_granularity_bytes, + sum_marks_in_lonely_parts); + + /// Reduce the number of num_streams_for_lonely_parts if the data is small. + if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) + num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); + + // ReadFromMergeTree::Settings step_settings + // { + // .max_block_size = max_block_size, + // .preferred_block_size_bytes = settings.preferred_block_size_bytes, + // .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, + // .min_marks_for_concurrent_read = min_marks_for_concurrent_read, + // .use_uncompressed_cache = use_uncompressed_cache, + // .reader_settings = reader_settings, + // .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), + // }; + + // auto plan = std::make_unique(); + // auto step = std::make_unique( + // data, + // metadata_snapshot, + // query_id, + // column_names, + // std::move(lonely_parts), + // // std::move(index_stats), + // query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, + // virt_columns, + // step_settings, + // num_streams_for_lonely_parts, + // ReadFromMergeTree::ReadType::Default); + + // plan->addStep(std::move(step)); + + auto pipe = read(std::move(lonely_parts), column_names, ReadFromMergeTree::ReadType::Default, + num_streams_for_lonely_parts, min_marks_for_concurrent_read, use_uncompressed_cache); + + /// Drop temporary columns, added by 'sorting_key_expr' + if (!out_projection) + out_projection = createProjection(pipe.getHeader()); + + auto sorting_expr = std::make_shared( + metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + + pipe.addSimpleTransform([sorting_expr](const Block & header) + { + return std::make_shared(header, sorting_expr); + }); + + // auto expression_step = std::make_unique( + // plan->getCurrentDataStream(), + // metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + + // expression_step->setStepDescription("Calculate sorting key expression"); + // plan->addStep(std::move(expression_step)); + + partition_pipes.emplace_back(std::move(pipe)); + } + + return Pipe::unitePipes(std::move(partition_pipes)); +} + void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) { + size_t total_parts = parts.size(); + auto part_values = filterPartsByVirtualColumns(data, parts, query_info.query, context); if (part_values && part_values->empty()) { @@ -320,9 +1010,127 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build sample_factor_column_queried = true; auto sampling = MergeTreeDataSelectExecutor::getSampling( - select, parts, metadata_snapshot, key_condition, settings, data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context) + select, parts, metadata_snapshot, key_condition, + data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); - Pipe pipe = read(); + if (sampling.read_nothing) + { + pipeline.init(Pipe(std::make_shared(getOutputStream().header))); + return; + } + + size_t total_marks_pk = 0; + for (const auto & part : parts) + total_marks_pk += part->index_granularity.getMarksCountWithoutFinal(); + + auto parts_with_ranges = MergeTreeDataSelectExecutor::filterParts( + parts, + metadata_snapshot, + query_info, + context, + key_condition, + settings.reader_settings, + log, + num_streams, + index_stats); + + size_t sum_marks_pk = total_marks_pk; + for (const auto & stat : index_stats) + if (stat.type == IndexType::PrimaryKey) + sum_marks_pk = stat.num_granules_after; + + size_t sum_marks = 0; + size_t sum_ranges = 0; + + for (const auto & part : parts_with_ranges) + { + sum_ranges += part.ranges.size(); + sum_marks += part.getMarksCount(); + } + + LOG_DEBUG( + log, + "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", + parts.size(), + total_parts, + parts_with_ranges.size(), + sum_marks_pk, + total_marks_pk, + sum_marks, + sum_ranges); + + String query_id = MergeTreeDataSelectExecutor::checkLimits(data, parts_with_ranges, context); + + ProfileEvents::increment(ProfileEvents::SelectedParts, parts_with_ranges.size()); + ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); + ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks); + + /// Projection, that needed to drop columns, which have appeared by execution + /// of some extra expressions, and to allow execute the same expressions later. + /// NOTE: It may lead to double computation of expressions. + ActionsDAGPtr result_projection; + + Names column_names_to_read = real_column_names; + if (!select.final() && sampling.use_sampling) + { + /// Add columns needed for `sample_by_ast` to `column_names_to_read`. + /// Skip this if final was used, because such columns were already added from PK. + std::vector add_columns = sampling.filter_expression->getRequiredColumns().getNames(); + column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); + std::sort(column_names_to_read.begin(), column_names_to_read.end()); + column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), + column_names_to_read.end()); + } + + const auto & input_order_info = query_info.input_order_info + ? query_info.input_order_info + : (query_info.projection ? query_info.projection->input_order_info : nullptr); + + Pipe pipe; + + const auto & q_settings = context->getSettingsRef(); + + if (select.final()) + { + /// Add columns needed to calculate the sorting expression and the sign. + std::vector add_columns = metadata_snapshot->getColumnsRequiredForSortingKey(); + column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); + + if (!data.merging_params.sign_column.empty()) + column_names_to_read.push_back(data.merging_params.sign_column); + if (!data.merging_params.version_column.empty()) + column_names_to_read.push_back(data.merging_params.version_column); + + std::sort(column_names_to_read.begin(), column_names_to_read.end()); + column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end()); + + pipe = spreadMarkRangesAmongStreamsFinal( + std::move(parts_with_ranges), + column_names_to_read, + result_projection); + } + else if ((q_settings.optimize_read_in_order || q_settings.optimize_aggregation_in_order) && input_order_info) + { + size_t prefix_size = input_order_info->order_key_prefix_descr.size(); + auto order_key_prefix_ast = metadata_snapshot->getSortingKey().expression_list_ast->clone(); + order_key_prefix_ast->children.resize(prefix_size); + + auto syntax_result = TreeRewriter(context).analyze(order_key_prefix_ast, metadata_snapshot->getColumns().getAllPhysical()); + auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActionsDAG(false); + + pipe = spreadMarkRangesAmongStreamsWithOrder( + std::move(parts_with_ranges), + column_names_to_read, + sorting_key_prefix_expr, + result_projection, + input_order_info); + } + else + { + pipe = spreadMarkRangesAmongStreams( + std::move(parts_with_ranges), + column_names_to_read); + } for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); @@ -371,40 +1179,40 @@ static const char * readTypeToString(ReadFromMergeTree::ReadType type) void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const { std::string prefix(format_settings.offset, format_settings.indent_char); - format_settings.out << prefix << "ReadType: " << readTypeToString(read_type) << '\n'; + //format_settings.out << prefix << "ReadType: " << readTypeToString(read_type) << '\n'; - if (index_stats && !index_stats->empty()) + if (!index_stats.empty()) { - format_settings.out << prefix << "Parts: " << index_stats->back().num_parts_after << '\n'; - format_settings.out << prefix << "Granules: " << index_stats->back().num_granules_after << '\n'; + format_settings.out << prefix << "Parts: " << index_stats.back().num_parts_after << '\n'; + format_settings.out << prefix << "Granules: " << index_stats.back().num_granules_after << '\n'; } } void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { - map.add("Read Type", readTypeToString(read_type)); - if (index_stats && !index_stats->empty()) + //map.add("Read Type", readTypeToString(read_type)); + if (!index_stats.empty()) { - map.add("Parts", index_stats->back().num_parts_after); - map.add("Granules", index_stats->back().num_granules_after); + map.add("Parts", index_stats.back().num_parts_after); + map.add("Granules", index_stats.back().num_granules_after); } } void ReadFromMergeTree::describeIndexes(FormatSettings & format_settings) const { std::string prefix(format_settings.offset, format_settings.indent_char); - if (index_stats && !index_stats->empty()) + if (!index_stats.empty()) { /// Do not print anything if no indexes is applied. - if (index_stats->size() == 1 && index_stats->front().type == IndexType::None) + if (index_stats.size() == 1 && index_stats.front().type == IndexType::None) return; std::string indent(format_settings.indent, format_settings.indent_char); format_settings.out << prefix << "Indexes:\n"; - for (size_t i = 0; i < index_stats->size(); ++i) + for (size_t i = 0; i < index_stats.size(); ++i) { - const auto & stat = (*index_stats)[i]; + const auto & stat = index_stats[i]; if (stat.type == IndexType::None) continue; @@ -428,12 +1236,12 @@ void ReadFromMergeTree::describeIndexes(FormatSettings & format_settings) const format_settings.out << prefix << indent << indent << "Parts: " << stat.num_parts_after; if (i) - format_settings.out << '/' << (*index_stats)[i - 1].num_parts_after; + format_settings.out << '/' << index_stats[i - 1].num_parts_after; format_settings.out << '\n'; format_settings.out << prefix << indent << indent << "Granules: " << stat.num_granules_after; if (i) - format_settings.out << '/' << (*index_stats)[i - 1].num_granules_after; + format_settings.out << '/' << index_stats[i - 1].num_granules_after; format_settings.out << '\n'; } } @@ -441,17 +1249,17 @@ void ReadFromMergeTree::describeIndexes(FormatSettings & format_settings) const void ReadFromMergeTree::describeIndexes(JSONBuilder::JSONMap & map) const { - if (index_stats && !index_stats->empty()) + if (!index_stats.empty()) { /// Do not print anything if no indexes is applied. - if (index_stats->size() == 1 && index_stats->front().type == IndexType::None) + if (index_stats.size() == 1 && index_stats.front().type == IndexType::None) return; auto indexes_array = std::make_unique(); - for (size_t i = 0; i < index_stats->size(); ++i) + for (size_t i = 0; i < index_stats.size(); ++i) { - const auto & stat = (*index_stats)[i]; + const auto & stat = index_stats[i]; if (stat.type == IndexType::None) continue; @@ -479,11 +1287,11 @@ void ReadFromMergeTree::describeIndexes(JSONBuilder::JSONMap & map) const index_map->add("Condition", stat.condition); if (i) - index_map->add("Initial Parts", (*index_stats)[i - 1].num_parts_after); + index_map->add("Initial Parts", index_stats[i - 1].num_parts_after); index_map->add("Selected Parts", stat.num_parts_after); if (i) - index_map->add("Initial Granules", (*index_stats)[i - 1].num_granules_after); + index_map->add("Initial Granules", index_stats[i - 1].num_granules_after); index_map->add("Selected Granules", stat.num_granules_after); indexes_array->add(std::move(index_map)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 762d7fea0e7..b3366afb118 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -74,7 +74,7 @@ public: const MergeTreeData & data_, const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, - String query_id_, + StorageMetadataPtr metadata_snapshot_base_, Names real_column_names_, MergeTreeData::DataPartsVector parts_, //IndexStatPtr index_stats_, @@ -82,7 +82,7 @@ public: Names virt_column_names_, Settings settings_, size_t num_streams_, - ReadType read_type_ + //ReadType read_type_ ); String getName() const override { return "ReadFromMergeTree"; } @@ -102,26 +102,42 @@ private: const MergeTreeData & data; const MergeTreeData & storage; StorageMetadataPtr metadata_snapshot; - String query_id; + StorageMetadataPtr metadata_snapshot_base; Names real_column_names; MergeTreeData::DataPartsVector parts; - IndexStat index_stats; + IndexStats index_stats; PrewhereInfoPtr prewhere_info; Names virt_column_names; Settings settings; size_t num_streams; - ReadType read_type; + //ReadType read_type; Poco::Logger * log; - Pipe read(); - Pipe readFromPool(); - Pipe readInOrder(); + Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); + Pipe readFromPool(RangesInDataParts parts_with_ranges, Names required_columns, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); + Pipe readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache); template - ProcessorPtr createSource(const RangesInDataPart & part); + ProcessorPtr createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache); + + Pipe spreadMarkRangesAmongStreams( + RangesInDataParts && parts_with_ranges, + const Names & column_names); + + Pipe spreadMarkRangesAmongStreamsWithOrder( + RangesInDataParts && parts_with_ranges, + const Names & column_names, + const ActionsDAGPtr & sorting_key_prefix_expr, + ActionsDAGPtr & out_projection, + const InputOrderInfoPtr & input_order_info); + + Pipe spreadMarkRangesAmongStreamsFinal( + RangesInDataParts && parts, + const Names & column_names, + ActionsDAGPtr & out_projection); }; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 444d074a4db..49bcf751bd1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -633,6 +633,271 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( return sampling; } +RangesInDataParts MergeTreeDataSelectExecutor::filterParts( + MergeTreeData::DataPartsVector & parts, + StorageMetadataPtr metadata_snapshot, + SelectQueryInfo & query_info, + ContextPtr & context, + KeyCondition & key_condition, + const MergeTreeReaderSettings & reader_settings, + Poco::Logger * log, + size_t num_streams, + ReadFromMergeTree::IndexStats & index_stats) +{ + RangesInDataParts parts_with_ranges(parts.size()); + const Settings & settings = context->getSettingsRef(); + + /// Let's start analyzing all useful indices + + struct DataSkippingIndexAndCondition + { + MergeTreeIndexPtr index; + MergeTreeIndexConditionPtr condition; + std::atomic total_granules{0}; + std::atomic granules_dropped{0}; + std::atomic total_parts{0}; + std::atomic parts_dropped{0}; + + DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_) + : index(index_), condition(condition_) + { + } + }; + std::list useful_indices; + + for (const auto & index : metadata_snapshot->getSecondaryIndices()) + { + auto index_helper = MergeTreeIndexFactory::instance().get(index); + auto condition = index_helper->createIndexCondition(query_info, context); + if (!condition->alwaysUnknownOrTrue()) + useful_indices.emplace_back(index_helper, condition); + } + + if (settings.force_data_skipping_indices.changed) + { + const auto & indices = settings.force_data_skipping_indices.toString(); + + Strings forced_indices; + { + Tokens tokens(&indices[0], &indices[indices.size()], settings.max_query_size); + IParser::Pos pos(tokens, settings.max_parser_depth); + Expected expected; + if (!parseIdentifiersOrStringLiterals(pos, expected, forced_indices)) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse force_data_skipping_indices ('{}')", indices); + } + + if (forced_indices.empty()) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "No indices parsed from force_data_skipping_indices ('{}')", indices); + + std::unordered_set useful_indices_names; + for (const auto & useful_index : useful_indices) + useful_indices_names.insert(useful_index.index->index.name); + + for (const auto & index_name : forced_indices) + { + if (!useful_indices_names.count(index_name)) + { + throw Exception( + ErrorCodes::INDEX_NOT_USED, + "Index {} is not used and setting 'force_data_skipping_indices' contains it", + backQuote(index_name)); + } + } + } + + std::atomic sum_marks_pk = 0; + std::atomic sum_parts_pk = 0; + + /// Let's find what range to read from each part. + { + std::atomic total_rows{0}; + + SizeLimits limits; + if (settings.read_overflow_mode == OverflowMode::THROW && settings.max_rows_to_read) + limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode); + + SizeLimits leaf_limits; + if (settings.read_overflow_mode_leaf == OverflowMode::THROW && settings.max_rows_to_read_leaf) + leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf); + + auto process_part = [&](size_t part_index) + { + auto & part = parts[part_index]; + + RangesInDataPart ranges(part, part_index); + + size_t total_marks_count = part->index_granularity.getMarksCountWithoutFinal(); + + if (metadata_snapshot->hasPrimaryKey()) + ranges.ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, settings, log); + else if (total_marks_count) + ranges.ranges = MarkRanges{MarkRange{0, total_marks_count}}; + + sum_marks_pk.fetch_add(ranges.getMarksCount(), std::memory_order_relaxed); + + if (!ranges.ranges.empty()) + sum_parts_pk.fetch_add(1, std::memory_order_relaxed); + + for (auto & index_and_condition : useful_indices) + { + if (ranges.ranges.empty()) + break; + + index_and_condition.total_parts.fetch_add(1, std::memory_order_relaxed); + + size_t total_granules = 0; + size_t granules_dropped = 0; + ranges.ranges = filterMarksUsingIndex( + index_and_condition.index, + index_and_condition.condition, + part, + ranges.ranges, + settings, + reader_settings, + total_granules, + granules_dropped, + log); + + index_and_condition.total_granules.fetch_add(total_granules, std::memory_order_relaxed); + index_and_condition.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); + + if (ranges.ranges.empty()) + index_and_condition.parts_dropped.fetch_add(1, std::memory_order_relaxed); + } + + if (!ranges.ranges.empty()) + { + if (limits.max_rows || leaf_limits.max_rows) + { + /// Fail fast if estimated number of rows to read exceeds the limit + auto current_rows_estimate = ranges.getRowsCount(); + size_t prev_total_rows_estimate = total_rows.fetch_add(current_rows_estimate); + size_t total_rows_estimate = current_rows_estimate + prev_total_rows_estimate; + limits.check(total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read' setting)", ErrorCodes::TOO_MANY_ROWS); + leaf_limits.check( + total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read_leaf' setting)", ErrorCodes::TOO_MANY_ROWS); + } + + parts_with_ranges[part_index] = std::move(ranges); + } + }; + + size_t num_threads = std::min(size_t(num_streams), parts.size()); + + if (num_threads <= 1) + { + for (size_t part_index = 0; part_index < parts.size(); ++part_index) + process_part(part_index); + } + else + { + /// Parallel loading of data parts. + ThreadPool pool(num_threads); + + for (size_t part_index = 0; part_index < parts.size(); ++part_index) + pool.scheduleOrThrowOnError([&, part_index, thread_group = CurrentThread::getGroup()] + { + SCOPE_EXIT_SAFE(if (thread_group) CurrentThread::detachQueryIfNotDetached();); + if (thread_group) + CurrentThread::attachTo(thread_group); + + process_part(part_index); + }); + + pool.wait(); + } + + /// Skip empty ranges. + size_t next_part = 0; + for (size_t part_index = 0; part_index < parts.size(); ++part_index) + { + auto & part = parts_with_ranges[part_index]; + if (!part.data_part) + continue; + + if (next_part != part_index) + std::swap(parts_with_ranges[next_part], part); + + ++next_part; + } + + parts_with_ranges.resize(next_part); + } + + if (metadata_snapshot->hasPrimaryKey()) + { + auto description = key_condition.getDescription(); + + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::PrimaryKey, + .condition = std::move(description.condition), + .used_keys = std::move(description.used_keys), + .num_parts_after = sum_parts_pk.load(std::memory_order_relaxed), + .num_granules_after = sum_marks_pk.load(std::memory_order_relaxed)}); + } + + for (const auto & index_and_condition : useful_indices) + { + const auto & index_name = index_and_condition.index->index.name; + LOG_DEBUG( + log, + "Index {} has dropped {}/{} granules.", + backQuote(index_name), + index_and_condition.granules_dropped, + index_and_condition.total_granules); + + std::string description + = index_and_condition.index->index.type + " GRANULARITY " + std::to_string(index_and_condition.index->index.granularity); + + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::Skip, + .name = index_name, + .description = std::move(description), + .num_parts_after = index_and_condition.total_parts - index_and_condition.parts_dropped, + .num_granules_after = index_and_condition.total_granules - index_and_condition.granules_dropped}); + } + + return parts_with_ranges; +} + +void checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_range, ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + // Check limitations. query_id is used as the quota RAII's resource key. + String query_id; + { + const auto data_settings = data.getSettings(); + auto max_partitions_to_read + = settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data_settings->max_partitions_to_read; + if (max_partitions_to_read > 0) + { + std::set partitions; + for (auto & part_with_ranges : parts_with_ranges) + partitions.insert(part_with_ranges.data_part->info.partition_id); + if (partitions.size() > size_t(max_partitions_to_read)) + throw Exception( + ErrorCodes::TOO_MANY_PARTITIONS, + "Too many partitions to read. Current {}, max {}", + partitions.size(), + max_partitions_to_read); + } + + if (data_settings->max_concurrent_queries > 0 && data_settings->min_marks_to_honor_max_concurrent_queries > 0) + { + size_t sum_marks = 0; + for (const auto & part : parts_with_ranges) + sum_marks += part.getMarksCount(); + + if (sum_marks >= data_settings->min_marks_to_honor_max_concurrent_queries) + { + query_id = context->getCurrentQueryId(); + if (!query_id.empty()) + data.insertQueryIdOrThrow(query_id, data_settings->max_concurrent_queries); + } + } + } +} + QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names_to_return, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 231908e2ee7..7597af2e173 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -72,7 +72,6 @@ private: QueryPlanPtr spreadMarkRangesAmongStreams( RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, size_t num_streams, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -87,7 +86,6 @@ private: /// out_projection - save projection only with columns, requested to read QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder( RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, size_t num_streams, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -104,7 +102,6 @@ private: QueryPlanPtr spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, size_t num_streams, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -178,6 +175,17 @@ public: PartFilterCounters & counters, Poco::Logger * log); + static RangesInDataParts filterParts( + MergeTreeData::DataPartsVector & parts, + StorageMetadataPtr metadata_snapshot, + SelectQueryInfo & query_info, + ContextPtr & context, + KeyCondition & key_condition, + const MergeTreeReaderSettings & reader_settings, + Poco::Logger * log, + size_t num_streams, + ReadFromMergeTree::IndexStats & index_stats); + static MergeTreeDataSelectSamplingData getSampling( const ASTSelectQuery & select, MergeTreeData::DataPartsVector & parts, @@ -188,6 +196,8 @@ public: bool sample_factor_column_queried, NamesAndTypesList available_real_columns, ContextPtr context); + + static String checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context); }; } From df400d1437e56fd0d8f9be89691ab6f0c8d874b6 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 26 May 2021 23:01:20 +0300 Subject: [PATCH 091/154] Fix links MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил ссылки. --- docs/en/engines/table-engines/special/dictionary.md | 2 +- .../external-dictionaries/external-dicts-dict-sources.md | 2 +- docs/ru/engines/table-engines/special/dictionary.md | 2 +- .../external-dictionaries/external-dicts-dict-sources.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/special/dictionary.md b/docs/en/engines/table-engines/special/dictionary.md index e143f3a8ee1..d76adebe01e 100644 --- a/docs/en/engines/table-engines/special/dictionary.md +++ b/docs/en/engines/table-engines/special/dictionary.md @@ -96,4 +96,4 @@ select * from products limit 1; **See Also** -- [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) +- [Dictionary function](../../../sql-reference/table-functions/dictionary.md#dictionary-function) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 3b9a28403f7..533ec6090ff 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -100,7 +100,7 @@ When dictionary with source `FILE` is created via DDL command (`CREATE DICTIONAR **See Also** -- [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) +- [Dictionary function](../../../sql-reference/table-functions/dictionary.md#dictionary-function) ## Executable File {#dicts-external_dicts_dict_sources-executable} diff --git a/docs/ru/engines/table-engines/special/dictionary.md b/docs/ru/engines/table-engines/special/dictionary.md index df47ade5a42..15d32419472 100644 --- a/docs/ru/engines/table-engines/special/dictionary.md +++ b/docs/ru/engines/table-engines/special/dictionary.md @@ -92,4 +92,4 @@ select * from products limit 1; **Смотрите также** -- [Функция dictionary](../../sql-reference/table-functions/dictionary.md#dictionary-function) +- [Функция dictionary](../../../sql-reference/table-functions/dictionary.md#dictionary-function) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index a9f3857f183..76290c72490 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -99,7 +99,7 @@ SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated')) **Смотрите также** -- [Функция dictionary](../../sql-reference/table-functions/dictionary.md#dictionary-function) +- [Функция dictionary](../../../sql-reference/table-functions/dictionary.md#dictionary-function) ## Исполняемый файл {#dicts-external_dicts_dict_sources-executable} From f9088b98de8b6880473fcac916a23fe46a70ae23 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 27 May 2021 00:02:24 +0300 Subject: [PATCH 092/154] Some disks update --- src/Disks/HDFS/DiskHDFS.cpp | 4 +- src/Disks/IDisk.h | 1 - src/Disks/IDiskRemote.cpp | 40 ++++++++++--------- src/Disks/IDiskRemote.h | 17 ++++---- src/Disks/ReadIndirectBufferFromRemoteFS.cpp | 1 + src/Disks/WriteIndirectBufferFromRemoteFS.cpp | 1 - src/Disks/WriteIndirectBufferFromRemoteFS.h | 2 +- 7 files changed, 34 insertions(+), 32 deletions(-) diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 0648fd9f08c..da6ccb024c7 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -172,8 +172,8 @@ void registerDiskHDFS(DiskFactory & factory) const String & config_prefix, ContextConstPtr context_) -> DiskPtr { - Poco::File disk{context_->getPath() + "disks/" + name}; - disk.createDirectories(); + fs::path disk = fs::path(context_->getPath()) / "disks" / name; + fs::create_directories(disk); String uri{config.getString(config_prefix + ".endpoint")}; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 1cbc038f034..cb718605dd6 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -7,7 +7,6 @@ #include #include #include -#include "Disks/Executor.h" #include #include diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index bcb399f5d07..f32b046b5ec 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -179,9 +178,9 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths { LOG_DEBUG(log, "Remove file by path: {}", backQuote(metadata_path + path)); - Poco::File file(metadata_path + path); + fs::path file(metadata_path + path); - if (!file.isFile()) + if (!fs::is_regular_file(file)) throw Exception(ErrorCodes::CANNOT_DELETE_DIRECTORY, "Path '{}' is a directory", path); try @@ -191,7 +190,7 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths /// If there is no references - delete content from remote FS. if (metadata.ref_count == 0) { - file.remove(); + fs::remove(file); for (const auto & [remote_fs_object_path, _] : metadata.remote_fs_objects) fs_paths_keeper->addPath(remote_fs_root_path + remote_fs_object_path); } @@ -199,7 +198,7 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths { --metadata.ref_count; metadata.save(); - file.remove(); + fs::remove(file); } } catch (const Exception & e) @@ -210,7 +209,7 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths LOG_WARNING(log, "Metadata file {} can't be read by reason: {}. Removing it forcibly.", backQuote(path), e.nested() ? e.nested()->message() : e.message()); - file.remove(); + fs::remove(file); } else throw; @@ -222,8 +221,8 @@ void IDiskRemote::removeMetaRecursive(const String & path, RemoteFSPathKeeperPtr { checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. - Poco::File file(metadata_path + path); - if (file.isFile()) + fs::path file = fs::path(metadata_path) / path; + if (fs::is_regular_file(file)) { removeMeta(path, fs_paths_keeper); } @@ -231,7 +230,7 @@ void IDiskRemote::removeMetaRecursive(const String & path, RemoteFSPathKeeperPtr { for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) removeMetaRecursive(it->path(), fs_paths_keeper); - file.remove(); + fs::remove(file); } } @@ -296,13 +295,13 @@ IDiskRemote::IDiskRemote( bool IDiskRemote::exists(const String & path) const { - return Poco::File(metadata_path + path).exists(); + return fs::exists(fs::path(metadata_path) / path); } bool IDiskRemote::isFile(const String & path) const { - return Poco::File(metadata_path + path).isFile(); + return fs::is_regular_file(fs::path(metadata_path) / path); } @@ -326,7 +325,7 @@ void IDiskRemote::moveFile(const String & from_path, const String & to_path) if (exists(to_path)) throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS); - Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path); + fs::rename(fs::path(metadata_path) / from_path, fs::path(metadata_path) / to_path); } @@ -347,7 +346,7 @@ void IDiskRemote::replaceFile(const String & from_path, const String & to_path) void IDiskRemote::removeFileIfExists(const String & path) { RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper(); - if (Poco::File(metadata_path + path).exists()) + if (fs::exists(fs::path(metadata_path) / path)) { removeMeta(path, fs_paths_keeper); removeFromRemoteFS(fs_paths_keeper); @@ -385,19 +384,19 @@ void IDiskRemote::setReadOnly(const String & path) bool IDiskRemote::isDirectory(const String & path) const { - return Poco::File(metadata_path + path).isDirectory(); + return fs::is_directory(fs::path(metadata_path) / path); } void IDiskRemote::createDirectory(const String & path) { - Poco::File(metadata_path + path).createDirectory(); + fs::create_directory(fs::path(metadata_path) / path); } void IDiskRemote::createDirectories(const String & path) { - Poco::File(metadata_path + path).createDirectories(); + fs::create_directories(fs::path(metadata_path) / path); } @@ -411,7 +410,7 @@ void IDiskRemote::clearDirectory(const String & path) void IDiskRemote::removeDirectory(const String & path) { - Poco::File(metadata_path + path).remove(); + fs::remove(fs::path(metadata_path) / path); } @@ -430,13 +429,16 @@ void IDiskRemote::listFiles(const String & path, std::vector & file_name void IDiskRemote::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - Poco::File(metadata_path + path).setLastModified(timestamp); + fs::last_write_time(fs::path(metadata_path) / path, + static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); } Poco::Timestamp IDiskRemote::getLastModified(const String & path) { - return Poco::File(metadata_path + path).getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(fs::path(metadata_path) / path); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index b32258331a7..a90621443e4 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -4,11 +4,12 @@ #include #include "Disks/DiskFactory.h" #include "Disks/Executor.h" -#include #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -193,21 +194,21 @@ public: void next() override { ++iter; } - bool isValid() const override { return iter != Poco::DirectoryIterator(); } + bool isValid() const override { return iter != fs::directory_iterator(); } String path() const override { - if (iter->isDirectory()) - return folder_path + iter.name() + '/'; + if (fs::is_directory(iter->path())) + return folder_path / iter->path().filename().string() / ""; else - return folder_path + iter.name(); + return folder_path / iter->path().filename().string(); } - String name() const override { return iter.name(); } + String name() const override { return iter->path().filename(); } private: - Poco::DirectoryIterator iter; - String folder_path; + fs::directory_iterator iter; + fs::path folder_path; }; diff --git a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp index 955986e5259..6d4764e4392 100644 --- a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp @@ -21,6 +21,7 @@ ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS( { } + template off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) { diff --git a/src/Disks/WriteIndirectBufferFromRemoteFS.cpp b/src/Disks/WriteIndirectBufferFromRemoteFS.cpp index adc711608d7..6951b9fa92e 100644 --- a/src/Disks/WriteIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/WriteIndirectBufferFromRemoteFS.cpp @@ -8,7 +8,6 @@ namespace DB { -/// Stores data in S3/HDFS and adds the object key (S3 path) and object size to metadata file on local FS. template WriteIndirectBufferFromRemoteFS::WriteIndirectBufferFromRemoteFS( std::unique_ptr impl_, diff --git a/src/Disks/WriteIndirectBufferFromRemoteFS.h b/src/Disks/WriteIndirectBufferFromRemoteFS.h index cda7523e19e..ece7b9d5871 100644 --- a/src/Disks/WriteIndirectBufferFromRemoteFS.h +++ b/src/Disks/WriteIndirectBufferFromRemoteFS.h @@ -10,7 +10,7 @@ namespace DB { -/// Stores data in S3/HDFS and adds the object key (S3 path) and object size to metadata file on local FS. +/// Stores data in S3/HDFS and adds the object path and object size to metadata file on local FS. template class WriteIndirectBufferFromRemoteFS final : public WriteBufferFromFileDecorator { From 47e8a53d2a632faa09689f38917f96c3a3c4883a Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 27 May 2021 12:55:15 +0300 Subject: [PATCH 093/154] Fix flaky test --- .../0_stateless/01881_total_bytes_storage_buffer.sql | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql index d962a6db456..1fd9ea56e8f 100644 --- a/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql +++ b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql @@ -6,18 +6,18 @@ CREATE TABLE test_buffer_table ) ENGINE = Buffer('', '', 1, 1000000, 1000000, 1000000, 1000000, 1000000, 1000000); -SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); INSERT INTO test_buffer_table SELECT number FROM numbers(1000); -SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); OPTIMIZE TABLE test_buffer_table; -SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); INSERT INTO test_buffer_table SELECT number FROM numbers(1000); -SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); OPTIMIZE TABLE test_buffer_table; -SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); DROP TABLE test_buffer_table; From f66c67a979cc303f9db46e9b9dae8ae450ac2d18 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 27 May 2021 10:35:36 +0300 Subject: [PATCH 094/154] Fixes --- base/daemon/BaseDaemon.cpp | 12 +++++++----- programs/server/Server.cpp | 12 ++++++------ src/Common/ErrorCodes.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 6 ++++-- src/Formats/FormatSchemaInfo.cpp | 11 +++++++++-- 5 files changed, 27 insertions(+), 16 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index a2118cc7bb1..310e656dac2 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -531,12 +531,14 @@ void BaseDaemon::closeFDs() #endif if (fs::is_directory(proc_path)) /// Hooray, proc exists { - std::vector fds; - /// in /proc/self/fd directory filenames are numeric file descriptors - Poco::File(proc_path.string()).list(fds); - for (const auto & fd_str : fds) + /// in /proc/self/fd directory filenames are numeric file descriptors. + /// Iterate directory separately from closing fds to avoid closing iterated directory fd. + std::vector fds; + for (const auto & path : fs::directory_iterator(proc_path)) + fds.push_back(DB::parse(path.path().filename())); + + for (const auto & fd : fds) { - int fd = DB::parse(fd_str); if (fd > 2 && fd != signal_pipe.fds_rw[0] && fd != signal_pipe.fds_rw[1]) ::close(fd); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 820bb73c6ba..f169d793ee9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -189,16 +189,16 @@ void setupTmpPath(Poco::Logger * log, const std::string & path) fs::create_directories(path); /// Clearing old temporary files. - Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator it(path); it != dir_end; ++it) + fs::directory_iterator dir_end; + for (fs::directory_iterator it(path); it != dir_end; ++it) { - if (it->isFile() && startsWith(it.name(), "tmp")) + if (it->is_regular_file() && startsWith(it->path().filename(), "tmp")) { - LOG_DEBUG(log, "Removing old temporary file {}", it->path()); - it->remove(); + LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); + fs::remove(it->path()); } else - LOG_DEBUG(log, "Skipped file in temporary path {}", it->path()); + LOG_DEBUG(log, "Skipped file in temporary path {}", it->path().string()); } } diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 34e34418acb..d840830bf28 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -553,8 +553,8 @@ M(583, ILLEGAL_PROJECTION) \ M(584, PROJECTION_NOT_USED) \ M(585, CANNOT_PARSE_YAML) \ + M(586, CANNOT_CREATE_FILE) \ \ - M(997, CANNOT_CREATE_FILE) \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 7054987527d..89c2d20db9f 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -874,9 +874,11 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); fs::path from_path = fs::path(metadata_path) / path; - fs::path to_path = fs::path(metadata_path) / detached_path / from_path.filename(); + fs::path to_path = fs::path(metadata_path) / detached_path; if (path.ends_with('/')) - to_path = to_path.parent_path(); + to_path /= from_path.parent_path().filename(); + else + to_path /= from_path.filename(); fs::copy(from_path, to_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); fs::remove_all(from_path); } diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index f985cb2681f..54e0961d161 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -74,6 +74,14 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & if (!path.has_extension() && !default_file_extension.empty()) path = path.parent_path() / (path.stem().string() + '.' + default_file_extension); + fs::path default_schema_directory_path(default_schema_directory()); + auto path_is_subdirectory_of = [](fs::path inner, const fs::path & outer) -> bool + { + while (inner != outer && inner != "/") + inner = inner.parent_path(); + return inner == outer; + }; + if (path.is_absolute()) { if (is_server) @@ -81,13 +89,12 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (path.string().starts_with("..")) + else if (!path_is_subdirectory_of(path, default_schema_directory_path)) { if (is_server) throw Exception( "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(), ErrorCodes::BAD_ARGUMENTS); - fs::path default_schema_directory_path(default_schema_directory()); if (default_schema_directory_path.is_absolute()) path = default_schema_directory_path; else From cbdf3752efc2bebdd1f207e2d2363c9a4d9fa661 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 27 May 2021 16:40:33 +0300 Subject: [PATCH 095/154] Part 3. --- .../QueryPlan/ReadFromMergeTree.cpp | 261 +- src/Processors/QueryPlan/ReadFromMergeTree.h | 26 +- src/Storages/MergeTree/MergeTreeData.cpp | 53 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2367 +++++++---------- .../MergeTree/MergeTreeDataSelectExecutor.h | 146 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 5 +- src/Storages/SelectQueryInfo.h | 4 +- 7 files changed, 1116 insertions(+), 1746 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 67377a54c34..6d6ee43acb3 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -3,6 +3,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -11,8 +14,6 @@ #include #include #include -#include -#include #include #include #include @@ -88,38 +89,35 @@ size_t minMarksForConcurrentRead( } ReadFromMergeTree::ReadFromMergeTree( - SelectQueryInfo query_info_, - const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, + const SelectQueryInfo & query_info_, + const PartitionIdToMaxBlock * max_block_numbers_to_read_, ContextPtr context_, const MergeTreeData & data_, - const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, StorageMetadataPtr metadata_snapshot_base_, Names real_column_names_, MergeTreeData::DataPartsVector parts_, - //IndexStatPtr index_stats_, PrewhereInfoPtr prewhere_info_, Names virt_column_names_, Settings settings_, - size_t num_streams_) + Poco::Logger * log_) : ISourceStep(DataStream{.header = MergeTreeBaseSelectProcessor::transformHeader( - metadata_snapshot_->getSampleBlockForColumns(real_column_names_, storage_.getVirtuals(), storage_.getStorageID()), + metadata_snapshot_->getSampleBlockForColumns(real_column_names_, data_.getVirtuals(), data_.getStorageID()), prewhere_info_, - storage_.getPartitionValueType(), + data_.getPartitionValueType(), virt_column_names_)}) , query_info(std::move(query_info_)) , max_block_numbers_to_read(max_block_numbers_to_read_) , context(std::move(context_)) , data(data_) - , storage(storage_) , metadata_snapshot(std::move(metadata_snapshot_)) , metadata_snapshot_base(std::move(metadata_snapshot_base_)) , real_column_names(std::move(real_column_names_)) - , parts(std::move(parts_)) + , prepared_parts(std::move(parts_)) , prewhere_info(std::move(prewhere_info_)) , virt_column_names(std::move(virt_column_names_)) , settings(std::move(settings_)) - , num_streams(num_streams_) + , log(log_) { } @@ -142,7 +140,7 @@ Pipe ReadFromMergeTree::readFromPool( sum_marks, min_marks_for_concurrent_read, std::move(parts_with_range), - storage, + data, metadata_snapshot, prewhere_info, true, @@ -151,7 +149,7 @@ Pipe ReadFromMergeTree::readFromPool( settings.preferred_block_size_bytes, false); - auto * logger = &Poco::Logger::get(storage.getLogName() + " (SelectExecutor)"); + auto * logger = &Poco::Logger::get(data.getLogName() + " (SelectExecutor)"); LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, used_max_streams); for (size_t i = 0; i < used_max_streams; ++i) @@ -159,7 +157,7 @@ Pipe ReadFromMergeTree::readFromPool( auto source = std::make_shared( i, pool, min_marks_for_concurrent_read, settings.max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, - storage, metadata_snapshot, use_uncompressed_cache, + data, metadata_snapshot, use_uncompressed_cache, prewhere_info, settings.reader_settings, virt_column_names); if (i == 0) @@ -178,7 +176,7 @@ template ProcessorPtr ReadFromMergeTree::createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache) { return std::make_shared( - storage, metadata_snapshot, part.data_part, settings.max_block_size, settings.preferred_block_size_bytes, + data, metadata_snapshot, part.data_part, settings.max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info, true, settings.reader_settings, virt_column_names, part.part_index_in_query); } @@ -212,7 +210,7 @@ Pipe ReadFromMergeTree::read( RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) { - if (read_type == ReadType::Default && num_streams > 1) + if (read_type == ReadType::Default && used_max_streams > 1) return readFromPool(parts_with_range, required_columns, used_max_streams, min_marks_for_concurrent_read, use_uncompressed_cache); auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache); @@ -225,126 +223,6 @@ Pipe ReadFromMergeTree::read( return pipe; } -static std::optional> filterPartsByVirtualColumns( - const MergeTreeData & data, - MergeTreeData::DataPartsVector & parts, - ASTPtr & query, - ContextPtr context) -{ - std::unordered_set part_values; - ASTPtr expression_ast; - auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); - - // Generate valid expressions for filtering - VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast); - - // If there is still something left, fill the virtual block and do the filtering. - if (expression_ast) - { - virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */); - VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context, expression_ast); - return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); - } - - return {}; -} - -static void filterPartsByPartition( - StorageMetadataPtr & metadata_snapshot, - const MergeTreeData & data, - SelectQueryInfo & query_info, - ContextPtr & context, - ContextPtr & query_context, - MergeTreeData::DataPartsVector & parts, - const std::optional> & part_values, - const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read, - Poco::Logger * log, - ReadFromMergeTree::IndexStats & index_stats) -{ - const Settings & settings = context->getSettingsRef(); - std::optional partition_pruner; - std::optional minmax_idx_condition; - DataTypes minmax_columns_types; - if (metadata_snapshot->hasPartitionKey()) - { - const auto & partition_key = metadata_snapshot->getPartitionKey(); - auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); - minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); - - minmax_idx_condition.emplace( - query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context))); - partition_pruner.emplace(metadata_snapshot->getPartitionKey(), query_info, context, false /* strict */); - - if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) - { - String msg = "Neither MinMax index by columns ("; - bool first = true; - for (const String & col : minmax_columns_names) - { - if (first) - first = false; - else - msg += ", "; - msg += col; - } - msg += ") nor partition expr is used and setting 'force_index_by_date' is set"; - - throw Exception(msg, ErrorCodes::INDEX_NOT_USED); - } - } - - MergeTreeDataSelectExecutor::PartFilterCounters part_filter_counters; - if (query_context->getSettingsRef().allow_experimental_query_deduplication) - MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( - parts, - part_values, - data.getPinnedPartUUIDs(), - minmax_idx_condition, - minmax_columns_types, - partition_pruner, - max_block_numbers_to_read, - query_context, - part_filter_counters, - log); - else - MergeTreeDataSelectExecutor::selectPartsToRead( - parts, - part_values, - minmax_idx_condition, - minmax_columns_types, - partition_pruner, - max_block_numbers_to_read, - part_filter_counters); - - index_stats.emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::None, - .num_parts_after = part_filter_counters.num_initial_selected_parts, - .num_granules_after = part_filter_counters.num_initial_selected_granules}); - - if (minmax_idx_condition) - { - auto description = minmax_idx_condition->getDescription(); - index_stats.emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::MinMax, - .condition = std::move(description.condition), - .used_keys = std::move(description.used_keys), - .num_parts_after = part_filter_counters.num_parts_after_minmax, - .num_granules_after = part_filter_counters.num_granules_after_minmax}); - LOG_DEBUG(log, "MinMax index condition: {}", minmax_idx_condition->toString()); - } - - if (partition_pruner) - { - auto description = partition_pruner->getKeyCondition().getDescription(); - index_stats.emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::Partition, - .condition = std::move(description.condition), - .used_keys = std::move(description.used_keys), - .num_parts_after = part_filter_counters.num_parts_after_partition_pruner, - .num_granules_after = part_filter_counters.num_granules_after_partition_pruner}); - } -} - Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( RangesInDataParts && parts_with_ranges, const Names & column_names) @@ -392,7 +270,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( if (0 == sum_marks) return {}; - size_t used_num_streams = num_streams; + size_t used_num_streams = settings.num_streams; if (used_num_streams > 1) { /// Reduce the number of num_streams if the data is small. @@ -506,12 +384,12 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( return new_ranges; }; - const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; + const size_t min_marks_per_stream = (sum_marks - 1) / settings.num_streams + 1; bool need_preliminary_merge = (parts_with_ranges.size() > q_settings.read_in_order_two_level_merge_threshold); Pipes pipes; - for (size_t i = 0; i < num_streams && !parts_with_ranges.empty(); ++i) + for (size_t i = 0; i < settings.num_streams && !parts_with_ranges.empty(); ++i) { size_t need_marks = min_marks_per_stream; RangesInDataParts new_parts; @@ -577,7 +455,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( : ReadFromMergeTree::ReadType::InReverseOrder; pipes.emplace_back(read(std::move(new_parts), column_names, read_type, - num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); + settings.num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); } if (need_preliminary_merge) @@ -755,7 +633,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; - size_t used_num_streams = num_streams; + size_t used_num_streams = settings.num_streams; if (used_num_streams > q_settings.max_final_threads) used_num_streams = q_settings.max_final_threads; @@ -966,9 +844,10 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) { + auto parts = std::move(prepared_parts); size_t total_parts = parts.size(); - auto part_values = filterPartsByVirtualColumns(data, parts, query_info.query, context); + auto part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context); if (part_values && part_values->empty()) { pipeline.init(Pipe(std::make_shared(getOutputStream().header))); @@ -1001,7 +880,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build const auto & select = query_info.query->as(); auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; - filterPartsByPartition( + MergeTreeDataSelectExecutor::filterPartsByPartition( metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read, log, index_stats); bool sample_factor_column_queried = false; @@ -1023,16 +902,17 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build for (const auto & part : parts) total_marks_pk += part->index_granularity.getMarksCountWithoutFinal(); - auto parts_with_ranges = MergeTreeDataSelectExecutor::filterParts( - parts, + auto parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( + std::move(parts), metadata_snapshot, query_info, context, key_condition, settings.reader_settings, log, - num_streams, - index_stats); + settings.num_streams, + index_stats, + true); size_t sum_marks_pk = total_marks_pk; for (const auto & stat : index_stats) @@ -1132,12 +1012,69 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build column_names_to_read); } + if (pipe.empty()) + { + pipeline.init(Pipe(std::make_shared(getOutputStream().header))); + return; + } + + if (sampling.use_sampling) + { + auto sampling_actions = std::make_shared(sampling.filter_expression); + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + sampling_actions, + sampling.filter_function->getColumnName(), + false); + }); + } + + if (result_projection) + { + auto projection_actions = std::make_shared(result_projection); + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, projection_actions); + }); + } + + /// 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(sampling.used_sample_factor)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto adding_column_action = std::make_shared(adding_column_dag); + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, adding_column_action); + }); + } + + // TODO There seems to be no place initializing remove_columns_actions + if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) + { + auto remove_columns_action = std::make_shared( + query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, remove_columns_action); + }); + } + for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); // Attach QueryIdHolder if needed if (!query_id.empty()) - pipe.addQueryIdHolder(std::make_shared(query_id, storage)); + pipe.addQueryIdHolder(std::make_shared(query_id, data)); pipeline.init(std::move(pipe)); } @@ -1161,20 +1098,20 @@ static const char * indexTypeToString(ReadFromMergeTree::IndexType type) __builtin_unreachable(); } -static const char * readTypeToString(ReadFromMergeTree::ReadType type) -{ - switch (type) - { - case ReadFromMergeTree::ReadType::Default: - return "Default"; - case ReadFromMergeTree::ReadType::InOrder: - return "InOrder"; - case ReadFromMergeTree::ReadType::InReverseOrder: - return "InReverseOrder"; - } +// static const char * readTypeToString(ReadFromMergeTree::ReadType type) +// { +// switch (type) +// { +// case ReadFromMergeTree::ReadType::Default: +// return "Default"; +// case ReadFromMergeTree::ReadType::InOrder: +// return "InOrder"; +// case ReadFromMergeTree::ReadType::InReverseOrder: +// return "InReverseOrder"; +// } - __builtin_unreachable(); -} +// __builtin_unreachable(); +// } void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const { diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index b3366afb118..e9341e46770 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -3,11 +3,13 @@ #include #include #include -#include +//#include namespace DB { +using PartitionIdToMaxBlock = std::unordered_map; + /// This step is created to read from MergeTree* table. /// For now, it takes a list of parts and creates source from it. class ReadFromMergeTree final : public ISourceStep @@ -42,9 +44,10 @@ public: struct Settings { UInt64 max_block_size; + size_t num_streams; size_t preferred_block_size_bytes; size_t preferred_max_column_in_block_size_bytes; - size_t min_marks_for_concurrent_read; + //size_t min_marks_for_concurrent_read; bool use_uncompressed_cache; bool force_primary_key; @@ -68,21 +71,18 @@ public: }; ReadFromMergeTree( - SelectQueryInfo query_info_, - const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, + const SelectQueryInfo & query_info_, + const PartitionIdToMaxBlock * max_block_numbers_to_read_, ContextPtr context_, const MergeTreeData & data_, - const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, StorageMetadataPtr metadata_snapshot_base_, Names real_column_names_, MergeTreeData::DataPartsVector parts_, - //IndexStatPtr index_stats_, PrewhereInfoPtr prewhere_info_, Names virt_column_names_, Settings settings_, - size_t num_streams_, - //ReadType read_type_ + Poco::Logger * log_ ); String getName() const override { return "ReadFromMergeTree"; } @@ -97,23 +97,19 @@ public: private: SelectQueryInfo query_info; - const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read; + const PartitionIdToMaxBlock * max_block_numbers_to_read; ContextPtr context; const MergeTreeData & data; - const MergeTreeData & storage; StorageMetadataPtr metadata_snapshot; StorageMetadataPtr metadata_snapshot_base; Names real_column_names; - MergeTreeData::DataPartsVector parts; - IndexStats index_stats; + MergeTreeData::DataPartsVector prepared_parts; PrewhereInfoPtr prewhere_info; + IndexStats index_stats; Names virt_column_names; Settings settings; - size_t num_streams; - //ReadType read_type; - Poco::Logger * log; Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 41adca37c60..e16bbb640e2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3839,21 +3839,18 @@ static void selectBestProjection( if (projection_parts.empty()) return; - candidate.merge_tree_data_select_base_cache = std::make_unique(); - candidate.merge_tree_data_select_projection_cache = std::make_unique(); - reader.readFromParts( + //candidate.merge_tree_data_select_base_cache = std::make_unique(); + //candidate.merge_tree_data_select_projection_cache = std::make_unique(); + auto sum_marks = reader.estimateNumMarksToRead( projection_parts, candidate.required_columns, metadata_snapshot, candidate.desc->metadata, query_info, // TODO syntax_analysis_result set in index query_context, - 0, // max_block_size is unused when getting cache settings.max_threads, - max_added_blocks, - candidate.merge_tree_data_select_projection_cache.get()); + max_added_blocks); - size_t sum_marks = candidate.merge_tree_data_select_projection_cache->sum_marks; if (normal_parts.empty()) { // All parts are projection parts which allows us to use in_order_optimization. @@ -3862,18 +3859,15 @@ static void selectBestProjection( } else { - reader.readFromParts( + sum_marks += reader.estimateNumMarksToRead( normal_parts, required_columns, metadata_snapshot, metadata_snapshot, query_info, // TODO syntax_analysis_result set in index query_context, - 0, // max_block_size is unused when getting cache settings.max_threads, - max_added_blocks, - candidate.merge_tree_data_select_base_cache.get()); - sum_marks += candidate.merge_tree_data_select_base_cache->sum_marks; + max_added_blocks); } // We choose the projection with least sum_marks to read. @@ -4101,7 +4095,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( if (!candidates.empty()) { // First build a MergeTreeDataSelectCache to check if a projection is indeed better than base - query_info.merge_tree_data_select_cache = std::make_unique(); + // query_info.merge_tree_data_select_cache = std::make_unique(); std::unique_ptr max_added_blocks; if (settings.select_sequential_consistency) @@ -4112,21 +4106,10 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( auto parts = getDataPartsVector(); MergeTreeDataSelectExecutor reader(*this); - reader.readFromParts( - parts, - analysis_result.required_columns, - metadata_snapshot, - metadata_snapshot, - query_info, // TODO syntax_analysis_result set in index - query_context, - 0, // max_block_size is unused when getting cache - settings.max_threads, - max_added_blocks.get(), - query_info.merge_tree_data_select_cache.get()); - // Add 1 to base sum_marks so that we prefer projections even when they have equal number of marks to read. - size_t min_sum_marks = query_info.merge_tree_data_select_cache->sum_marks + 1; ProjectionCandidate * selected_candidate = nullptr; + size_t min_sum_marks = std::numeric_limits::max(); + bool has_ordinary_projection = false; /// Favor aggregate projections for (auto & candidate : candidates) { @@ -4145,11 +4128,27 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( selected_candidate, min_sum_marks); } + else + has_ordinary_projection = true; } /// Select the best normal projection if no aggregate projection is available - if (!selected_candidate) + if (!selected_candidate && has_ordinary_projection) { + min_sum_marks = reader.estimateNumMarksToRead( + parts, + analysis_result.required_columns, + metadata_snapshot, + metadata_snapshot, + query_info, // TODO syntax_analysis_result set in index + query_context, + settings.max_threads, + max_added_blocks.get()); + + // Add 1 to base sum_marks so that we prefer projections even when they have equal number of marks to read. + // NOTE: It is not clear if we need it. E.g. projections do not support skip index for now. + min_sum_marks += 1; + for (auto & candidate : candidates) { if (candidate.desc->type == ProjectionDescription::Type::Normal) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 49bcf751bd1..8a3550fc511 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -47,7 +47,6 @@ #include #include #include -#include #include namespace ProfileEvents @@ -143,6 +142,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( const PartitionIdToMaxBlock * max_block_numbers_to_read) const { const auto & settings = context->getSettingsRef(); + auto parts = data.getDataPartsVector(); if (!query_info.projection) { if (settings.allow_experimental_projection_optimization && settings.force_optimize_projection @@ -150,7 +150,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( throw Exception("No projection is used when allow_experimental_projection_optimization = 1", ErrorCodes::PROJECTION_NOT_USED); return readFromParts( - data.getDataPartsVector(), + parts, column_names_to_return, metadata_snapshot, metadata_snapshot, @@ -158,8 +158,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( context, max_block_size, num_streams, - max_block_numbers_to_read, - query_info.merge_tree_data_select_cache.get()); + max_block_numbers_to_read); } LOG_DEBUG( @@ -168,21 +167,33 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( ProjectionDescription::typeToString(query_info.projection->desc->type), query_info.projection->desc->name); - if (query_info.projection->merge_tree_data_select_base_cache->sum_marks - + query_info.projection->merge_tree_data_select_projection_cache->sum_marks - == 0) - return std::make_unique(); + // if (query_info.projection->merge_tree_data_select_base_cache->sum_marks + // + query_info.projection->merge_tree_data_select_projection_cache->sum_marks + // == 0) + // return std::make_unique(); + + MergeTreeData::DataPartsVector projection_parts; + MergeTreeData::DataPartsVector normal_parts; + for (const auto & part : parts) + { + const auto & projections = part->getProjectionParts(); + auto it = projections.find(query_info.projection->desc->name); + if (it != projections.end()) + projection_parts.push_back(it->second); + else + normal_parts.push_back(part); + } Pipes pipes; Pipe projection_pipe; Pipe ordinary_pipe; const auto & given_select = query_info.query->as(); - if (query_info.projection->merge_tree_data_select_projection_cache->sum_marks > 0) + if (!projection_parts.empty()) { LOG_DEBUG(log, "projection required columns: {}", fmt::join(query_info.projection->required_columns, ", ")); auto plan = readFromParts( - {}, + projection_parts, query_info.projection->required_columns, metadata_snapshot, query_info.projection->desc->metadata, @@ -190,8 +201,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( context, max_block_size, num_streams, - max_block_numbers_to_read, - query_info.projection->merge_tree_data_select_projection_cache.get()); + max_block_numbers_to_read); if (plan) { @@ -225,9 +235,9 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( } } - if (query_info.projection->merge_tree_data_select_base_cache->sum_marks > 0) + if (!normal_parts.empty()) { - auto storage_from_base_parts_of_projection = StorageFromBasePartsOfProjection::create(data, metadata_snapshot); + auto storage_from_base_parts_of_projection = StorageFromMergeTreeDataPart::create(std::move(normal_parts)); auto ast = query_info.projection->desc->query_ast->clone(); auto & select = ast->as(); if (given_select.where()) @@ -382,7 +392,6 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( ContextPtr context) { const Settings & settings = context->getSettingsRef(); - Float64 used_sample_factor = 1; /// Sampling. MergeTreeDataSelectSamplingData sampling; @@ -485,7 +494,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (sampling.use_sampling) { if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) - used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); + sampling.used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); RelativeSize size_of_universum = 0; const auto & sampling_key = metadata_snapshot->getSamplingKey(); @@ -633,16 +642,137 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( return sampling; } -RangesInDataParts MergeTreeDataSelectExecutor::filterParts( +std::optional> MergeTreeDataSelectExecutor::filterPartsByVirtualColumns( + const MergeTreeData & data, MergeTreeData::DataPartsVector & parts, + const ASTPtr & query, + ContextPtr context) +{ + std::unordered_set part_values; + ASTPtr expression_ast; + auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); + + // Generate valid expressions for filtering + VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast); + + // If there is still something left, fill the virtual block and do the filtering. + if (expression_ast) + { + virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */); + VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context, expression_ast); + return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); + } + + return {}; +} + +void MergeTreeDataSelectExecutor::filterPartsByPartition( + const StorageMetadataPtr & metadata_snapshot, + const MergeTreeData & data, + const SelectQueryInfo & query_info, + ContextPtr & context, + ContextPtr & query_context, + MergeTreeData::DataPartsVector & parts, + const std::optional> & part_values, + const PartitionIdToMaxBlock * max_block_numbers_to_read, + Poco::Logger * log, + ReadFromMergeTree::IndexStats & index_stats) +{ + const Settings & settings = context->getSettingsRef(); + std::optional partition_pruner; + std::optional minmax_idx_condition; + DataTypes minmax_columns_types; + if (metadata_snapshot->hasPartitionKey()) + { + const auto & partition_key = metadata_snapshot->getPartitionKey(); + auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); + minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); + + minmax_idx_condition.emplace( + query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context))); + partition_pruner.emplace(metadata_snapshot->getPartitionKey(), query_info, context, false /* strict */); + + if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) + { + String msg = "Neither MinMax index by columns ("; + bool first = true; + for (const String & col : minmax_columns_names) + { + if (first) + first = false; + else + msg += ", "; + msg += col; + } + msg += ") nor partition expr is used and setting 'force_index_by_date' is set"; + + throw Exception(msg, ErrorCodes::INDEX_NOT_USED); + } + } + + PartFilterCounters part_filter_counters; + if (query_context->getSettingsRef().allow_experimental_query_deduplication) + selectPartsToReadWithUUIDFilter( + parts, + part_values, + data.getPinnedPartUUIDs(), + minmax_idx_condition, + minmax_columns_types, + partition_pruner, + max_block_numbers_to_read, + query_context, + part_filter_counters, + log); + else + selectPartsToRead( + parts, + part_values, + minmax_idx_condition, + minmax_columns_types, + partition_pruner, + max_block_numbers_to_read, + part_filter_counters); + + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::None, + .num_parts_after = part_filter_counters.num_initial_selected_parts, + .num_granules_after = part_filter_counters.num_initial_selected_granules}); + + if (minmax_idx_condition) + { + auto description = minmax_idx_condition->getDescription(); + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::MinMax, + .condition = std::move(description.condition), + .used_keys = std::move(description.used_keys), + .num_parts_after = part_filter_counters.num_parts_after_minmax, + .num_granules_after = part_filter_counters.num_granules_after_minmax}); + LOG_DEBUG(log, "MinMax index condition: {}", minmax_idx_condition->toString()); + } + + if (partition_pruner) + { + auto description = partition_pruner->getKeyCondition().getDescription(); + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::Partition, + .condition = std::move(description.condition), + .used_keys = std::move(description.used_keys), + .num_parts_after = part_filter_counters.num_parts_after_partition_pruner, + .num_granules_after = part_filter_counters.num_granules_after_partition_pruner}); + } +} + +RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( + MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, - SelectQueryInfo & query_info, + const SelectQueryInfo & query_info, ContextPtr & context, KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, - ReadFromMergeTree::IndexStats & index_stats) + ReadFromMergeTree::IndexStats & index_stats, + bool use_skip_indexes) { RangesInDataParts parts_with_ranges(parts.size()); const Settings & settings = context->getSettingsRef(); @@ -665,15 +795,18 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterParts( }; std::list useful_indices; - for (const auto & index : metadata_snapshot->getSecondaryIndices()) + if (use_skip_indexes) { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - auto condition = index_helper->createIndexCondition(query_info, context); - if (!condition->alwaysUnknownOrTrue()) - useful_indices.emplace_back(index_helper, condition); + for (const auto & index : metadata_snapshot->getSecondaryIndices()) + { + auto index_helper = MergeTreeIndexFactory::instance().get(index); + auto condition = index_helper->createIndexCondition(query_info, context); + if (!condition->alwaysUnknownOrTrue()) + useful_indices.emplace_back(index_helper, condition); + } } - if (settings.force_data_skipping_indices.changed) + if (use_skip_indexes && settings.force_data_skipping_indices.changed) { const auto & indices = settings.force_data_skipping_indices.toString(); @@ -860,7 +993,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterParts( return parts_with_ranges; } -void checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_range, ContextPtr & context) +String MergeTreeDataSelectExecutor::checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context) { const auto & settings = context->getSettingsRef(); // Check limitations. query_id is used as the quota RAII's resource key. @@ -872,7 +1005,7 @@ void checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_rang if (max_partitions_to_read > 0) { std::set partitions; - for (auto & part_with_ranges : parts_with_ranges) + for (const auto & part_with_ranges : parts_with_ranges) partitions.insert(part_with_ranges.data_part->info.partition_id); if (partitions.size() > size_t(max_partitions_to_read)) throw Exception( @@ -896,33 +1029,18 @@ void checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_rang } } } + + return query_id; } -QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( - MergeTreeData::DataPartsVector parts, +static void selectColumnNames( const Names & column_names_to_return, - const StorageMetadataPtr & metadata_snapshot_base, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - ContextPtr context, - const UInt64 max_block_size, - const unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read, - MergeTreeDataSelectCache * cache) const + const MergeTreeData & data, + Names & real_column_names, + Names & virt_column_names, + bool & sample_factor_column_queried) { - bool use_cache = cache && cache->use_cache; - - /// If query contains restrictions on the virtual column `_part` or `_part_index`, select only parts suitable for it. - /// The virtual column `_sample_factor` (which is equal to 1 / used sample rate) can be requested in the query. - Names virt_column_names; - Names real_column_names; - - size_t total_parts = parts.size(); - if (!use_cache && total_parts == 0) - return std::make_unique(); - - bool sample_factor_column_queried = false; - Float64 used_sample_factor = 1; + sample_factor_column_queried = false; for (const String & name : column_names_to_return) { @@ -964,395 +1082,112 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( real_column_names.push_back(name); } } +} - // Filter parts by virtual columns. - std::unordered_set part_values; - if (!use_cache) - { - ASTPtr expression_ast; - auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); +size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead( + MergeTreeData::DataPartsVector parts, + const Names & column_names_to_return, + const StorageMetadataPtr & metadata_snapshot_base, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + ContextPtr context, + unsigned num_streams, + const PartitionIdToMaxBlock * max_block_numbers_to_read) const +{ + size_t total_parts = parts.size(); + if (total_parts == 0) + return 0; - // Generate valid expressions for filtering - VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, context, virtual_columns_block, expression_ast); + Names real_column_names; + Names virt_column_names; + /// If query contains restrictions on the virtual column `_part` or `_part_index`, select only parts suitable for it. + /// The virtual column `_sample_factor` (which is equal to 1 / used sample rate) can be requested in the query. + bool sample_factor_column_queried = false; - // If there is still something left, fill the virtual block and do the filtering. - if (expression_ast) - { - virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */); - VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context, expression_ast); - part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); - if (part_values.empty()) - return std::make_unique(); - } - } - // At this point, empty `part_values` means all parts. + selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - const Settings & settings = context->getSettingsRef(); - NamesAndTypesList available_real_columns = metadata_snapshot->getColumns().getAllPhysical(); + auto part_values = filterPartsByVirtualColumns(data, parts, query_info.query, context); + if (part_values && part_values->empty()) + return 0; /// If there are only virtual columns in the query, you must request at least one non-virtual one. if (real_column_names.empty()) + { + NamesAndTypesList available_real_columns = metadata_snapshot->getColumns().getAllPhysical(); real_column_names.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); + } metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); - // Build and check if primary key is used when necessary - std::optional key_condition; - if (!use_cache) - { - const auto & primary_key = metadata_snapshot->getPrimaryKey(); - Names primary_key_columns = primary_key.column_names; - key_condition.emplace(query_info, context, primary_key_columns, primary_key.expression); + const auto & primary_key = metadata_snapshot->getPrimaryKey(); + Names primary_key_columns = primary_key.column_names; + KeyCondition key_condition(query_info, context, primary_key_columns, primary_key.expression); - if (settings.force_primary_key && key_condition->alwaysUnknownOrTrue()) - { - throw Exception( - ErrorCodes::INDEX_NOT_USED, - "Primary key ({}) is not used and setting 'force_primary_key' is set.", - fmt::join(primary_key_columns, ", ")); - } - LOG_DEBUG(log, "Key condition: {}", key_condition->toString()); + if (key_condition.alwaysUnknownOrTrue()) + { + size_t total_marks = 0; + for (const auto & part : parts) + total_marks += part->index_granularity.getMarksCountWithoutFinal(); + + return total_marks; } const auto & select = query_info.query->as(); auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; - auto index_stats = use_cache ? std::move(cache->index_stats) : std::make_unique(); + ReadFromMergeTree::IndexStats index_stats; - // Select parts to read and do partition pruning via partition value and minmax indices - if (!use_cache) - { - std::optional partition_pruner; - std::optional minmax_idx_condition; - DataTypes minmax_columns_types; - if (metadata_snapshot_base->hasPartitionKey()) - { - const auto & partition_key = metadata_snapshot_base->getPartitionKey(); - auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); - minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); + filterPartsByPartition( + metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read, log, index_stats); - minmax_idx_condition.emplace( - query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context))); - partition_pruner.emplace(metadata_snapshot_base->getPartitionKey(), query_info, context, false /* strict */); + auto sampling = MergeTreeDataSelectExecutor::getSampling( + select, parts, metadata_snapshot, key_condition, + data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); - if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) - { - String msg = "Neither MinMax index by columns ("; - bool first = true; - for (const String & col : minmax_columns_names) - { - if (first) - first = false; - else - msg += ", "; - msg += col; - } - msg += ") nor partition expr is used and setting 'force_index_by_date' is set"; + if (sampling.read_nothing) + return 0; - throw Exception(msg, ErrorCodes::INDEX_NOT_USED); - } - } + /// Do not init. Ther are not used (cause skip index is ignored) + MergeTreeReaderSettings reader_settings; - PartFilterCounters part_filter_counters; - if (query_context->getSettingsRef().allow_experimental_query_deduplication) - selectPartsToReadWithUUIDFilter( - parts, - part_values, - data.getPinnedPartUUIDs(), - minmax_idx_condition, - minmax_columns_types, - partition_pruner, - max_block_numbers_to_read, - query_context, - part_filter_counters, - log); - else - selectPartsToRead( - parts, - part_values, - minmax_idx_condition, - minmax_columns_types, - partition_pruner, - max_block_numbers_to_read, - part_filter_counters); + auto parts_with_ranges = filterPartsByPrimaryKeyAndSkipIndexes( + std::move(parts), + metadata_snapshot, + query_info, + context, + key_condition, + reader_settings, + log, + num_streams, + index_stats, + false); - index_stats->emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::None, - .num_parts_after = part_filter_counters.num_initial_selected_parts, - .num_granules_after = part_filter_counters.num_initial_selected_granules}); + return index_stats.back().num_granules_after; +} - if (minmax_idx_condition) - { - auto description = minmax_idx_condition->getDescription(); - index_stats->emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::MinMax, - .condition = std::move(description.condition), - .used_keys = std::move(description.used_keys), - .num_parts_after = part_filter_counters.num_parts_after_minmax, - .num_granules_after = part_filter_counters.num_granules_after_minmax}); - LOG_DEBUG(log, "MinMax index condition: {}", minmax_idx_condition->toString()); - } +QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( + MergeTreeData::DataPartsVector parts, + const Names & column_names_to_return, + const StorageMetadataPtr & metadata_snapshot_base, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + ContextPtr context, + const UInt64 max_block_size, + const unsigned num_streams, + const PartitionIdToMaxBlock * max_block_numbers_to_read) const +{ + size_t total_parts = parts.size(); + if (total_parts == 0) + return std::make_unique(); - if (partition_pruner) - { - auto description = partition_pruner->getKeyCondition().getDescription(); - index_stats->emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::Partition, - .condition = std::move(description.condition), - .used_keys = std::move(description.used_keys), - .num_parts_after = part_filter_counters.num_parts_after_partition_pruner, - .num_granules_after = part_filter_counters.num_granules_after_partition_pruner}); - } - } + Names real_column_names; + Names virt_column_names; + /// If query contains restrictions on the virtual column `_part` or `_part_index`, select only parts suitable for it. + /// The virtual column `_sample_factor` (which is equal to 1 / used sample rate) can be requested in the query. + bool sample_factor_column_queried = false; - /// Sampling. - MergeTreeDataSelectSamplingData sampling = use_cache ? std::move(cache->sampling) : MergeTreeDataSelectSamplingData{}; - if (!use_cache) - { - assert(key_condition.has_value()); + selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - RelativeSize relative_sample_size = 0; - RelativeSize relative_sample_offset = 0; - - auto select_sample_size = select.sampleSize(); - auto select_sample_offset = select.sampleOffset(); - - if (select_sample_size) - { - relative_sample_size.assign( - select_sample_size->as().ratio.numerator, - select_sample_size->as().ratio.denominator); - - if (relative_sample_size < 0) - throw Exception("Negative sample size", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - relative_sample_offset = 0; - if (select_sample_offset) - relative_sample_offset.assign( - select_sample_offset->as().ratio.numerator, - select_sample_offset->as().ratio.denominator); - - if (relative_sample_offset < 0) - throw Exception("Negative sample offset", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - /// Convert absolute value of the sampling (in form `SAMPLE 1000000` - how many rows to - /// read) into the relative `SAMPLE 0.1` (how much data to read). - size_t approx_total_rows = 0; - if (relative_sample_size > 1 || relative_sample_offset > 1) - approx_total_rows = getApproximateTotalRowsToRead(parts, metadata_snapshot, *key_condition, settings); //-V1007 - - if (relative_sample_size > 1) - { - relative_sample_size = convertAbsoluteSampleSizeToRelative(select_sample_size, approx_total_rows); - LOG_DEBUG(log, "Selected relative sample size: {}", toString(relative_sample_size)); - } - - /// SAMPLE 1 is the same as the absence of SAMPLE. - if (relative_sample_size == RelativeSize(1)) - relative_sample_size = 0; - - if (relative_sample_offset > 0 && RelativeSize(0) == relative_sample_size) - throw Exception("Sampling offset is incorrect because no sampling", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - if (relative_sample_offset > 1) - { - relative_sample_offset = convertAbsoluteSampleSizeToRelative(select_sample_offset, approx_total_rows); - LOG_DEBUG(log, "Selected relative sample offset: {}", toString(relative_sample_offset)); - } - } - - /** Which range of sampling key values do I need to read? - * First, in the whole range ("universe") we select the interval - * of relative `relative_sample_size` size, offset from the beginning by `relative_sample_offset`. - * - * Example: SAMPLE 0.4 OFFSET 0.3 - * - * [------********------] - * ^ - offset - * <------> - size - * - * If the interval passes through the end of the universe, then cut its right side. - * - * Example: SAMPLE 0.4 OFFSET 0.8 - * - * [----------------****] - * ^ - offset - * <------> - size - * - * Next, if the `parallel_replicas_count`, `parallel_replica_offset` settings are set, - * then it is necessary to break the received interval into pieces of the number `parallel_replicas_count`, - * and select a piece with the number `parallel_replica_offset` (from zero). - * - * Example: SAMPLE 0.4 OFFSET 0.3, parallel_replicas_count = 2, parallel_replica_offset = 1 - * - * [----------****------] - * ^ - offset - * <------> - size - * <--><--> - pieces for different `parallel_replica_offset`, select the second one. - * - * It is very important that the intervals for different `parallel_replica_offset` cover the entire range without gaps and overlaps. - * It is also important that the entire universe can be covered using SAMPLE 0.1 OFFSET 0, ... OFFSET 0.9 and similar decimals. - */ - - /// Parallel replicas has been requested but there is no way to sample data. - /// Select all data from first replica and no data from other replicas. - if (settings.parallel_replicas_count > 1 && !data.supportsSampling() && settings.parallel_replica_offset > 0) - { - LOG_DEBUG(log, "Will use no data on this replica because parallel replicas processing has been requested" - " (the setting 'max_parallel_replicas') but the table does not support sampling and this replica is not the first."); - return std::make_unique(); - } - - sampling.use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && data.supportsSampling()); - bool no_data = false; /// There is nothing left after sampling. - - if (sampling.use_sampling) - { - if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) - used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); - - RelativeSize size_of_universum = 0; - const auto & sampling_key = metadata_snapshot->getSamplingKey(); - DataTypePtr sampling_column_type = sampling_key.data_types[0]; - - if (sampling_key.data_types.size() == 1) - { - if (typeid_cast(sampling_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(sampling_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(sampling_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(sampling_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - } - - if (size_of_universum == RelativeSize(0)) - throw Exception( - "Invalid sampling column type in storage parameters: " + sampling_column_type->getName() - + ". Must be one unsigned integer type", - ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); - - if (settings.parallel_replicas_count > 1) - { - if (relative_sample_size == RelativeSize(0)) - relative_sample_size = 1; - - relative_sample_size /= settings.parallel_replicas_count.value; - relative_sample_offset += relative_sample_size * RelativeSize(settings.parallel_replica_offset.value); - } - - if (relative_sample_offset >= RelativeSize(1)) - no_data = true; - - /// Calculate the half-interval of `[lower, upper)` column values. - bool has_lower_limit = false; - bool has_upper_limit = false; - - RelativeSize lower_limit_rational = relative_sample_offset * size_of_universum; - RelativeSize upper_limit_rational = (relative_sample_offset + relative_sample_size) * size_of_universum; - - UInt64 lower = boost::rational_cast(lower_limit_rational); - UInt64 upper = boost::rational_cast(upper_limit_rational); - - if (lower > 0) - has_lower_limit = true; - - if (upper_limit_rational < size_of_universum) - has_upper_limit = true; - - /*std::cerr << std::fixed << std::setprecision(100) - << "relative_sample_size: " << relative_sample_size << "\n" - << "relative_sample_offset: " << relative_sample_offset << "\n" - << "lower_limit_float: " << lower_limit_rational << "\n" - << "upper_limit_float: " << upper_limit_rational << "\n" - << "lower: " << lower << "\n" - << "upper: " << upper << "\n";*/ - - if ((has_upper_limit && upper == 0) - || (has_lower_limit && has_upper_limit && lower == upper)) - no_data = true; - - if (no_data || (!has_lower_limit && !has_upper_limit)) - { - sampling.use_sampling = false; - } - else - { - /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. - - std::shared_ptr lower_function; - std::shared_ptr upper_function; - - /// If sample and final are used together no need to calculate sampling expression twice. - /// The first time it was calculated for final, because sample key is a part of the PK. - /// So, assume that we already have calculated column. - ASTPtr sampling_key_ast = metadata_snapshot->getSamplingKeyAST(); - - if (select.final()) - { - sampling_key_ast = std::make_shared(sampling_key.column_names[0]); - /// We do spoil available_real_columns here, but it is not used later. - available_real_columns.emplace_back(sampling_key.column_names[0], std::move(sampling_column_type)); - } - - if (has_lower_limit) - { - if (!key_condition->addCondition(sampling_key.column_names[0], Range::createLeftBounded(lower, true))) //-V1007 - throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); - - ASTPtr args = std::make_shared(); - args->children.push_back(sampling_key_ast); - args->children.push_back(std::make_shared(lower)); - - lower_function = std::make_shared(); - lower_function->name = "greaterOrEquals"; - lower_function->arguments = args; - lower_function->children.push_back(lower_function->arguments); - - sampling.filter_function = lower_function; - } - - if (has_upper_limit) - { - if (!key_condition->addCondition(sampling_key.column_names[0], Range::createRightBounded(upper, false))) //-V1007 - throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); - - ASTPtr args = std::make_shared(); - args->children.push_back(sampling_key_ast); - args->children.push_back(std::make_shared(upper)); - - upper_function = std::make_shared(); - upper_function->name = "less"; - upper_function->arguments = args; - upper_function->children.push_back(upper_function->arguments); - - sampling.filter_function = upper_function; - } - - if (has_lower_limit && has_upper_limit) - { - ASTPtr args = std::make_shared(); - args->children.push_back(lower_function); - args->children.push_back(upper_function); - - sampling.filter_function = std::make_shared(); - sampling.filter_function->name = "and"; - sampling.filter_function->arguments = args; - sampling.filter_function->children.push_back(sampling.filter_function->arguments); - } - - ASTPtr query = sampling.filter_function; - auto syntax_result = TreeRewriter(context).analyze(query, available_real_columns); - sampling.filter_expression = ExpressionAnalyzer(sampling.filter_function, syntax_result, context).getActionsDAG(false); - } - } - - if (no_data) - { - LOG_DEBUG(log, "Sampling yields no data."); - return std::make_unique(); - } - } + const auto & settings = context->getSettingsRef(); MergeTreeReaderSettings reader_settings = { @@ -1364,443 +1199,48 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( .checksum_on_read = settings.checksum_on_read, }; - RangesInDataParts parts_with_ranges(parts.size()); - size_t sum_marks = 0; - size_t sum_ranges = 0; - - /// Let's start analyzing all useful indices - if (!use_cache) + ReadFromMergeTree::Settings step_settings { - struct DataSkippingIndexAndCondition - { - MergeTreeIndexPtr index; - MergeTreeIndexConditionPtr condition; - std::atomic total_granules{0}; - std::atomic granules_dropped{0}; - std::atomic total_parts{0}; - std::atomic parts_dropped{0}; - - DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_) - : index(index_), condition(condition_) - { - } - }; - std::list useful_indices; - - for (const auto & index : metadata_snapshot->getSecondaryIndices()) - { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - auto condition = index_helper->createIndexCondition(query_info, context); - if (!condition->alwaysUnknownOrTrue()) - useful_indices.emplace_back(index_helper, condition); - } - - if (settings.force_data_skipping_indices.changed) - { - const auto & indices = settings.force_data_skipping_indices.toString(); - - Strings forced_indices; - { - Tokens tokens(&indices[0], &indices[indices.size()], settings.max_query_size); - IParser::Pos pos(tokens, settings.max_parser_depth); - Expected expected; - if (!parseIdentifiersOrStringLiterals(pos, expected, forced_indices)) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse force_data_skipping_indices ('{}')", indices); - } - - if (forced_indices.empty()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "No indices parsed from force_data_skipping_indices ('{}')", indices); - - std::unordered_set useful_indices_names; - for (const auto & useful_index : useful_indices) - useful_indices_names.insert(useful_index.index->index.name); - - for (const auto & index_name : forced_indices) - { - if (!useful_indices_names.count(index_name)) - { - throw Exception( - ErrorCodes::INDEX_NOT_USED, - "Index {} is not used and setting 'force_data_skipping_indices' contains it", - backQuote(index_name)); - } - } - } - - std::atomic sum_marks_pk = 0; - std::atomic sum_parts_pk = 0; - std::atomic total_marks_pk = 0; - - /// Let's find what range to read from each part. - { - std::atomic total_rows{0}; - - SizeLimits limits; - if (settings.read_overflow_mode == OverflowMode::THROW && settings.max_rows_to_read) - limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode); - - SizeLimits leaf_limits; - if (settings.read_overflow_mode_leaf == OverflowMode::THROW && settings.max_rows_to_read_leaf) - leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf); - - auto process_part = [&](size_t part_index) - { - auto & part = parts[part_index]; - - RangesInDataPart ranges(part, part_index); - - size_t total_marks_count = part->getMarksCount(); - if (total_marks_count && part->index_granularity.hasFinalMark()) - --total_marks_count; - - total_marks_pk.fetch_add(total_marks_count, std::memory_order_relaxed); - - if (metadata_snapshot->hasPrimaryKey()) - ranges.ranges = markRangesFromPKRange(part, metadata_snapshot, *key_condition, settings, log); - else if (total_marks_count) - ranges.ranges = MarkRanges{MarkRange{0, total_marks_count}}; - - sum_marks_pk.fetch_add(ranges.getMarksCount(), std::memory_order_relaxed); - - if (!ranges.ranges.empty()) - sum_parts_pk.fetch_add(1, std::memory_order_relaxed); - - for (auto & index_and_condition : useful_indices) - { - if (ranges.ranges.empty()) - break; - - index_and_condition.total_parts.fetch_add(1, std::memory_order_relaxed); - - size_t total_granules = 0; - size_t granules_dropped = 0; - ranges.ranges = filterMarksUsingIndex( - index_and_condition.index, - index_and_condition.condition, - part, - ranges.ranges, - settings, - reader_settings, - total_granules, - granules_dropped, - log); - - index_and_condition.total_granules.fetch_add(total_granules, std::memory_order_relaxed); - index_and_condition.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); - - if (ranges.ranges.empty()) - index_and_condition.parts_dropped.fetch_add(1, std::memory_order_relaxed); - } - - if (!ranges.ranges.empty()) - { - if (limits.max_rows || leaf_limits.max_rows) - { - /// Fail fast if estimated number of rows to read exceeds the limit - auto current_rows_estimate = ranges.getRowsCount(); - size_t prev_total_rows_estimate = total_rows.fetch_add(current_rows_estimate); - size_t total_rows_estimate = current_rows_estimate + prev_total_rows_estimate; - limits.check(total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read' setting)", ErrorCodes::TOO_MANY_ROWS); - leaf_limits.check( - total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read_leaf' setting)", ErrorCodes::TOO_MANY_ROWS); - } - - parts_with_ranges[part_index] = std::move(ranges); - } - }; - - size_t num_threads = std::min(size_t(num_streams), parts.size()); - - if (num_threads <= 1) - { - for (size_t part_index = 0; part_index < parts.size(); ++part_index) - process_part(part_index); - } - else - { - /// Parallel loading of data parts. - ThreadPool pool(num_threads); - - for (size_t part_index = 0; part_index < parts.size(); ++part_index) - pool.scheduleOrThrowOnError([&, part_index, thread_group = CurrentThread::getGroup()] - { - SCOPE_EXIT_SAFE(if (thread_group) CurrentThread::detachQueryIfNotDetached();); - if (thread_group) - CurrentThread::attachTo(thread_group); - - process_part(part_index); - }); - - pool.wait(); - } - - /// Skip empty ranges. - size_t next_part = 0; - for (size_t part_index = 0; part_index < parts.size(); ++part_index) - { - auto & part = parts_with_ranges[part_index]; - if (!part.data_part) - continue; - - sum_ranges += part.ranges.size(); - sum_marks += part.getMarksCount(); - - if (next_part != part_index) - std::swap(parts_with_ranges[next_part], part); - - ++next_part; - } - - parts_with_ranges.resize(next_part); - } - - if (metadata_snapshot->hasPrimaryKey()) - { - auto description = key_condition->getDescription(); - - index_stats->emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::PrimaryKey, - .condition = std::move(description.condition), - .used_keys = std::move(description.used_keys), - .num_parts_after = sum_parts_pk.load(std::memory_order_relaxed), - .num_granules_after = sum_marks_pk.load(std::memory_order_relaxed)}); - } - - for (const auto & index_and_condition : useful_indices) - { - const auto & index_name = index_and_condition.index->index.name; - LOG_DEBUG( - log, - "Index {} has dropped {}/{} granules.", - backQuote(index_name), - index_and_condition.granules_dropped, - index_and_condition.total_granules); - - std::string description - = index_and_condition.index->index.type + " GRANULARITY " + std::to_string(index_and_condition.index->index.granularity); - - index_stats->emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::Skip, - .name = index_name, - .description = std::move(description), - .num_parts_after = index_and_condition.total_parts - index_and_condition.parts_dropped, - .num_granules_after = index_and_condition.total_granules - index_and_condition.granules_dropped}); - } - - LOG_DEBUG( - log, - "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", - parts.size(), - total_parts, - parts_with_ranges.size(), - sum_marks_pk.load(std::memory_order_relaxed), - total_marks_pk.load(std::memory_order_relaxed), - sum_marks, - sum_ranges); - } - - if (cache) - { - if (cache->use_cache) - { - parts_with_ranges = std::move(cache->parts_with_ranges); - sum_marks = cache->sum_marks; - sum_ranges = cache->sum_ranges; - } - else - { - // We are asking for ranges_to_read. Return immediately without further planning. - cache->parts_with_ranges = std::move(parts_with_ranges); - cache->sampling = std::move(sampling); - cache->index_stats = std::move(index_stats); - cache->sum_marks = sum_marks; - cache->sum_ranges = sum_ranges; - cache->use_cache = true; - return std::make_unique(); - } - } - - if (parts_with_ranges.empty()) - return std::make_unique(); - - // Check limitations. query_id is used as the quota RAII's resource key. - String query_id; - { - const auto data_settings = data.getSettings(); - auto max_partitions_to_read - = settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data_settings->max_partitions_to_read; - if (max_partitions_to_read > 0) - { - std::set partitions; - for (auto & part_with_ranges : parts_with_ranges) - partitions.insert(part_with_ranges.data_part->info.partition_id); - if (partitions.size() > size_t(max_partitions_to_read)) - throw Exception( - ErrorCodes::TOO_MANY_PARTITIONS, - "Too many partitions to read. Current {}, max {}", - partitions.size(), - max_partitions_to_read); - } - - if (data_settings->max_concurrent_queries > 0) - { - if (data_settings->min_marks_to_honor_max_concurrent_queries > 0 - && sum_marks >= data_settings->min_marks_to_honor_max_concurrent_queries) - { - query_id = context->getCurrentQueryId(); - if (!query_id.empty()) - data.insertQueryIdOrThrow(query_id, data_settings->max_concurrent_queries); - } - } - } - - ProfileEvents::increment(ProfileEvents::SelectedParts, parts_with_ranges.size()); - ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); - ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks); - - QueryPlanPtr plan; - - /// Projection, that needed to drop columns, which have appeared by execution - /// of some extra expressions, and to allow execute the same expressions later. - /// NOTE: It may lead to double computation of expressions. - ActionsDAGPtr result_projection; - - Names column_names_to_read = real_column_names; - if (!select.final() && sampling.use_sampling) - { - /// Add columns needed for `sample_by_ast` to `column_names_to_read`. - /// Skip this if final was used, because such columns were already added from PK. - std::vector add_columns = sampling.filter_expression->getRequiredColumns().getNames(); - column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); - std::sort(column_names_to_read.begin(), column_names_to_read.end()); - column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), - column_names_to_read.end()); - } - - const auto & input_order_info = query_info.input_order_info - ? query_info.input_order_info - : (query_info.projection ? query_info.projection->input_order_info : nullptr); - - if (select.final()) - { - /// Add columns needed to calculate the sorting expression and the sign. - std::vector add_columns = metadata_snapshot->getColumnsRequiredForSortingKey(); - column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); - - if (!data.merging_params.sign_column.empty()) - column_names_to_read.push_back(data.merging_params.sign_column); - if (!data.merging_params.version_column.empty()) - column_names_to_read.push_back(data.merging_params.version_column); - - std::sort(column_names_to_read.begin(), column_names_to_read.end()); - column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end()); - - plan = spreadMarkRangesAmongStreamsFinal( - std::move(parts_with_ranges), - std::move(index_stats), - num_streams, - column_names_to_read, - metadata_snapshot, - max_block_size, - settings.use_uncompressed_cache, - query_info, - virt_column_names, - settings, - reader_settings, - result_projection, - query_id); - } - else if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && input_order_info) - { - size_t prefix_size = input_order_info->order_key_prefix_descr.size(); - auto order_key_prefix_ast = metadata_snapshot->getSortingKey().expression_list_ast->clone(); - order_key_prefix_ast->children.resize(prefix_size); - - auto syntax_result = TreeRewriter(context).analyze(order_key_prefix_ast, metadata_snapshot->getColumns().getAllPhysical()); - auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActionsDAG(false); - - plan = spreadMarkRangesAmongStreamsWithOrder( - std::move(parts_with_ranges), - std::move(index_stats), - num_streams, - column_names_to_read, - metadata_snapshot, - max_block_size, - settings.use_uncompressed_cache, - query_info, - sorting_key_prefix_expr, - virt_column_names, - settings, - reader_settings, - result_projection, - query_id, - input_order_info); - } - else - { - plan = spreadMarkRangesAmongStreams( - std::move(parts_with_ranges), - std::move(index_stats), - num_streams, - column_names_to_read, - metadata_snapshot, - max_block_size, - settings.use_uncompressed_cache, - query_info, - virt_column_names, - settings, - reader_settings, - query_id); - } - - if (!plan) - return std::make_unique(); - - if (sampling.use_sampling) - { - auto sampling_step = std::make_unique( - plan->getCurrentDataStream(), - sampling.filter_expression, - sampling.filter_function->getColumnName(), - false); - - sampling_step->setStepDescription("Sampling"); - plan->addStep(std::move(sampling_step)); - } - - if (result_projection) - { - auto projection_step = std::make_unique(plan->getCurrentDataStream(), result_projection); - projection_step->setStepDescription("Remove unused columns after reading from storage"); - plan->addStep(std::move(projection_step)); - } - - /// 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(used_sample_factor)); - - auto adding_column_action = ActionsDAG::makeAddingColumnActions(std::move(column)); - - auto adding_column = std::make_unique(plan->getCurrentDataStream(), std::move(adding_column_action)); - adding_column->setStepDescription("Add _sample_factor column"); - plan->addStep(std::move(adding_column)); - } - - // TODO There seems to be no place initializing remove_columns_actions - if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) - { - auto expression_step = std::make_unique( - plan->getCurrentDataStream(), - query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); - - expression_step->setStepDescription("Remove unused columns after PREWHERE"); - plan->addStep(std::move(expression_step)); - } - + .max_block_size = max_block_size, + .num_streams = num_streams, + .preferred_block_size_bytes = settings.preferred_block_size_bytes, + .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, + //.min_marks_for_concurrent_read = settings.min_marks_for_concurrent_read, + .use_uncompressed_cache = settings.use_uncompressed_cache, + .reader_settings = reader_settings, + .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), + }; + + // const SelectQueryInfo & query_info_, + // const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, + // ContextPtr context_, + // const MergeTreeData & data_, + // StorageMetadataPtr metadata_snapshot_, + // StorageMetadataPtr metadata_snapshot_base_, + // Names real_column_names_, + // MergeTreeData::DataPartsVector parts_, + // PrewhereInfoPtr prewhere_info_, + // Names virt_column_names_, + // Settings settings_, + // Poco::Logger * log_ + + auto read_from_merge_tree = std::make_unique( + query_info, + max_block_numbers_to_read, + context, + data, + metadata_snapshot, + metadata_snapshot_base, + real_column_names, + parts, + query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, + virt_column_names, + step_settings, + log + ); + + QueryPlanPtr plan = std::make_unique(); + plan->addStep(std::move(read_from_merge_tree)); return plan; } @@ -1822,638 +1262,609 @@ size_t roundRowsOrBytesToMarks( else return std::max(res, (bytes_setting + bytes_granularity - 1) / bytes_granularity); } -/// Same as roundRowsOrBytesToMarks() but do not return more then max_marks -size_t minMarksForConcurrentRead( - size_t rows_setting, - size_t bytes_setting, - size_t rows_granularity, - size_t bytes_granularity, - size_t max_marks) -{ - size_t marks = 1; - - if (rows_setting + rows_granularity <= rows_setting) /// overflow - marks = max_marks; - else if (rows_setting) - marks = (rows_setting + rows_granularity - 1) / rows_granularity; - - if (bytes_granularity == 0) - return marks; - else - { - /// Overflow - if (bytes_setting + bytes_granularity <= bytes_setting) /// overflow - return max_marks; - if (bytes_setting) - return std::max(marks, (bytes_setting + bytes_granularity - 1) / bytes_granularity); - else - return marks; - } -} } -QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( - RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - const String & query_id) const -{ - /// Count marks for each part. - std::vector sum_marks_in_parts(parts.size()); - size_t sum_marks = 0; - size_t total_rows = 0; - - const auto data_settings = data.getSettings(); - size_t adaptive_parts = 0; - for (size_t i = 0; i < parts.size(); ++i) - { - total_rows += parts[i].getRowsCount(); - sum_marks_in_parts[i] = parts[i].getMarksCount(); - sum_marks += sum_marks_in_parts[i]; - - if (parts[i].data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } - - size_t index_granularity_bytes = 0; - if (adaptive_parts > parts.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( - settings.merge_tree_min_rows_for_concurrent_read, - settings.merge_tree_min_bytes_for_concurrent_read, - data_settings->index_granularity, - index_granularity_bytes, - sum_marks); - - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; - - if (0 == sum_marks) - return {}; - - ReadFromMergeTree::Settings step_settings - { - .max_block_size = max_block_size, - .preferred_block_size_bytes = settings.preferred_block_size_bytes, - .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - .min_marks_for_concurrent_read = min_marks_for_concurrent_read, - .use_uncompressed_cache = use_uncompressed_cache, - .reader_settings = reader_settings, - .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - }; - - if (num_streams > 1) - { - /// Reduce the number of num_streams if the data is small. - if (sum_marks < num_streams * min_marks_for_concurrent_read && parts.size() < num_streams) - num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts.size()); - } - - auto plan = std::make_unique(); - auto step = std::make_unique( - data, - metadata_snapshot, - query_id, - column_names, - std::move(parts), - std::move(index_stats), - query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - virt_columns, - step_settings, - num_streams, - ReadFromMergeTree::ReadType::Default); - - plan->addStep(std::move(step)); - return plan; -} - -static ActionsDAGPtr createProjection(const Block & header) -{ - auto projection = std::make_shared(header.getNamesAndTypesList()); - projection->removeUnusedActions(header.getNames()); - projection->projectInput(); - return projection; -} - -QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( - RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const ActionsDAGPtr & sorting_key_prefix_expr, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection, - const String & query_id, - const InputOrderInfoPtr & input_order_info) const -{ - size_t sum_marks = 0; - size_t adaptive_parts = 0; - std::vector sum_marks_in_parts(parts.size()); - const auto data_settings = data.getSettings(); - - for (size_t i = 0; i < parts.size(); ++i) - { - sum_marks_in_parts[i] = parts[i].getMarksCount(); - sum_marks += sum_marks_in_parts[i]; - - if (parts[i].data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } - - size_t index_granularity_bytes = 0; - if (adaptive_parts > parts.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( - settings.merge_tree_min_rows_for_concurrent_read, - settings.merge_tree_min_bytes_for_concurrent_read, - data_settings->index_granularity, - index_granularity_bytes, - sum_marks); - - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; - - Pipes res; - - if (sum_marks == 0) - return {}; - - /// Let's split ranges to avoid reading much data. - auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size](const auto & ranges, int direction) - { - MarkRanges new_ranges; - const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; - size_t marks_in_range = 1; - - if (direction == 1) - { - /// Split first few ranges to avoid reading much data. - bool split = false; - for (auto range : ranges) - { - while (!split && range.begin + marks_in_range < range.end) - { - new_ranges.emplace_back(range.begin, range.begin + marks_in_range); - range.begin += marks_in_range; - marks_in_range *= 2; - - if (marks_in_range > max_marks_in_range) - split = true; - } - new_ranges.emplace_back(range.begin, range.end); - } - } - else - { - /// Split all ranges to avoid reading much data, because we have to - /// store whole range in memory to reverse it. - for (auto it = ranges.rbegin(); it != ranges.rend(); ++it) - { - auto range = *it; - while (range.begin + marks_in_range < range.end) - { - new_ranges.emplace_front(range.end - marks_in_range, range.end); - range.end -= marks_in_range; - marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); - } - new_ranges.emplace_front(range.begin, range.end); - } - } - - return new_ranges; - }; - - const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; - bool need_preliminary_merge = (parts.size() > settings.read_in_order_two_level_merge_threshold); - - std::vector plans; - - for (size_t i = 0; i < num_streams && !parts.empty(); ++i) - { - size_t need_marks = min_marks_per_stream; - RangesInDataParts new_parts; - - /// Loop over parts. - /// We will iteratively take part or some subrange of a part from the back - /// and assign a stream to read from it. - while (need_marks > 0 && !parts.empty()) - { - RangesInDataPart part = parts.back(); - parts.pop_back(); - - size_t & marks_in_part = sum_marks_in_parts.back(); - - /// We will not take too few rows from a part. - if (marks_in_part >= min_marks_for_concurrent_read && - need_marks < min_marks_for_concurrent_read) - need_marks = min_marks_for_concurrent_read; - - /// Do not leave too few rows in the part. - if (marks_in_part > need_marks && - marks_in_part - need_marks < min_marks_for_concurrent_read) - need_marks = marks_in_part; - - MarkRanges ranges_to_get_from_part; - - /// We take the whole part if it is small enough. - if (marks_in_part <= need_marks) - { - ranges_to_get_from_part = part.ranges; - - need_marks -= marks_in_part; - sum_marks_in_parts.pop_back(); - } - else - { - /// Loop through ranges in part. Take enough ranges to cover "need_marks". - while (need_marks > 0) - { - if (part.ranges.empty()) - throw Exception("Unexpected end of ranges while spreading marks among streams", ErrorCodes::LOGICAL_ERROR); - - MarkRange & range = part.ranges.front(); - - const size_t marks_in_range = range.end - range.begin; - const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); - - ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); - range.begin += marks_to_get_from_range; - marks_in_part -= marks_to_get_from_range; - need_marks -= marks_to_get_from_range; - if (range.begin == range.end) - part.ranges.pop_front(); - } - parts.emplace_back(part); - } - ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); - new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); - } - - ReadFromMergeTree::Settings step_settings - { - .max_block_size = max_block_size, - .preferred_block_size_bytes = settings.preferred_block_size_bytes, - .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - .min_marks_for_concurrent_read = min_marks_for_concurrent_read, - .use_uncompressed_cache = use_uncompressed_cache, - .reader_settings = reader_settings, - .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - }; - - auto read_type = input_order_info->direction == 1 - ? ReadFromMergeTree::ReadType::InOrder - : ReadFromMergeTree::ReadType::InReverseOrder; - - auto plan = std::make_unique(); - auto step = std::make_unique( - data, - metadata_snapshot, - query_id, - column_names, - std::move(new_parts), - std::move(index_stats), - query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - virt_columns, - step_settings, - num_streams, - read_type); - - plan->addStep(std::move(step)); - plans.emplace_back(std::move(plan)); - } - - if (need_preliminary_merge) - { - SortDescription sort_description; - for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) - sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], - input_order_info->direction, 1); - - for (auto & plan : plans) - { - /// Drop temporary columns, added by 'sorting_key_prefix_expr' - out_projection = createProjection(plan->getCurrentDataStream().header); - - auto expression_step = std::make_unique( - plan->getCurrentDataStream(), - sorting_key_prefix_expr); - - expression_step->setStepDescription("Calculate sorting key prefix"); - plan->addStep(std::move(expression_step)); - - auto merging_sorted = std::make_unique( - plan->getCurrentDataStream(), - sort_description, - max_block_size); - - merging_sorted->setStepDescription("Merge sorting mark ranges"); - plan->addStep(std::move(merging_sorted)); - } - } - - if (plans.size() == 1) - return std::move(plans.front()); - - DataStreams input_streams; - for (const auto & plan : plans) - input_streams.emplace_back(plan->getCurrentDataStream()); - - auto union_step = std::make_unique(std::move(input_streams)); - - auto plan = std::make_unique(); - plan->unitePlans(std::move(union_step), std::move(plans)); - - return plan; -} - - -QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection, - const String & query_id) const -{ - const auto data_settings = data.getSettings(); - size_t sum_marks = 0; - size_t adaptive_parts = 0; - for (const auto & part : parts) - { - for (const auto & range : part.ranges) - sum_marks += range.end - range.begin; - - if (part.data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } - - size_t index_granularity_bytes = 0; - if (adaptive_parts >= parts.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; - - if (num_streams > settings.max_final_threads) - num_streams = settings.max_final_threads; - - /// If setting do_not_merge_across_partitions_select_final is true than we won't merge parts from different partitions. - /// We have all parts in parts vector, where parts with same partition are nearby. - /// So we will store iterators pointed to the beginning of each partition range (and parts.end()), - /// then we will create a pipe for each partition that will run selecting processor and merging processor - /// for the parts with this partition. In the end we will unite all the pipes. - std::vector parts_to_merge_ranges; - auto it = parts.begin(); - parts_to_merge_ranges.push_back(it); - - if (settings.do_not_merge_across_partitions_select_final) - { - while (it != parts.end()) - { - it = std::find_if( - it, parts.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); - parts_to_merge_ranges.push_back(it); - } - /// We divide threads for each partition equally. But we will create at least the number of partitions threads. - /// (So, the total number of threads could be more than initial num_streams. - num_streams /= (parts_to_merge_ranges.size() - 1); - } - else - { - /// If do_not_merge_across_partitions_select_final is false we just merge all the parts. - parts_to_merge_ranges.push_back(parts.end()); - } - - std::vector partition_plans; - - /// If do_not_merge_across_partitions_select_final is true and num_streams > 1 - /// we will store lonely parts with level > 0 to use parallel select on them. - std::vector lonely_parts; - size_t total_rows_in_lonely_parts = 0; - size_t sum_marks_in_lonely_parts = 0; - - for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) - { - QueryPlanPtr plan; - - { - RangesInDataParts new_parts; - - /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition - /// with level > 0 then we won't postprocess this part and if num_streams > 1 we - /// can use parallel select on such parts. We save such parts in one vector and then use - /// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select. - if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final && - std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && - parts_to_merge_ranges[range_index]->data_part->info.level > 0) - { - total_rows_in_lonely_parts += parts_to_merge_ranges[range_index]->getRowsCount(); - sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount(); - lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index])); - continue; - } - else - { - for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) - { - new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); - } - } - - if (new_parts.empty()) - continue; - - ReadFromMergeTree::Settings step_settings - { - .max_block_size = max_block_size, - .preferred_block_size_bytes = settings.preferred_block_size_bytes, - .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - .min_marks_for_concurrent_read = 0, /// this setting is not used for reading in order - .use_uncompressed_cache = use_uncompressed_cache, - .reader_settings = reader_settings, - .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - }; - - plan = std::make_unique(); - auto step = std::make_unique( - data, - metadata_snapshot, - query_id, - column_names, - std::move(new_parts), - std::move(index_stats), - query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - virt_columns, - step_settings, - num_streams, - ReadFromMergeTree::ReadType::InOrder); - - plan->addStep(std::move(step)); - - /// Drop temporary columns, added by 'sorting_key_expr' - if (!out_projection) - out_projection = createProjection(plan->getCurrentDataStream().header); - } - - auto expression_step = std::make_unique( - plan->getCurrentDataStream(), - metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); - - expression_step->setStepDescription("Calculate sorting key expression"); - plan->addStep(std::move(expression_step)); - - /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition - /// with level > 0 then we won't postprocess this part - if (settings.do_not_merge_across_partitions_select_final && - std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && - parts_to_merge_ranges[range_index]->data_part->info.level > 0) - { - partition_plans.emplace_back(std::move(plan)); - continue; - } - - Names sort_columns = metadata_snapshot->getSortingKeyColumns(); - SortDescription sort_description; - size_t sort_columns_size = sort_columns.size(); - sort_description.reserve(sort_columns_size); - - Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; - - const auto & header = plan->getCurrentDataStream().header; - for (size_t i = 0; i < sort_columns_size; ++i) - sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); - - auto final_step = std::make_unique( - plan->getCurrentDataStream(), - std::min(num_streams, settings.max_final_threads), - sort_description, - data.merging_params, - partition_key_columns, - max_block_size); - - final_step->setStepDescription("Merge rows for FINAL"); - plan->addStep(std::move(final_step)); - - partition_plans.emplace_back(std::move(plan)); - } - - if (!lonely_parts.empty()) - { - RangesInDataParts new_parts; - - size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size(); - - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( - settings.merge_tree_min_rows_for_concurrent_read, - settings.merge_tree_min_bytes_for_concurrent_read, - data_settings->index_granularity, - index_granularity_bytes, - sum_marks_in_lonely_parts); - - /// Reduce the number of num_streams_for_lonely_parts if the data is small. - if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) - num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); - - ReadFromMergeTree::Settings step_settings - { - .max_block_size = max_block_size, - .preferred_block_size_bytes = settings.preferred_block_size_bytes, - .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - .min_marks_for_concurrent_read = min_marks_for_concurrent_read, - .use_uncompressed_cache = use_uncompressed_cache, - .reader_settings = reader_settings, - .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - }; - - auto plan = std::make_unique(); - auto step = std::make_unique( - data, - metadata_snapshot, - query_id, - column_names, - std::move(lonely_parts), - std::move(index_stats), - query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - virt_columns, - step_settings, - num_streams_for_lonely_parts, - ReadFromMergeTree::ReadType::Default); - - plan->addStep(std::move(step)); - - /// Drop temporary columns, added by 'sorting_key_expr' - if (!out_projection) - out_projection = createProjection(plan->getCurrentDataStream().header); - - auto expression_step = std::make_unique( - plan->getCurrentDataStream(), - metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); - - expression_step->setStepDescription("Calculate sorting key expression"); - plan->addStep(std::move(expression_step)); - - partition_plans.emplace_back(std::move(plan)); - } - - if (partition_plans.empty()) - return {}; - - if (partition_plans.size() == 1) - return std::move(partition_plans.front()); - - auto result_header = partition_plans.front()->getCurrentDataStream().header; - DataStreams input_streams; - for (const auto & partition_plan : partition_plans) - input_streams.push_back(partition_plan->getCurrentDataStream()); - - auto union_step = std::make_unique(std::move(input_streams), result_header); - union_step->setStepDescription("Unite sources after FINAL"); - QueryPlanPtr plan = std::make_unique(); - plan->unitePlans(std::move(union_step), std::move(partition_plans)); - return plan; -} +// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( +// RangesInDataParts && parts, +// // ReadFromMergeTree::IndexStatPtr index_stats, +// size_t num_streams, +// const Names & column_names, +// const StorageMetadataPtr & metadata_snapshot, +// UInt64 max_block_size, +// bool use_uncompressed_cache, +// const SelectQueryInfo & query_info, +// const Names & virt_columns, +// const Settings & settings, +// const MergeTreeReaderSettings & reader_settings, +// const String & query_id) const +// { +// /// Count marks for each part. +// std::vector sum_marks_in_parts(parts.size()); +// size_t sum_marks = 0; +// size_t total_rows = 0; + +// const auto data_settings = data.getSettings(); +// size_t adaptive_parts = 0; +// for (size_t i = 0; i < parts.size(); ++i) +// { +// total_rows += parts[i].getRowsCount(); +// sum_marks_in_parts[i] = parts[i].getMarksCount(); +// sum_marks += sum_marks_in_parts[i]; + +// if (parts[i].data_part->index_granularity_info.is_adaptive) +// ++adaptive_parts; +// } + +// size_t index_granularity_bytes = 0; +// if (adaptive_parts > parts.size() / 2) +// index_granularity_bytes = data_settings->index_granularity_bytes; + +// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( +// settings.merge_tree_max_rows_to_use_cache, +// settings.merge_tree_max_bytes_to_use_cache, +// data_settings->index_granularity, +// index_granularity_bytes); + +// const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( +// settings.merge_tree_min_rows_for_concurrent_read, +// settings.merge_tree_min_bytes_for_concurrent_read, +// data_settings->index_granularity, +// index_granularity_bytes, +// sum_marks); + +// if (sum_marks > max_marks_to_use_cache) +// use_uncompressed_cache = false; + +// if (0 == sum_marks) +// return {}; + +// ReadFromMergeTree::Settings step_settings +// { +// .max_block_size = max_block_size, +// .preferred_block_size_bytes = settings.preferred_block_size_bytes, +// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, +// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, +// .use_uncompressed_cache = use_uncompressed_cache, +// .reader_settings = reader_settings, +// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), +// }; + +// if (num_streams > 1) +// { +// /// Reduce the number of num_streams if the data is small. +// if (sum_marks < num_streams * min_marks_for_concurrent_read && parts.size() < num_streams) +// num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts.size()); +// } + +// auto plan = std::make_unique(); +// auto step = std::make_unique( +// data, +// metadata_snapshot, +// query_id, +// column_names, +// std::move(parts), +// // std::move(index_stats), +// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, +// virt_columns, +// step_settings, +// num_streams, +// ReadFromMergeTree::ReadType::Default); + +// plan->addStep(std::move(step)); +// return plan; +// } + +// static ActionsDAGPtr createProjection(const Block & header) +// { +// auto projection = std::make_shared(header.getNamesAndTypesList()); +// projection->removeUnusedActions(header.getNames()); +// projection->projectInput(); +// return projection; +// } + +// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( +// RangesInDataParts && parts, +// // ReadFromMergeTree::IndexStatPtr index_stats, +// size_t num_streams, +// const Names & column_names, +// const StorageMetadataPtr & metadata_snapshot, +// UInt64 max_block_size, +// bool use_uncompressed_cache, +// const SelectQueryInfo & query_info, +// const ActionsDAGPtr & sorting_key_prefix_expr, +// const Names & virt_columns, +// const Settings & settings, +// const MergeTreeReaderSettings & reader_settings, +// ActionsDAGPtr & out_projection, +// const String & query_id, +// const InputOrderInfoPtr & input_order_info) const +// { +// size_t sum_marks = 0; +// size_t adaptive_parts = 0; +// std::vector sum_marks_in_parts(parts.size()); +// const auto data_settings = data.getSettings(); + +// for (size_t i = 0; i < parts.size(); ++i) +// { +// sum_marks_in_parts[i] = parts[i].getMarksCount(); +// sum_marks += sum_marks_in_parts[i]; + +// if (parts[i].data_part->index_granularity_info.is_adaptive) +// ++adaptive_parts; +// } + +// size_t index_granularity_bytes = 0; +// if (adaptive_parts > parts.size() / 2) +// index_granularity_bytes = data_settings->index_granularity_bytes; + +// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( +// settings.merge_tree_max_rows_to_use_cache, +// settings.merge_tree_max_bytes_to_use_cache, +// data_settings->index_granularity, +// index_granularity_bytes); + +// const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( +// settings.merge_tree_min_rows_for_concurrent_read, +// settings.merge_tree_min_bytes_for_concurrent_read, +// data_settings->index_granularity, +// index_granularity_bytes, +// sum_marks); + +// if (sum_marks > max_marks_to_use_cache) +// use_uncompressed_cache = false; + +// Pipes res; + +// if (sum_marks == 0) +// return {}; + +// /// Let's split ranges to avoid reading much data. +// auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size](const auto & ranges, int direction) +// { +// MarkRanges new_ranges; +// const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; +// size_t marks_in_range = 1; + +// if (direction == 1) +// { +// /// Split first few ranges to avoid reading much data. +// bool split = false; +// for (auto range : ranges) +// { +// while (!split && range.begin + marks_in_range < range.end) +// { +// new_ranges.emplace_back(range.begin, range.begin + marks_in_range); +// range.begin += marks_in_range; +// marks_in_range *= 2; + +// if (marks_in_range > max_marks_in_range) +// split = true; +// } +// new_ranges.emplace_back(range.begin, range.end); +// } +// } +// else +// { +// /// Split all ranges to avoid reading much data, because we have to +// /// store whole range in memory to reverse it. +// for (auto it = ranges.rbegin(); it != ranges.rend(); ++it) +// { +// auto range = *it; +// while (range.begin + marks_in_range < range.end) +// { +// new_ranges.emplace_front(range.end - marks_in_range, range.end); +// range.end -= marks_in_range; +// marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); +// } +// new_ranges.emplace_front(range.begin, range.end); +// } +// } + +// return new_ranges; +// }; + +// const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; +// bool need_preliminary_merge = (parts.size() > settings.read_in_order_two_level_merge_threshold); + +// std::vector plans; + +// for (size_t i = 0; i < num_streams && !parts.empty(); ++i) +// { +// size_t need_marks = min_marks_per_stream; +// RangesInDataParts new_parts; + +// /// Loop over parts. +// /// We will iteratively take part or some subrange of a part from the back +// /// and assign a stream to read from it. +// while (need_marks > 0 && !parts.empty()) +// { +// RangesInDataPart part = parts.back(); +// parts.pop_back(); + +// size_t & marks_in_part = sum_marks_in_parts.back(); + +// /// We will not take too few rows from a part. +// if (marks_in_part >= min_marks_for_concurrent_read && +// need_marks < min_marks_for_concurrent_read) +// need_marks = min_marks_for_concurrent_read; + +// /// Do not leave too few rows in the part. +// if (marks_in_part > need_marks && +// marks_in_part - need_marks < min_marks_for_concurrent_read) +// need_marks = marks_in_part; + +// MarkRanges ranges_to_get_from_part; + +// /// We take the whole part if it is small enough. +// if (marks_in_part <= need_marks) +// { +// ranges_to_get_from_part = part.ranges; + +// need_marks -= marks_in_part; +// sum_marks_in_parts.pop_back(); +// } +// else +// { +// /// Loop through ranges in part. Take enough ranges to cover "need_marks". +// while (need_marks > 0) +// { +// if (part.ranges.empty()) +// throw Exception("Unexpected end of ranges while spreading marks among streams", ErrorCodes::LOGICAL_ERROR); + +// MarkRange & range = part.ranges.front(); + +// const size_t marks_in_range = range.end - range.begin; +// const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); + +// ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); +// range.begin += marks_to_get_from_range; +// marks_in_part -= marks_to_get_from_range; +// need_marks -= marks_to_get_from_range; +// if (range.begin == range.end) +// part.ranges.pop_front(); +// } +// parts.emplace_back(part); +// } +// ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); +// new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); +// } + +// ReadFromMergeTree::Settings step_settings +// { +// .max_block_size = max_block_size, +// .preferred_block_size_bytes = settings.preferred_block_size_bytes, +// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, +// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, +// .use_uncompressed_cache = use_uncompressed_cache, +// .reader_settings = reader_settings, +// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), +// }; + +// auto read_type = input_order_info->direction == 1 +// ? ReadFromMergeTree::ReadType::InOrder +// : ReadFromMergeTree::ReadType::InReverseOrder; + +// auto plan = std::make_unique(); +// auto step = std::make_unique( +// data, +// metadata_snapshot, +// query_id, +// column_names, +// std::move(new_parts), +// // std::move(index_stats), +// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, +// virt_columns, +// step_settings, +// num_streams, +// read_type); + +// plan->addStep(std::move(step)); +// plans.emplace_back(std::move(plan)); +// } + +// if (need_preliminary_merge) +// { +// SortDescription sort_description; +// for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) +// sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], +// input_order_info->direction, 1); + +// for (auto & plan : plans) +// { +// /// Drop temporary columns, added by 'sorting_key_prefix_expr' +// out_projection = createProjection(plan->getCurrentDataStream().header); + +// auto expression_step = std::make_unique( +// plan->getCurrentDataStream(), +// sorting_key_prefix_expr); + +// expression_step->setStepDescription("Calculate sorting key prefix"); +// plan->addStep(std::move(expression_step)); + +// auto merging_sorted = std::make_unique( +// plan->getCurrentDataStream(), +// sort_description, +// max_block_size); + +// merging_sorted->setStepDescription("Merge sorting mark ranges"); +// plan->addStep(std::move(merging_sorted)); +// } +// } + +// if (plans.size() == 1) +// return std::move(plans.front()); + +// DataStreams input_streams; +// for (const auto & plan : plans) +// input_streams.emplace_back(plan->getCurrentDataStream()); + +// auto union_step = std::make_unique(std::move(input_streams)); + +// auto plan = std::make_unique(); +// plan->unitePlans(std::move(union_step), std::move(plans)); + +// return plan; +// } + + +// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( +// RangesInDataParts && parts, +// size_t num_streams, +// const Names & column_names, +// const StorageMetadataPtr & metadata_snapshot, +// UInt64 max_block_size, +// bool use_uncompressed_cache, +// const SelectQueryInfo & query_info, +// const Names & virt_columns, +// const Settings & settings, +// const MergeTreeReaderSettings & reader_settings, +// ActionsDAGPtr & out_projection, +// const String & query_id) const +// { +// const auto data_settings = data.getSettings(); +// size_t sum_marks = 0; +// size_t adaptive_parts = 0; +// for (const auto & part : parts) +// { +// for (const auto & range : part.ranges) +// sum_marks += range.end - range.begin; + +// if (part.data_part->index_granularity_info.is_adaptive) +// ++adaptive_parts; +// } + +// size_t index_granularity_bytes = 0; +// if (adaptive_parts >= parts.size() / 2) +// index_granularity_bytes = data_settings->index_granularity_bytes; + +// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( +// settings.merge_tree_max_rows_to_use_cache, +// settings.merge_tree_max_bytes_to_use_cache, +// data_settings->index_granularity, +// index_granularity_bytes); + +// if (sum_marks > max_marks_to_use_cache) +// use_uncompressed_cache = false; + +// if (num_streams > settings.max_final_threads) +// num_streams = settings.max_final_threads; + +// /// If setting do_not_merge_across_partitions_select_final is true than we won't merge parts from different partitions. +// /// We have all parts in parts vector, where parts with same partition are nearby. +// /// So we will store iterators pointed to the beginning of each partition range (and parts.end()), +// /// then we will create a pipe for each partition that will run selecting processor and merging processor +// /// for the parts with this partition. In the end we will unite all the pipes. +// std::vector parts_to_merge_ranges; +// auto it = parts.begin(); +// parts_to_merge_ranges.push_back(it); + +// if (settings.do_not_merge_across_partitions_select_final) +// { +// while (it != parts.end()) +// { +// it = std::find_if( +// it, parts.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); +// parts_to_merge_ranges.push_back(it); +// } +// /// We divide threads for each partition equally. But we will create at least the number of partitions threads. +// /// (So, the total number of threads could be more than initial num_streams. +// num_streams /= (parts_to_merge_ranges.size() - 1); +// } +// else +// { +// /// If do_not_merge_across_partitions_select_final is false we just merge all the parts. +// parts_to_merge_ranges.push_back(parts.end()); +// } + +// std::vector partition_plans; + +// /// If do_not_merge_across_partitions_select_final is true and num_streams > 1 +// /// we will store lonely parts with level > 0 to use parallel select on them. +// std::vector lonely_parts; +// size_t total_rows_in_lonely_parts = 0; +// size_t sum_marks_in_lonely_parts = 0; + +// for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) +// { +// QueryPlanPtr plan; + +// { +// RangesInDataParts new_parts; + +// /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition +// /// with level > 0 then we won't postprocess this part and if num_streams > 1 we +// /// can use parallel select on such parts. We save such parts in one vector and then use +// /// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select. +// if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final && +// std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && +// parts_to_merge_ranges[range_index]->data_part->info.level > 0) +// { +// total_rows_in_lonely_parts += parts_to_merge_ranges[range_index]->getRowsCount(); +// sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount(); +// lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index])); +// continue; +// } +// else +// { +// for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) +// { +// new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); +// } +// } + +// if (new_parts.empty()) +// continue; + +// ReadFromMergeTree::Settings step_settings +// { +// .max_block_size = max_block_size, +// .preferred_block_size_bytes = settings.preferred_block_size_bytes, +// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, +// .min_marks_for_concurrent_read = 0, /// this setting is not used for reading in order +// .use_uncompressed_cache = use_uncompressed_cache, +// .reader_settings = reader_settings, +// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), +// }; + +// plan = std::make_unique(); +// auto step = std::make_unique( +// data, +// metadata_snapshot, +// query_id, +// column_names, +// std::move(new_parts), +// // std::move(index_stats), +// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, +// virt_columns, +// step_settings, +// num_streams, +// ReadFromMergeTree::ReadType::InOrder); + +// plan->addStep(std::move(step)); + +// /// Drop temporary columns, added by 'sorting_key_expr' +// if (!out_projection) +// out_projection = createProjection(plan->getCurrentDataStream().header); +// } + +// auto expression_step = std::make_unique( +// plan->getCurrentDataStream(), +// metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + +// expression_step->setStepDescription("Calculate sorting key expression"); +// plan->addStep(std::move(expression_step)); + +// /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition +// /// with level > 0 then we won't postprocess this part +// if (settings.do_not_merge_across_partitions_select_final && +// std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && +// parts_to_merge_ranges[range_index]->data_part->info.level > 0) +// { +// partition_plans.emplace_back(std::move(plan)); +// continue; +// } + +// Names sort_columns = metadata_snapshot->getSortingKeyColumns(); +// SortDescription sort_description; +// size_t sort_columns_size = sort_columns.size(); +// sort_description.reserve(sort_columns_size); + +// Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; + +// const auto & header = plan->getCurrentDataStream().header; +// for (size_t i = 0; i < sort_columns_size; ++i) +// sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); + +// auto final_step = std::make_unique( +// plan->getCurrentDataStream(), +// std::min(num_streams, settings.max_final_threads), +// sort_description, +// data.merging_params, +// partition_key_columns, +// max_block_size); + +// final_step->setStepDescription("Merge rows for FINAL"); +// plan->addStep(std::move(final_step)); + +// partition_plans.emplace_back(std::move(plan)); +// } + +// if (!lonely_parts.empty()) +// { +// RangesInDataParts new_parts; + +// size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size(); + +// const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( +// settings.merge_tree_min_rows_for_concurrent_read, +// settings.merge_tree_min_bytes_for_concurrent_read, +// data_settings->index_granularity, +// index_granularity_bytes, +// sum_marks_in_lonely_parts); + +// /// Reduce the number of num_streams_for_lonely_parts if the data is small. +// if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) +// num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); + +// ReadFromMergeTree::Settings step_settings +// { +// .max_block_size = max_block_size, +// .preferred_block_size_bytes = settings.preferred_block_size_bytes, +// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, +// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, +// .use_uncompressed_cache = use_uncompressed_cache, +// .reader_settings = reader_settings, +// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), +// }; + +// auto plan = std::make_unique(); +// auto step = std::make_unique( +// data, +// metadata_snapshot, +// query_id, +// column_names, +// std::move(lonely_parts), +// // std::move(index_stats), +// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, +// virt_columns, +// step_settings, +// num_streams_for_lonely_parts, +// ReadFromMergeTree::ReadType::Default); + +// plan->addStep(std::move(step)); + +// /// Drop temporary columns, added by 'sorting_key_expr' +// if (!out_projection) +// out_projection = createProjection(plan->getCurrentDataStream().header); + +// auto expression_step = std::make_unique( +// plan->getCurrentDataStream(), +// metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + +// expression_step->setStepDescription("Calculate sorting key expression"); +// plan->addStep(std::move(expression_step)); + +// partition_plans.emplace_back(std::move(plan)); +// } + +// if (partition_plans.empty()) +// return {}; + +// if (partition_plans.size() == 1) +// return std::move(partition_plans.front()); + +// auto result_header = partition_plans.front()->getCurrentDataStream().header; +// DataStreams input_streams; +// for (const auto & partition_plan : partition_plans) +// input_streams.push_back(partition_plan->getCurrentDataStream()); + +// auto union_step = std::make_unique(std::move(input_streams), result_header); +// union_step->setStepDescription("Unite sources after FINAL"); +// QueryPlanPtr plan = std::make_unique(); +// plan->unitePlans(std::move(union_step), std::move(partition_plans)); +// return plan; +// } /// Calculates a set of mark ranges, that could possibly contain keys, required by condition. /// In other words, it removes subranges from whole range, that definitely could not contain required keys. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 7597af2e173..3e8076de8d3 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -17,19 +17,22 @@ struct MergeTreeDataSelectSamplingData { bool use_sampling = false; bool read_nothing = false; + Float64 used_sample_factor = 1.0; std::shared_ptr filter_function; ActionsDAGPtr filter_expression; }; -struct MergeTreeDataSelectCache -{ - RangesInDataParts parts_with_ranges; - MergeTreeDataSelectSamplingData sampling; - std::unique_ptr index_stats; - size_t sum_marks = 0; - size_t sum_ranges = 0; - bool use_cache = false; -}; +// struct MergeTreeDataSelectCache +// { +// RangesInDataParts parts_with_ranges; +// MergeTreeDataSelectSamplingData sampling; +// std::unique_ptr index_stats; +// size_t sum_marks = 0; +// size_t sum_ranges = 0; +// bool use_cache = false; +// }; + +using PartitionIdToMaxBlock = std::unordered_map; /** Executes SELECT queries on data from the merge tree. */ @@ -41,7 +44,6 @@ public: /** When reading, selects a set of parts that covers the desired range of the index. * max_blocks_number_to_read - if not nullptr, do not read all the parts whose right border is greater than max_block in partition. */ - using PartitionIdToMaxBlock = std::unordered_map; QueryPlanPtr read( const Names & column_names, @@ -53,6 +55,16 @@ public: QueryProcessingStage::Enum processed_stage, const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; + size_t estimateNumMarksToRead( + MergeTreeData::DataPartsVector parts, + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot_base, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + ContextPtr context, + unsigned num_streams, + const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; + QueryPlanPtr readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names, @@ -62,57 +74,56 @@ public: ContextPtr context, UInt64 max_block_size, unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr, - MergeTreeDataSelectCache * cache = nullptr) const; + const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; private: const MergeTreeData & data; Poco::Logger * log; - QueryPlanPtr spreadMarkRangesAmongStreams( - RangesInDataParts && parts, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - const String & query_id) const; + // QueryPlanPtr spreadMarkRangesAmongStreams( + // RangesInDataParts && parts, + // size_t num_streams, + // const Names & column_names, + // const StorageMetadataPtr & metadata_snapshot, + // UInt64 max_block_size, + // bool use_uncompressed_cache, + // const SelectQueryInfo & query_info, + // const Names & virt_columns, + // const Settings & settings, + // const MergeTreeReaderSettings & reader_settings, + // const String & query_id) const; - /// out_projection - save projection only with columns, requested to read - QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder( - RangesInDataParts && parts, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const ActionsDAGPtr & sorting_key_prefix_expr, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection, - const String & query_id, - const InputOrderInfoPtr & input_order_info) const; + // /// out_projection - save projection only with columns, requested to read + // QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder( + // RangesInDataParts && parts, + // size_t num_streams, + // const Names & column_names, + // const StorageMetadataPtr & metadata_snapshot, + // UInt64 max_block_size, + // bool use_uncompressed_cache, + // const SelectQueryInfo & query_info, + // const ActionsDAGPtr & sorting_key_prefix_expr, + // const Names & virt_columns, + // const Settings & settings, + // const MergeTreeReaderSettings & reader_settings, + // ActionsDAGPtr & out_projection, + // const String & query_id, + // const InputOrderInfoPtr & input_order_info) const; - QueryPlanPtr spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection, - const String & query_id) const; + // QueryPlanPtr spreadMarkRangesAmongStreamsFinal( + // RangesInDataParts && parts, + // size_t num_streams, + // const Names & column_names, + // const StorageMetadataPtr & metadata_snapshot, + // UInt64 max_block_size, + // bool use_uncompressed_cache, + // const SelectQueryInfo & query_info, + // const Names & virt_columns, + // const Settings & settings, + // const MergeTreeReaderSettings & reader_settings, + // ActionsDAGPtr & out_projection, + // const String & query_id) const; /// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index. static size_t getApproximateTotalRowsToRead( @@ -140,7 +151,6 @@ private: size_t & granules_dropped, Poco::Logger * log); -public: struct PartFilterCounters { size_t num_initial_selected_parts = 0; @@ -175,16 +185,36 @@ public: PartFilterCounters & counters, Poco::Logger * log); - static RangesInDataParts filterParts( +public: + static std::optional> filterPartsByVirtualColumns( + const MergeTreeData & data, MergeTreeData::DataPartsVector & parts, + const ASTPtr & query, + ContextPtr context); + + static void filterPartsByPartition( + const StorageMetadataPtr & metadata_snapshot, + const MergeTreeData & data, + const SelectQueryInfo & query_info, + ContextPtr & context, + ContextPtr & query_context, + MergeTreeData::DataPartsVector & parts, + const std::optional> & part_values, + const PartitionIdToMaxBlock * max_block_numbers_to_read, + Poco::Logger * log, + ReadFromMergeTree::IndexStats & index_stats); + + static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes( + MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, - SelectQueryInfo & query_info, + const SelectQueryInfo & query_info, ContextPtr & context, KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, - ReadFromMergeTree::IndexStats & index_stats); + ReadFromMergeTree::IndexStats & index_stats, + bool use_skip_indexes); static MergeTreeDataSelectSamplingData getSampling( const ASTSelectQuery & select, @@ -197,7 +227,7 @@ public: NamesAndTypesList available_real_columns, ContextPtr context); - static String checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context); + static String checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context); }; } diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index e8b39c8e28c..dff864bc58c 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -41,10 +41,7 @@ public: query_info, context, max_block_size, - num_streams, - nullptr, - query_info.projection ? query_info.projection->merge_tree_data_select_base_cache.get() - : query_info.merge_tree_data_select_cache.get())); + num_streams)); return query_plan.convertToPipe( QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index afed41189c2..73cf3893a89 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -137,8 +137,8 @@ struct ProjectionCandidate ReadInOrderOptimizerPtr order_optimizer; InputOrderInfoPtr input_order_info; ManyExpressionActions group_by_elements_actions; - std::shared_ptr merge_tree_data_select_base_cache; - std::shared_ptr merge_tree_data_select_projection_cache; + // std::shared_ptr merge_tree_data_select_base_cache; + // std::shared_ptr merge_tree_data_select_projection_cache; }; /** Query along with some additional data, From 0d393c000617b955b521ef3d8f8b6bc7a1a864ca Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 27 May 2021 17:16:42 +0300 Subject: [PATCH 096/154] Fix tests --- programs/client/Client.cpp | 2 +- src/Formats/FormatSchemaInfo.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 2de95635d04..7dea4c9375b 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -279,7 +279,7 @@ private: /// Set path for format schema files if (config().has("format_schema_path")) - context->setFormatSchemaPath(config().getString("format_schema_path")); + context->setFormatSchemaPath(fs::absolute(config().getString("format_schema_path"))); /// Initialize query_id_formats if any if (config().has("query_id_formats")) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 54e0961d161..25ed6051382 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -89,7 +89,7 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (!path_is_subdirectory_of(path, default_schema_directory_path)) + else if (path.has_parent_path() && !path_is_subdirectory_of(path, default_schema_directory_path)) { if (is_server) throw Exception( From 1aeb705b20ef37f0608e6c639dc333b90388bdb4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 27 May 2021 19:53:58 +0300 Subject: [PATCH 097/154] Fix some tests. --- .../QueryPlan/ReadFromMergeTree.cpp | 165 +++++++++++------- src/Processors/QueryPlan/ReadFromMergeTree.h | 8 +- src/Storages/MergeTree/MergeTreeData.cpp | 12 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 20 ++- .../MergeTree/MergeTreeDataSelectExecutor.h | 14 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- .../0_stateless/01651_bugs_from_15889.sql | 10 +- .../01861_explain_pipeline.reference | 10 +- 8 files changed, 141 insertions(+), 102 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6d6ee43acb3..2ad10461613 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -88,9 +88,20 @@ size_t minMarksForConcurrentRead( } +struct ReadFromMergeTree::AnalysisResult +{ + RangesInDataParts parts_with_ranges; + MergeTreeDataSelectSamplingData sampling; + bool sample_factor_column_queried = false; + String query_id; + IndexStats index_stats; + Names column_names_to_read; + ReadFromMergeTree::ReadType read_type = ReadFromMergeTree::ReadType::Default; +}; + ReadFromMergeTree::ReadFromMergeTree( const SelectQueryInfo & query_info_, - const PartitionIdToMaxBlock * max_block_numbers_to_read_, + std::shared_ptr max_block_numbers_to_read_, ContextPtr context_, const MergeTreeData & data_, StorageMetadataPtr metadata_snapshot_, @@ -107,7 +118,7 @@ ReadFromMergeTree::ReadFromMergeTree( data_.getPartitionValueType(), virt_column_names_)}) , query_info(std::move(query_info_)) - , max_block_numbers_to_read(max_block_numbers_to_read_) + , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) , context(std::move(context_)) , data(data_) , metadata_snapshot(std::move(metadata_snapshot_)) @@ -842,26 +853,26 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( return Pipe::unitePipes(std::move(partition_pipes)); } -void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTreeData::DataPartsVector parts) const { - auto parts = std::move(prepared_parts); + AnalysisResult result; + size_t total_parts = parts.size(); auto part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context); if (part_values && part_values->empty()) - { - pipeline.init(Pipe(std::make_shared(getOutputStream().header))); - return; - } + return result; + + result.column_names_to_read = real_column_names; /// If there are only virtual columns in the query, you must request at least one non-virtual one. - if (real_column_names.empty()) + if (result.column_names_to_read.empty()) { NamesAndTypesList available_real_columns = metadata_snapshot->getColumns().getAllPhysical(); - real_column_names.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); + result.column_names_to_read.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); } - metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); + metadata_snapshot->check(result.column_names_to_read, data.getVirtuals(), data.getStorageID()); // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); @@ -881,28 +892,26 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; MergeTreeDataSelectExecutor::filterPartsByPartition( - metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read, log, index_stats); + metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read.get(), log, result.index_stats); - bool sample_factor_column_queried = false; for (const auto & col : virt_column_names) if (col == "_sample_factor") - sample_factor_column_queried = true; + result.sample_factor_column_queried = true; - auto sampling = MergeTreeDataSelectExecutor::getSampling( + result.sampling = MergeTreeDataSelectExecutor::getSampling( select, parts, metadata_snapshot, key_condition, - data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); + data, log, result.sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); - if (sampling.read_nothing) - { - pipeline.init(Pipe(std::make_shared(getOutputStream().header))); - return; - } + if (result.sampling.read_nothing) + return result; size_t total_marks_pk = 0; for (const auto & part : parts) total_marks_pk += part->index_granularity.getMarksCountWithoutFinal(); - auto parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( + size_t parts_before_pk = parts.size(); + + result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( std::move(parts), metadata_snapshot, query_info, @@ -911,18 +920,18 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build settings.reader_settings, log, settings.num_streams, - index_stats, + result.index_stats, true); size_t sum_marks_pk = total_marks_pk; - for (const auto & stat : index_stats) + for (const auto & stat : result.index_stats) if (stat.type == IndexType::PrimaryKey) sum_marks_pk = stat.num_granules_after; size_t sum_marks = 0; size_t sum_ranges = 0; - for (const auto & part : parts_with_ranges) + for (const auto & part : result.parts_with_ranges) { sum_ranges += part.ranges.size(); sum_marks += part.getMarksCount(); @@ -931,31 +940,53 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build LOG_DEBUG( log, "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", - parts.size(), + parts_before_pk, total_parts, - parts_with_ranges.size(), + result.parts_with_ranges.size(), sum_marks_pk, total_marks_pk, sum_marks, sum_ranges); - String query_id = MergeTreeDataSelectExecutor::checkLimits(data, parts_with_ranges, context); + result.query_id = MergeTreeDataSelectExecutor::checkLimits(data, result.parts_with_ranges, context); - ProfileEvents::increment(ProfileEvents::SelectedParts, parts_with_ranges.size()); + ProfileEvents::increment(ProfileEvents::SelectedParts, result.parts_with_ranges.size()); ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks); + const auto & input_order_info = query_info.input_order_info + ? query_info.input_order_info + : (query_info.projection ? query_info.projection->input_order_info : nullptr); + + const auto & q_settings = context->getSettingsRef(); + if ((q_settings.optimize_read_in_order || q_settings.optimize_aggregation_in_order) && input_order_info) + result.read_type = (input_order_info->direction > 0) ? ReadType::InOrder + : ReadType::InReverseOrder; + + return result; +} + +void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +{ + auto result = selectRangesToRead(prepared_parts); + if (result.parts_with_ranges.empty()) + { + pipeline.init(Pipe(std::make_shared(getOutputStream().header))); + return; + } + /// Projection, that needed to drop columns, which have appeared by execution /// of some extra expressions, and to allow execute the same expressions later. /// NOTE: It may lead to double computation of expressions. ActionsDAGPtr result_projection; - Names column_names_to_read = real_column_names; - if (!select.final() && sampling.use_sampling) + Names column_names_to_read = std::move(result.column_names_to_read); + const auto & select = query_info.query->as(); + if (!select.final() && result.sampling.use_sampling) { /// Add columns needed for `sample_by_ast` to `column_names_to_read`. /// Skip this if final was used, because such columns were already added from PK. - std::vector add_columns = sampling.filter_expression->getRequiredColumns().getNames(); + std::vector add_columns = result.sampling.filter_expression->getRequiredColumns().getNames(); column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); std::sort(column_names_to_read.begin(), column_names_to_read.end()); column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), @@ -985,7 +1016,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end()); pipe = spreadMarkRangesAmongStreamsFinal( - std::move(parts_with_ranges), + std::move(result.parts_with_ranges), column_names_to_read, result_projection); } @@ -999,7 +1030,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActionsDAG(false); pipe = spreadMarkRangesAmongStreamsWithOrder( - std::move(parts_with_ranges), + std::move(result.parts_with_ranges), column_names_to_read, sorting_key_prefix_expr, result_projection, @@ -1008,7 +1039,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build else { pipe = spreadMarkRangesAmongStreams( - std::move(parts_with_ranges), + std::move(result.parts_with_ranges), column_names_to_read); } @@ -1018,15 +1049,15 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build return; } - if (sampling.use_sampling) + if (result.sampling.use_sampling) { - auto sampling_actions = std::make_shared(sampling.filter_expression); + auto sampling_actions = std::make_shared(result.sampling.filter_expression); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( header, sampling_actions, - sampling.filter_function->getColumnName(), + result.sampling.filter_function->getColumnName(), false); }); } @@ -1041,12 +1072,12 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build } /// 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) + if (result.sample_factor_column_queried) { ColumnWithTypeAndName column; column.name = "_sample_factor"; column.type = std::make_shared(); - column.column = column.type->createColumnConst(0, Field(sampling.used_sample_factor)); + column.column = column.type->createColumnConst(0, Field(result.sampling.used_sample_factor)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); auto adding_column_action = std::make_shared(adding_column_dag); @@ -1073,8 +1104,8 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build processors.emplace_back(processor); // Attach QueryIdHolder if needed - if (!query_id.empty()) - pipe.addQueryIdHolder(std::make_shared(query_id, data)); + if (!result.query_id.empty()) + pipe.addQueryIdHolder(std::make_shared(result.query_id, data)); pipeline.init(std::move(pipe)); } @@ -1098,45 +1129,50 @@ static const char * indexTypeToString(ReadFromMergeTree::IndexType type) __builtin_unreachable(); } -// static const char * readTypeToString(ReadFromMergeTree::ReadType type) -// { -// switch (type) -// { -// case ReadFromMergeTree::ReadType::Default: -// return "Default"; -// case ReadFromMergeTree::ReadType::InOrder: -// return "InOrder"; -// case ReadFromMergeTree::ReadType::InReverseOrder: -// return "InReverseOrder"; -// } +static const char * readTypeToString(ReadFromMergeTree::ReadType type) +{ + switch (type) + { + case ReadFromMergeTree::ReadType::Default: + return "Default"; + case ReadFromMergeTree::ReadType::InOrder: + return "InOrder"; + case ReadFromMergeTree::ReadType::InReverseOrder: + return "InReverseOrder"; + } -// __builtin_unreachable(); -// } + __builtin_unreachable(); +} void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const { + auto result = selectRangesToRead(prepared_parts); std::string prefix(format_settings.offset, format_settings.indent_char); - //format_settings.out << prefix << "ReadType: " << readTypeToString(read_type) << '\n'; + format_settings.out << prefix << "ReadType: " << readTypeToString(result.read_type) << '\n'; - if (!index_stats.empty()) + if (!result.index_stats.empty()) { - format_settings.out << prefix << "Parts: " << index_stats.back().num_parts_after << '\n'; - format_settings.out << prefix << "Granules: " << index_stats.back().num_granules_after << '\n'; + format_settings.out << prefix << "Parts: " << result.index_stats.back().num_parts_after << '\n'; + format_settings.out << prefix << "Granules: " << result.index_stats.back().num_granules_after << '\n'; } } void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { - //map.add("Read Type", readTypeToString(read_type)); - if (!index_stats.empty()) + auto result = selectRangesToRead(prepared_parts); + map.add("Read Type", readTypeToString(result.read_type)); + if (!result.index_stats.empty()) { - map.add("Parts", index_stats.back().num_parts_after); - map.add("Granules", index_stats.back().num_granules_after); + map.add("Parts", result.index_stats.back().num_parts_after); + map.add("Granules", result.index_stats.back().num_granules_after); } } void ReadFromMergeTree::describeIndexes(FormatSettings & format_settings) const { + auto result = selectRangesToRead(prepared_parts); + auto index_stats = std::move(result.index_stats); + std::string prefix(format_settings.offset, format_settings.indent_char); if (!index_stats.empty()) { @@ -1186,6 +1222,9 @@ void ReadFromMergeTree::describeIndexes(FormatSettings & format_settings) const void ReadFromMergeTree::describeIndexes(JSONBuilder::JSONMap & map) const { + auto result = selectRangesToRead(prepared_parts); + auto index_stats = std::move(result.index_stats); + if (!index_stats.empty()) { /// Do not print anything if no indexes is applied. diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index e9341e46770..ef5cc5dc70c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -72,7 +72,7 @@ public: ReadFromMergeTree( const SelectQueryInfo & query_info_, - const PartitionIdToMaxBlock * max_block_numbers_to_read_, + std::shared_ptr max_block_numbers_to_read_, ContextPtr context_, const MergeTreeData & data_, StorageMetadataPtr metadata_snapshot_, @@ -97,7 +97,7 @@ public: private: SelectQueryInfo query_info; - const PartitionIdToMaxBlock * max_block_numbers_to_read; + std::shared_ptr max_block_numbers_to_read; ContextPtr context; const MergeTreeData & data; StorageMetadataPtr metadata_snapshot; @@ -106,7 +106,6 @@ private: Names real_column_names; MergeTreeData::DataPartsVector prepared_parts; PrewhereInfoPtr prewhere_info; - IndexStats index_stats; Names virt_column_names; Settings settings; @@ -134,6 +133,9 @@ private: RangesInDataParts && parts, const Names & column_names, ActionsDAGPtr & out_projection); + + struct AnalysisResult; + AnalysisResult selectRangesToRead(MergeTreeData::DataPartsVector parts) const; }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e16bbb640e2..3f86432e2ae 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3818,7 +3818,7 @@ static void selectBestProjection( const Names & required_columns, ProjectionCandidate & candidate, ContextPtr query_context, - const PartitionIdToMaxBlock * max_added_blocks, + std::shared_ptr max_added_blocks, const Settings & settings, const MergeTreeData::DataPartsVector & parts, ProjectionCandidate *& selected_candidate, @@ -4097,11 +4097,11 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( // First build a MergeTreeDataSelectCache to check if a projection is indeed better than base // query_info.merge_tree_data_select_cache = std::make_unique(); - std::unique_ptr max_added_blocks; + std::shared_ptr max_added_blocks; if (settings.select_sequential_consistency) { if (const StorageReplicatedMergeTree * replicated = dynamic_cast(this)) - max_added_blocks = std::make_unique(replicated->getMaxAddedBlocks()); + max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); } auto parts = getDataPartsVector(); @@ -4122,7 +4122,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( analysis_result.required_columns, candidate, query_context, - max_added_blocks.get(), + max_added_blocks, settings, parts, selected_candidate, @@ -4143,7 +4143,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( query_info, // TODO syntax_analysis_result set in index query_context, settings.max_threads, - max_added_blocks.get()); + max_added_blocks); // Add 1 to base sum_marks so that we prefer projections even when they have equal number of marks to read. // NOTE: It is not clear if we need it. E.g. projections do not support skip index for now. @@ -4160,7 +4160,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( analysis_result.required_columns, candidate, query_context, - max_added_blocks.get(), + max_added_blocks, settings, parts, selected_candidate, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8a3550fc511..cb2ead7952a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -139,7 +139,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( const UInt64 max_block_size, const unsigned num_streams, QueryProcessingStage::Enum processed_stage, - const PartitionIdToMaxBlock * max_block_numbers_to_read) const + std::shared_ptr max_block_numbers_to_read) const { const auto & settings = context->getSettingsRef(); auto parts = data.getDataPartsVector(); @@ -670,8 +670,8 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const SelectQueryInfo & query_info, - ContextPtr & context, - ContextPtr & query_context, + const ContextPtr & context, + const ContextPtr & query_context, MergeTreeData::DataPartsVector & parts, const std::optional> & part_values, const PartitionIdToMaxBlock * max_block_numbers_to_read, @@ -766,7 +766,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, const SelectQueryInfo & query_info, - ContextPtr & context, + const ContextPtr & context, KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, @@ -993,7 +993,10 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd return parts_with_ranges; } -String MergeTreeDataSelectExecutor::checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context) +String MergeTreeDataSelectExecutor::checkLimits( + const MergeTreeData & data, + const RangesInDataParts & parts_with_ranges, + const ContextPtr & context) { const auto & settings = context->getSettingsRef(); // Check limitations. query_id is used as the quota RAII's resource key. @@ -1092,7 +1095,7 @@ size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead( const SelectQueryInfo & query_info, ContextPtr context, unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read) const + std::shared_ptr max_block_numbers_to_read) const { size_t total_parts = parts.size(); if (total_parts == 0) @@ -1137,7 +1140,7 @@ size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead( ReadFromMergeTree::IndexStats index_stats; filterPartsByPartition( - metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read, log, index_stats); + metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read.get(), log, index_stats); auto sampling = MergeTreeDataSelectExecutor::getSampling( select, parts, metadata_snapshot, key_condition, @@ -1173,7 +1176,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( ContextPtr context, const UInt64 max_block_size, const unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read) const + std::shared_ptr max_block_numbers_to_read) const { size_t total_parts = parts.size(); if (total_parts == 0) @@ -1207,6 +1210,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, //.min_marks_for_concurrent_read = settings.min_marks_for_concurrent_read, .use_uncompressed_cache = settings.use_uncompressed_cache, + .force_primary_key = settings.force_primary_key, .reader_settings = reader_settings, .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), }; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 3e8076de8d3..077584039a1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -53,7 +53,7 @@ public: UInt64 max_block_size, unsigned num_streams, QueryProcessingStage::Enum processed_stage, - const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; + std::shared_ptr max_block_numbers_to_read = nullptr) const; size_t estimateNumMarksToRead( MergeTreeData::DataPartsVector parts, @@ -63,7 +63,7 @@ public: const SelectQueryInfo & query_info, ContextPtr context, unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; + std::shared_ptr max_block_numbers_to_read = nullptr) const; QueryPlanPtr readFromParts( MergeTreeData::DataPartsVector parts, @@ -74,7 +74,7 @@ public: ContextPtr context, UInt64 max_block_size, unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; + std::shared_ptr max_block_numbers_to_read = nullptr) const; private: const MergeTreeData & data; @@ -196,8 +196,8 @@ public: const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const SelectQueryInfo & query_info, - ContextPtr & context, - ContextPtr & query_context, + const ContextPtr & context, + const ContextPtr & query_context, MergeTreeData::DataPartsVector & parts, const std::optional> & part_values, const PartitionIdToMaxBlock * max_block_numbers_to_read, @@ -208,7 +208,7 @@ public: MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, const SelectQueryInfo & query_info, - ContextPtr & context, + const ContextPtr & context, KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, @@ -227,7 +227,7 @@ public: NamesAndTypesList available_real_columns, ContextPtr context); - static String checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context); + static String checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, const ContextPtr & context); }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3adf3026b23..e38589a5143 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4377,9 +4377,9 @@ void StorageReplicatedMergeTree::read( */ if (local_context->getSettingsRef().select_sequential_consistency) { - auto max_added_blocks = getMaxAddedBlocks(); + auto max_added_blocks = std::make_shared(getMaxAddedBlocks()); if (auto plan = reader.read( - column_names, metadata_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage, &max_added_blocks)) + column_names, metadata_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage, std::move(max_added_blocks))) query_plan = std::move(*plan); return; } diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql index 1fbf669a1b8..2764ed61291 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.sql +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -8,7 +8,7 @@ INSERT INTO xp SELECT '2020-01-01', number, '' FROM numbers(100000); CREATE TABLE xp_d AS xp ENGINE = Distributed(test_shard_localhost, currentDatabase(), xp); -SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 20 } +SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 8 } SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); -- { serverError 53 } @@ -45,7 +45,7 @@ SYSTEM FLUSH LOGS; WITH concat(addressToLine(arrayJoin(trace) AS addr), '#') AS symbol SELECT count() > 7 FROM trace_log AS t -WHERE (query_id = +WHERE (query_id = ( SELECT [NULL, NULL, NULL, NULL, 0.00009999999747378752, NULL, NULL, NULL, NULL, NULL], @@ -60,7 +60,7 @@ WHERE (query_id = WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM trace_log AS t -WHERE greaterOrEquals(event_date, ignore(ignore(ignore(NULL, '')), 256), yesterday()) AND (trace_type = 'Memory') AND (query_id = +WHERE greaterOrEquals(event_date, ignore(ignore(ignore(NULL, '')), 256), yesterday()) AND (trace_type = 'Memory') AND (query_id = ( SELECT ignore(ignore(ignore(ignore(65536)), ignore(65537), ignore(2)), ''), @@ -82,7 +82,7 @@ WITH ( WHERE current_database = currentDatabase() ORDER BY query_start_time DESC LIMIT 1 - ) AS time_with_microseconds, + ) AS time_with_microseconds, ( SELECT inf, @@ -101,7 +101,7 @@ WITH ( WHERE current_database = currentDatabase() ORDER BY query_start_time DESC LIMIT 1 - ) AS time_with_microseconds, + ) AS time_with_microseconds, ( SELECT query_start_time FROM system.query_log diff --git a/tests/queries/0_stateless/01861_explain_pipeline.reference b/tests/queries/0_stateless/01861_explain_pipeline.reference index 8d755f807c0..9d62fb9f6b8 100644 --- a/tests/queries/0_stateless/01861_explain_pipeline.reference +++ b/tests/queries/0_stateless/01861_explain_pipeline.reference @@ -1,13 +1,10 @@ (Expression) ExpressionTransform (SettingQuotaAndLimits) - (Expression) + (ReadFromMergeTree) ExpressionTransform - (MergingFinal) ReplacingSorted 2 → 1 - (Expression) ExpressionTransform × 2 - (ReadFromMergeTree) MergeTree × 2 0 → 1 0 0 1 1 @@ -19,13 +16,10 @@ ExpressionTransform (Expression) ExpressionTransform × 2 (SettingQuotaAndLimits) - (Expression) + (ReadFromMergeTree) ExpressionTransform × 2 - (MergingFinal) ReplacingSorted × 2 2 → 1 Copy × 2 1 → 2 AddingSelector × 2 - (Expression) ExpressionTransform × 2 - (ReadFromMergeTree) MergeTree × 2 0 → 1 From 1ed00b06e091c47cdfdf6143cd0bab63d22dec62 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 27 May 2021 20:56:06 +0300 Subject: [PATCH 098/154] Fix tests. --- ...materialized_view_select_extra_columns.sql | 2 +- ...1_mergetree_read_in_order_spread.reference | 21 +++++-------------- .../01576_alias_column_rewrite.reference | 20 ++++-------------- 3 files changed, 10 insertions(+), 33 deletions(-) diff --git a/tests/queries/0_stateless/01019_materialized_view_select_extra_columns.sql b/tests/queries/0_stateless/01019_materialized_view_select_extra_columns.sql index 10bda34e6b9..4b7ea127190 100644 --- a/tests/queries/0_stateless/01019_materialized_view_select_extra_columns.sql +++ b/tests/queries/0_stateless/01019_materialized_view_select_extra_columns.sql @@ -28,7 +28,7 @@ FROM mv_extra_columns_src; INSERT INTO mv_extra_columns_src VALUES (0, 0), (1, 1), (2, 2); SELECT * FROM mv_extra_columns_dst ORDER by v; -SELECT * FROM mv_extra_columns_view; -- { serverError 16 } +SELECT * FROM mv_extra_columns_view; -- { serverError 10 } DROP TABLE mv_extra_columns_view; DROP TABLE mv_extra_columns_src; diff --git a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference index 835e2af269a..2843b305f0a 100644 --- a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference +++ b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference @@ -7,22 +7,11 @@ ExpressionTransform (Expression) ExpressionTransform × 3 (SettingQuotaAndLimits) - (Expression) - ExpressionTransform × 3 - (Union) - (MergingSorted) - (Expression) - ExpressionTransform - (ReadFromMergeTree) - MergeTree 0 → 1 - (MergingSorted) + (ReadFromMergeTree) + ExpressionTransform × 4 + MergeTree 0 → 1 MergingSortedTransform 2 → 1 - (Expression) ExpressionTransform × 2 - (ReadFromMergeTree) MergeTree × 2 0 → 1 - (MergingSorted) - (Expression) - ExpressionTransform - (ReadFromMergeTree) - MergeTree 0 → 1 + ExpressionTransform + MergeTree 0 → 1 diff --git a/tests/queries/0_stateless/01576_alias_column_rewrite.reference b/tests/queries/0_stateless/01576_alias_column_rewrite.reference index c5679544e1d..9d3db8c1d00 100644 --- a/tests/queries/0_stateless/01576_alias_column_rewrite.reference +++ b/tests/queries/0_stateless/01576_alias_column_rewrite.reference @@ -34,19 +34,13 @@ Expression (Projection) FinishSorting Expression ((Before ORDER BY + Add table aliases)) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Union - ReadFromMergeTree - ReadFromMergeTree - ReadFromMergeTree + ReadFromMergeTree Expression (Projection) Limit (preliminary LIMIT) FinishSorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Union - ReadFromMergeTree - ReadFromMergeTree - ReadFromMergeTree + ReadFromMergeTree optimize_aggregation_in_order Expression ((Projection + Before ORDER BY)) Aggregating @@ -57,18 +51,12 @@ Expression ((Projection + Before ORDER BY)) Aggregating Expression ((Before GROUP BY + Add table aliases)) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Union - ReadFromMergeTree - ReadFromMergeTree - ReadFromMergeTree + ReadFromMergeTree Expression ((Projection + Before ORDER BY)) Aggregating Expression (Before GROUP BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Union - ReadFromMergeTree - ReadFromMergeTree - ReadFromMergeTree + ReadFromMergeTree second-index 1 1 From 179f172c4c3bd83c9fbf6981591f32ded2c948d3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 28 May 2021 02:10:44 +0300 Subject: [PATCH 099/154] fix build --- .../MergeTree/PartMovesBetweenShardsOrchestrator.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 12 ++++++------ src/Storages/StorageReplicatedMergeTree.h | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index 7a4c3d530a7..4c187109ac6 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -347,7 +347,7 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil:: { { ReplicatedMergeTreeLogEntry log_entry; - if (storage.dropPart(zk, entry.part_name, log_entry,false, false)) + if (storage.dropPartImpl(zk, entry.part_name, log_entry, false, false)) storage.waitForAllReplicasToProcessLogEntry(log_entry, true); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 9908213bd18..0d0d1f7a173 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6949,7 +6949,7 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI return true; } -void StorageReplicatedMergeTree::dropPartImpl( +bool StorageReplicatedMergeTree::dropPartImpl( zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop) { LOG_TRACE(log, "Will try to insert a log entry to DROP_RANGE for part: " + part_name); @@ -6966,7 +6966,7 @@ void StorageReplicatedMergeTree::dropPartImpl( { if (throw_if_noop) throw Exception("Part " + part_name + " not found locally, won't try to drop it.", ErrorCodes::NO_SUCH_DATA_PART); - return; + return false; } /// There isn't a lot we can do otherwise. Can't cancel merges because it is possible that a replica already @@ -6977,7 +6977,7 @@ void StorageReplicatedMergeTree::dropPartImpl( throw Exception("Part " + part_name + " is currently participating in a background operation (mutation/merge)" + ", try again later", ErrorCodes::PART_IS_TEMPORARILY_LOCKED); - return; + return false; } if (partIsLastQuorumPart(part->info)) @@ -6985,7 +6985,7 @@ void StorageReplicatedMergeTree::dropPartImpl( if (throw_if_noop) throw Exception("Part " + part_name + " is last inserted part with quorum in partition. Cannot drop", ErrorCodes::NOT_IMPLEMENTED); - return; + return false; } if (partIsInsertingWithParallelQuorum(part->info)) @@ -6993,7 +6993,7 @@ void StorageReplicatedMergeTree::dropPartImpl( if (throw_if_noop) throw Exception("Part " + part_name + " is inserting with parallel quorum. Cannot drop", ErrorCodes::NOT_IMPLEMENTED); - return; + return false; } Coordination::Requests ops; @@ -7035,7 +7035,7 @@ void StorageReplicatedMergeTree::dropPartImpl( String log_znode_path = dynamic_cast(*responses[clear_block_ops_size + 1]).path_created; entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - return; + return true; } } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 8454dec7393..711c36e10f3 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -661,7 +661,7 @@ private: bool fetch_part, ContextPtr query_context) override; - void dropPartImpl(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop); + bool dropPartImpl(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop); /// Check granularity of already existing replicated table in zookeeper if it exists /// return true if it's fixed From ce11f35dcc64eeb04f2b5b48931fffe10211d704 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 11:11:17 +0300 Subject: [PATCH 100/154] Convert header from MergeTree to expected from query plan. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 2ad10461613..30281a394e8 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1100,6 +1100,20 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build }); } + if (!isCompatibleHeader(pipe.getHeader(), getOutputStream().header)) + { + auto converting_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + getOutputStream().header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + pipe.addSimpleTransform([&](const Block & cur_header) + { + return std::make_shared(cur_header, converting_actions); + }); + } + for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); From 4e28b7cb0283f16c467c34d63303aab581b9c81a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 12:23:46 +0300 Subject: [PATCH 101/154] Fix _sample_factor column. --- .../QueryPlan/ReadFromMergeTree.cpp | 80 ++++++++++--------- src/Processors/QueryPlan/ReadFromMergeTree.h | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 15 +--- 3 files changed, 46 insertions(+), 50 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 30281a394e8..9241009ee87 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -92,7 +92,6 @@ struct ReadFromMergeTree::AnalysisResult { RangesInDataParts parts_with_ranges; MergeTreeDataSelectSamplingData sampling; - bool sample_factor_column_queried = false; String query_id; IndexStats index_stats; Names column_names_to_read; @@ -130,6 +129,13 @@ ReadFromMergeTree::ReadFromMergeTree( , settings(std::move(settings_)) , log(log_) { + if (settings.sample_factor_column_queried) + { + /// Only _sample_factor virtual column is added by ReadFromMergeTree + /// Other virtual columns are added by MergeTreeBaseSelectProcessor. + auto type = std::make_shared(); + output_stream->header.insert({type->createColumn(), type, "_sample_factor"}); + } } Pipe ReadFromMergeTree::readFromPool( @@ -894,13 +900,9 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre MergeTreeDataSelectExecutor::filterPartsByPartition( metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read.get(), log, result.index_stats); - for (const auto & col : virt_column_names) - if (col == "_sample_factor") - result.sample_factor_column_queried = true; - result.sampling = MergeTreeDataSelectExecutor::getSampling( select, parts, metadata_snapshot, key_condition, - data, log, result.sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); + data, log, settings.sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); if (result.sampling.read_nothing) return result; @@ -1062,55 +1064,61 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build }); } - if (result_projection) + Block cur_header = result_projection ? result_projection->getResultColumns() + : pipe.getHeader(); + + auto append_actions = [&result_projection, &cur_header](ActionsDAGPtr actions) { - auto projection_actions = std::make_shared(result_projection); - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, projection_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 (result.sample_factor_column_queried) + if (settings.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)); - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_action = std::make_shared(adding_column_dag); - - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, adding_column_action); - }); + auto adding_column = ActionsDAG::makeAddingColumnActions(std::move(column)); + append_actions(std::move(adding_column)); } // TODO There seems to be no place initializing remove_columns_actions - if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) - { - auto remove_columns_action = std::make_shared( - query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); + // if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) + // { + // auto remove_columns_action = std::make_shared( + // query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, remove_columns_action); - }); - } + // pipe.addSimpleTransform([&](const Block & header) + // { + // return std::make_shared(header, remove_columns_action); + // }); + // } - if (!isCompatibleHeader(pipe.getHeader(), getOutputStream().header)) + /// Extra columns may be returned (for example, if sampling is used). + /// Convert pipe to step header structure. + if (!isCompatibleHeader(cur_header, getOutputStream().header)) { - auto converting_dag = ActionsDAG::makeConvertingActions( - pipe.getHeader().getColumnsWithTypeAndName(), + auto converting = ActionsDAG::makeConvertingActions( + cur_header.getColumnsWithTypeAndName(), getOutputStream().header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto converting_actions = std::make_shared(std::move(converting_dag)); - pipe.addSimpleTransform([&](const Block & cur_header) + append_actions(std::move(converting)); + } + + if (result_projection) + { + auto projection_actions = std::make_shared(result_projection); + pipe.addSimpleTransform([&](const Block & header) { - return std::make_shared(cur_header, converting_actions); + return std::make_shared(header, projection_actions); }); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index ef5cc5dc70c..0c3814c507f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -50,6 +50,7 @@ public: //size_t min_marks_for_concurrent_read; bool use_uncompressed_cache; bool force_primary_key; + bool sample_factor_column_queried; MergeTreeReaderSettings reader_settings; MergeTreeReadPool::BackoffSettings backoff_settings; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index db9986dd25d..e3c584a4708 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1208,26 +1208,13 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( .num_streams = num_streams, .preferred_block_size_bytes = settings.preferred_block_size_bytes, .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - //.min_marks_for_concurrent_read = settings.min_marks_for_concurrent_read, .use_uncompressed_cache = settings.use_uncompressed_cache, .force_primary_key = settings.force_primary_key, + .sample_factor_column_queried = sample_factor_column_queried, .reader_settings = reader_settings, .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), }; - // const SelectQueryInfo & query_info_, - // const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, - // ContextPtr context_, - // const MergeTreeData & data_, - // StorageMetadataPtr metadata_snapshot_, - // StorageMetadataPtr metadata_snapshot_base_, - // Names real_column_names_, - // MergeTreeData::DataPartsVector parts_, - // PrewhereInfoPtr prewhere_info_, - // Names virt_column_names_, - // Settings settings_, - // Poco::Logger * log_ - auto read_from_merge_tree = std::make_unique( query_info, max_block_numbers_to_read, From c80ab6166a74950223da737f360322c11cba23a8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 12:26:44 +0300 Subject: [PATCH 102/154] Fix style. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 9241009ee87..4af3fec6099 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -37,6 +36,7 @@ namespace DB namespace ErrorCodes { extern const int INDEX_NOT_USED; + extern const int LOGICAL_ERROR; } namespace diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index e3c584a4708..299abf66049 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1149,7 +1149,7 @@ size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead( if (sampling.read_nothing) return 0; - /// Do not init. Ther are not used (cause skip index is ignored) + /// Do not init. It is not used (cause skip index is ignored) MergeTreeReaderSettings reader_settings; auto parts_with_ranges = filterPartsByPrimaryKeyAndSkipIndexes( From 94f1ac5a167df0ed54c5d4ed76ebb8b55764ea5f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 12:41:07 +0300 Subject: [PATCH 103/154] Remove some commented code. --- .../QueryPlan/ReadFromMergeTree.cpp | 107 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 631 +----------------- .../MergeTree/MergeTreeDataSelectExecutor.h | 73 +- 3 files changed, 51 insertions(+), 760 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 4af3fec6099..7e4a8ba5a55 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -39,55 +39,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -namespace -{ - -/// Marks are placed whenever threshold on rows or bytes is met. -/// So we have to return the number of marks on whatever estimate is higher - by rows or by bytes. -size_t roundRowsOrBytesToMarks( - size_t rows_setting, - size_t bytes_setting, - size_t rows_granularity, - size_t bytes_granularity) -{ - size_t res = (rows_setting + rows_granularity - 1) / rows_granularity; - - if (bytes_granularity == 0) - return res; - else - return std::max(res, (bytes_setting + bytes_granularity - 1) / bytes_granularity); -} -/// Same as roundRowsOrBytesToMarks() but do not return more then max_marks -size_t minMarksForConcurrentRead( - size_t rows_setting, - size_t bytes_setting, - size_t rows_granularity, - size_t bytes_granularity, - size_t max_marks) -{ - size_t marks = 1; - - if (rows_setting + rows_granularity <= rows_setting) /// overflow - marks = max_marks; - else if (rows_setting) - marks = (rows_setting + rows_granularity - 1) / rows_granularity; - - if (bytes_granularity == 0) - return marks; - else - { - /// Overflow - if (bytes_setting + bytes_granularity <= bytes_setting) /// overflow - return max_marks; - if (bytes_setting) - return std::max(marks, (bytes_setting + bytes_granularity - 1) / bytes_granularity); - else - return marks; - } -} - -} - struct ReadFromMergeTree::AnalysisResult { RangesInDataParts parts_with_ranges; @@ -267,13 +218,13 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( if (adaptive_parts > parts_with_ranges.size() / 2) index_granularity_bytes = data_settings->index_granularity_bytes; - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( q_settings.merge_tree_max_rows_to_use_cache, q_settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, index_granularity_bytes); - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( q_settings.merge_tree_min_rows_for_concurrent_read, q_settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, @@ -333,13 +284,13 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( if (adaptive_parts > parts_with_ranges.size() / 2) index_granularity_bytes = data_settings->index_granularity_bytes; - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( q_settings.merge_tree_max_rows_to_use_cache, q_settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, index_granularity_bytes); - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( q_settings.merge_tree_min_rows_for_concurrent_read, q_settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, @@ -496,7 +447,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( if (pipe.numOutputPorts() > 1) { - auto transform = std::make_shared( pipe.getHeader(), pipe.numOutputPorts(), @@ -640,7 +590,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (adaptive_parts >= parts_with_range.size() / 2) index_granularity_bytes = data_settings->index_granularity_bytes; - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( q_settings.merge_tree_max_rows_to_use_cache, q_settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, @@ -773,17 +723,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( std::min(used_num_streams, q_settings.max_final_threads), sort_description, data.merging_params, partition_key_columns, settings.max_block_size); - // auto final_step = std::make_unique( - // plan->getCurrentDataStream(), - // std::min(used_num_streams, settings.max_final_threads), - // sort_description, - // data.merging_params, - // partition_key_columns, - // max_block_size); - - // final_step->setStepDescription("Merge rows for FINAL"); - // plan->addStep(std::move(final_step)); - partition_pipes.emplace_back(std::move(pipe)); } @@ -793,7 +732,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( size_t num_streams_for_lonely_parts = used_num_streams * lonely_parts.size(); - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( q_settings.merge_tree_min_rows_for_concurrent_read, q_settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, @@ -804,33 +743,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); - // ReadFromMergeTree::Settings step_settings - // { - // .max_block_size = max_block_size, - // .preferred_block_size_bytes = settings.preferred_block_size_bytes, - // .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - // .min_marks_for_concurrent_read = min_marks_for_concurrent_read, - // .use_uncompressed_cache = use_uncompressed_cache, - // .reader_settings = reader_settings, - // .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - // }; - - // auto plan = std::make_unique(); - // auto step = std::make_unique( - // data, - // metadata_snapshot, - // query_id, - // column_names, - // std::move(lonely_parts), - // // std::move(index_stats), - // query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - // virt_columns, - // step_settings, - // num_streams_for_lonely_parts, - // ReadFromMergeTree::ReadType::Default); - - // plan->addStep(std::move(step)); - auto pipe = read(std::move(lonely_parts), column_names, ReadFromMergeTree::ReadType::Default, num_streams_for_lonely_parts, min_marks_for_concurrent_read, use_uncompressed_cache); @@ -846,13 +758,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( return std::make_shared(header, sorting_expr); }); - // auto expression_step = std::make_unique( - // plan->getCurrentDataStream(), - // metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); - - // expression_step->setStepDescription("Calculate sorting key expression"); - // plan->addStep(std::move(expression_step)); - partition_pipes.emplace_back(std::move(pipe)); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 299abf66049..baf545f7557 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1235,12 +1235,10 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( return plan; } -namespace -{ /// Marks are placed whenever threshold on rows or bytes is met. /// So we have to return the number of marks on whatever estimate is higher - by rows or by bytes. -size_t roundRowsOrBytesToMarks( +size_t MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( size_t rows_setting, size_t bytes_setting, size_t rows_granularity, @@ -1254,608 +1252,35 @@ size_t roundRowsOrBytesToMarks( return std::max(res, (bytes_setting + bytes_granularity - 1) / bytes_granularity); } +/// Same as roundRowsOrBytesToMarks() but do not return more then max_marks +size_t MergeTreeDataSelectExecutor::minMarksForConcurrentRead( + size_t rows_setting, + size_t bytes_setting, + size_t rows_granularity, + size_t bytes_granularity, + size_t max_marks) +{ + size_t marks = 1; + + if (rows_setting + rows_granularity <= rows_setting) /// overflow + marks = max_marks; + else if (rows_setting) + marks = (rows_setting + rows_granularity - 1) / rows_granularity; + + if (bytes_granularity == 0) + return marks; + else + { + /// Overflow + if (bytes_setting + bytes_granularity <= bytes_setting) /// overflow + return max_marks; + if (bytes_setting) + return std::max(marks, (bytes_setting + bytes_granularity - 1) / bytes_granularity); + else + return marks; + } } -// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( -// RangesInDataParts && parts, -// // ReadFromMergeTree::IndexStatPtr index_stats, -// size_t num_streams, -// const Names & column_names, -// const StorageMetadataPtr & metadata_snapshot, -// UInt64 max_block_size, -// bool use_uncompressed_cache, -// const SelectQueryInfo & query_info, -// const Names & virt_columns, -// const Settings & settings, -// const MergeTreeReaderSettings & reader_settings, -// const String & query_id) const -// { -// /// Count marks for each part. -// std::vector sum_marks_in_parts(parts.size()); -// size_t sum_marks = 0; -// size_t total_rows = 0; - -// const auto data_settings = data.getSettings(); -// size_t adaptive_parts = 0; -// for (size_t i = 0; i < parts.size(); ++i) -// { -// total_rows += parts[i].getRowsCount(); -// sum_marks_in_parts[i] = parts[i].getMarksCount(); -// sum_marks += sum_marks_in_parts[i]; - -// if (parts[i].data_part->index_granularity_info.is_adaptive) -// ++adaptive_parts; -// } - -// size_t index_granularity_bytes = 0; -// if (adaptive_parts > parts.size() / 2) -// index_granularity_bytes = data_settings->index_granularity_bytes; - -// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( -// settings.merge_tree_max_rows_to_use_cache, -// settings.merge_tree_max_bytes_to_use_cache, -// data_settings->index_granularity, -// index_granularity_bytes); - -// const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( -// settings.merge_tree_min_rows_for_concurrent_read, -// settings.merge_tree_min_bytes_for_concurrent_read, -// data_settings->index_granularity, -// index_granularity_bytes, -// sum_marks); - -// if (sum_marks > max_marks_to_use_cache) -// use_uncompressed_cache = false; - -// if (0 == sum_marks) -// return {}; - -// ReadFromMergeTree::Settings step_settings -// { -// .max_block_size = max_block_size, -// .preferred_block_size_bytes = settings.preferred_block_size_bytes, -// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, -// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, -// .use_uncompressed_cache = use_uncompressed_cache, -// .reader_settings = reader_settings, -// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), -// }; - -// if (num_streams > 1) -// { -// /// Reduce the number of num_streams if the data is small. -// if (sum_marks < num_streams * min_marks_for_concurrent_read && parts.size() < num_streams) -// num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts.size()); -// } - -// auto plan = std::make_unique(); -// auto step = std::make_unique( -// data, -// metadata_snapshot, -// query_id, -// column_names, -// std::move(parts), -// // std::move(index_stats), -// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, -// virt_columns, -// step_settings, -// num_streams, -// ReadFromMergeTree::ReadType::Default); - -// plan->addStep(std::move(step)); -// return plan; -// } - -// static ActionsDAGPtr createProjection(const Block & header) -// { -// auto projection = std::make_shared(header.getNamesAndTypesList()); -// projection->removeUnusedActions(header.getNames()); -// projection->projectInput(); -// return projection; -// } - -// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( -// RangesInDataParts && parts, -// // ReadFromMergeTree::IndexStatPtr index_stats, -// size_t num_streams, -// const Names & column_names, -// const StorageMetadataPtr & metadata_snapshot, -// UInt64 max_block_size, -// bool use_uncompressed_cache, -// const SelectQueryInfo & query_info, -// const ActionsDAGPtr & sorting_key_prefix_expr, -// const Names & virt_columns, -// const Settings & settings, -// const MergeTreeReaderSettings & reader_settings, -// ActionsDAGPtr & out_projection, -// const String & query_id, -// const InputOrderInfoPtr & input_order_info) const -// { -// size_t sum_marks = 0; -// size_t adaptive_parts = 0; -// std::vector sum_marks_in_parts(parts.size()); -// const auto data_settings = data.getSettings(); - -// for (size_t i = 0; i < parts.size(); ++i) -// { -// sum_marks_in_parts[i] = parts[i].getMarksCount(); -// sum_marks += sum_marks_in_parts[i]; - -// if (parts[i].data_part->index_granularity_info.is_adaptive) -// ++adaptive_parts; -// } - -// size_t index_granularity_bytes = 0; -// if (adaptive_parts > parts.size() / 2) -// index_granularity_bytes = data_settings->index_granularity_bytes; - -// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( -// settings.merge_tree_max_rows_to_use_cache, -// settings.merge_tree_max_bytes_to_use_cache, -// data_settings->index_granularity, -// index_granularity_bytes); - -// const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( -// settings.merge_tree_min_rows_for_concurrent_read, -// settings.merge_tree_min_bytes_for_concurrent_read, -// data_settings->index_granularity, -// index_granularity_bytes, -// sum_marks); - -// if (sum_marks > max_marks_to_use_cache) -// use_uncompressed_cache = false; - -// Pipes res; - -// if (sum_marks == 0) -// return {}; - -// /// Let's split ranges to avoid reading much data. -// auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size](const auto & ranges, int direction) -// { -// MarkRanges new_ranges; -// const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; -// size_t marks_in_range = 1; - -// if (direction == 1) -// { -// /// Split first few ranges to avoid reading much data. -// bool split = false; -// for (auto range : ranges) -// { -// while (!split && range.begin + marks_in_range < range.end) -// { -// new_ranges.emplace_back(range.begin, range.begin + marks_in_range); -// range.begin += marks_in_range; -// marks_in_range *= 2; - -// if (marks_in_range > max_marks_in_range) -// split = true; -// } -// new_ranges.emplace_back(range.begin, range.end); -// } -// } -// else -// { -// /// Split all ranges to avoid reading much data, because we have to -// /// store whole range in memory to reverse it. -// for (auto it = ranges.rbegin(); it != ranges.rend(); ++it) -// { -// auto range = *it; -// while (range.begin + marks_in_range < range.end) -// { -// new_ranges.emplace_front(range.end - marks_in_range, range.end); -// range.end -= marks_in_range; -// marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); -// } -// new_ranges.emplace_front(range.begin, range.end); -// } -// } - -// return new_ranges; -// }; - -// const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; -// bool need_preliminary_merge = (parts.size() > settings.read_in_order_two_level_merge_threshold); - -// std::vector plans; - -// for (size_t i = 0; i < num_streams && !parts.empty(); ++i) -// { -// size_t need_marks = min_marks_per_stream; -// RangesInDataParts new_parts; - -// /// Loop over parts. -// /// We will iteratively take part or some subrange of a part from the back -// /// and assign a stream to read from it. -// while (need_marks > 0 && !parts.empty()) -// { -// RangesInDataPart part = parts.back(); -// parts.pop_back(); - -// size_t & marks_in_part = sum_marks_in_parts.back(); - -// /// We will not take too few rows from a part. -// if (marks_in_part >= min_marks_for_concurrent_read && -// need_marks < min_marks_for_concurrent_read) -// need_marks = min_marks_for_concurrent_read; - -// /// Do not leave too few rows in the part. -// if (marks_in_part > need_marks && -// marks_in_part - need_marks < min_marks_for_concurrent_read) -// need_marks = marks_in_part; - -// MarkRanges ranges_to_get_from_part; - -// /// We take the whole part if it is small enough. -// if (marks_in_part <= need_marks) -// { -// ranges_to_get_from_part = part.ranges; - -// need_marks -= marks_in_part; -// sum_marks_in_parts.pop_back(); -// } -// else -// { -// /// Loop through ranges in part. Take enough ranges to cover "need_marks". -// while (need_marks > 0) -// { -// if (part.ranges.empty()) -// throw Exception("Unexpected end of ranges while spreading marks among streams", ErrorCodes::LOGICAL_ERROR); - -// MarkRange & range = part.ranges.front(); - -// const size_t marks_in_range = range.end - range.begin; -// const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); - -// ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); -// range.begin += marks_to_get_from_range; -// marks_in_part -= marks_to_get_from_range; -// need_marks -= marks_to_get_from_range; -// if (range.begin == range.end) -// part.ranges.pop_front(); -// } -// parts.emplace_back(part); -// } -// ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); -// new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); -// } - -// ReadFromMergeTree::Settings step_settings -// { -// .max_block_size = max_block_size, -// .preferred_block_size_bytes = settings.preferred_block_size_bytes, -// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, -// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, -// .use_uncompressed_cache = use_uncompressed_cache, -// .reader_settings = reader_settings, -// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), -// }; - -// auto read_type = input_order_info->direction == 1 -// ? ReadFromMergeTree::ReadType::InOrder -// : ReadFromMergeTree::ReadType::InReverseOrder; - -// auto plan = std::make_unique(); -// auto step = std::make_unique( -// data, -// metadata_snapshot, -// query_id, -// column_names, -// std::move(new_parts), -// // std::move(index_stats), -// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, -// virt_columns, -// step_settings, -// num_streams, -// read_type); - -// plan->addStep(std::move(step)); -// plans.emplace_back(std::move(plan)); -// } - -// if (need_preliminary_merge) -// { -// SortDescription sort_description; -// for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) -// sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], -// input_order_info->direction, 1); - -// for (auto & plan : plans) -// { -// /// Drop temporary columns, added by 'sorting_key_prefix_expr' -// out_projection = createProjection(plan->getCurrentDataStream().header); - -// auto expression_step = std::make_unique( -// plan->getCurrentDataStream(), -// sorting_key_prefix_expr); - -// expression_step->setStepDescription("Calculate sorting key prefix"); -// plan->addStep(std::move(expression_step)); - -// auto merging_sorted = std::make_unique( -// plan->getCurrentDataStream(), -// sort_description, -// max_block_size); - -// merging_sorted->setStepDescription("Merge sorting mark ranges"); -// plan->addStep(std::move(merging_sorted)); -// } -// } - -// if (plans.size() == 1) -// return std::move(plans.front()); - -// DataStreams input_streams; -// for (const auto & plan : plans) -// input_streams.emplace_back(plan->getCurrentDataStream()); - -// auto union_step = std::make_unique(std::move(input_streams)); - -// auto plan = std::make_unique(); -// plan->unitePlans(std::move(union_step), std::move(plans)); - -// return plan; -// } - - -// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( -// RangesInDataParts && parts, -// size_t num_streams, -// const Names & column_names, -// const StorageMetadataPtr & metadata_snapshot, -// UInt64 max_block_size, -// bool use_uncompressed_cache, -// const SelectQueryInfo & query_info, -// const Names & virt_columns, -// const Settings & settings, -// const MergeTreeReaderSettings & reader_settings, -// ActionsDAGPtr & out_projection, -// const String & query_id) const -// { -// const auto data_settings = data.getSettings(); -// size_t sum_marks = 0; -// size_t adaptive_parts = 0; -// for (const auto & part : parts) -// { -// for (const auto & range : part.ranges) -// sum_marks += range.end - range.begin; - -// if (part.data_part->index_granularity_info.is_adaptive) -// ++adaptive_parts; -// } - -// size_t index_granularity_bytes = 0; -// if (adaptive_parts >= parts.size() / 2) -// index_granularity_bytes = data_settings->index_granularity_bytes; - -// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( -// settings.merge_tree_max_rows_to_use_cache, -// settings.merge_tree_max_bytes_to_use_cache, -// data_settings->index_granularity, -// index_granularity_bytes); - -// if (sum_marks > max_marks_to_use_cache) -// use_uncompressed_cache = false; - -// if (num_streams > settings.max_final_threads) -// num_streams = settings.max_final_threads; - -// /// If setting do_not_merge_across_partitions_select_final is true than we won't merge parts from different partitions. -// /// We have all parts in parts vector, where parts with same partition are nearby. -// /// So we will store iterators pointed to the beginning of each partition range (and parts.end()), -// /// then we will create a pipe for each partition that will run selecting processor and merging processor -// /// for the parts with this partition. In the end we will unite all the pipes. -// std::vector parts_to_merge_ranges; -// auto it = parts.begin(); -// parts_to_merge_ranges.push_back(it); - -// if (settings.do_not_merge_across_partitions_select_final) -// { -// while (it != parts.end()) -// { -// it = std::find_if( -// it, parts.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); -// parts_to_merge_ranges.push_back(it); -// } -// /// We divide threads for each partition equally. But we will create at least the number of partitions threads. -// /// (So, the total number of threads could be more than initial num_streams. -// num_streams /= (parts_to_merge_ranges.size() - 1); -// } -// else -// { -// /// If do_not_merge_across_partitions_select_final is false we just merge all the parts. -// parts_to_merge_ranges.push_back(parts.end()); -// } - -// std::vector partition_plans; - -// /// If do_not_merge_across_partitions_select_final is true and num_streams > 1 -// /// we will store lonely parts with level > 0 to use parallel select on them. -// std::vector lonely_parts; -// size_t total_rows_in_lonely_parts = 0; -// size_t sum_marks_in_lonely_parts = 0; - -// for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) -// { -// QueryPlanPtr plan; - -// { -// RangesInDataParts new_parts; - -// /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition -// /// with level > 0 then we won't postprocess this part and if num_streams > 1 we -// /// can use parallel select on such parts. We save such parts in one vector and then use -// /// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select. -// if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final && -// std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && -// parts_to_merge_ranges[range_index]->data_part->info.level > 0) -// { -// total_rows_in_lonely_parts += parts_to_merge_ranges[range_index]->getRowsCount(); -// sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount(); -// lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index])); -// continue; -// } -// else -// { -// for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) -// { -// new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); -// } -// } - -// if (new_parts.empty()) -// continue; - -// ReadFromMergeTree::Settings step_settings -// { -// .max_block_size = max_block_size, -// .preferred_block_size_bytes = settings.preferred_block_size_bytes, -// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, -// .min_marks_for_concurrent_read = 0, /// this setting is not used for reading in order -// .use_uncompressed_cache = use_uncompressed_cache, -// .reader_settings = reader_settings, -// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), -// }; - -// plan = std::make_unique(); -// auto step = std::make_unique( -// data, -// metadata_snapshot, -// query_id, -// column_names, -// std::move(new_parts), -// // std::move(index_stats), -// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, -// virt_columns, -// step_settings, -// num_streams, -// ReadFromMergeTree::ReadType::InOrder); - -// plan->addStep(std::move(step)); - -// /// Drop temporary columns, added by 'sorting_key_expr' -// if (!out_projection) -// out_projection = createProjection(plan->getCurrentDataStream().header); -// } - -// auto expression_step = std::make_unique( -// plan->getCurrentDataStream(), -// metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); - -// expression_step->setStepDescription("Calculate sorting key expression"); -// plan->addStep(std::move(expression_step)); - -// /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition -// /// with level > 0 then we won't postprocess this part -// if (settings.do_not_merge_across_partitions_select_final && -// std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && -// parts_to_merge_ranges[range_index]->data_part->info.level > 0) -// { -// partition_plans.emplace_back(std::move(plan)); -// continue; -// } - -// Names sort_columns = metadata_snapshot->getSortingKeyColumns(); -// SortDescription sort_description; -// size_t sort_columns_size = sort_columns.size(); -// sort_description.reserve(sort_columns_size); - -// Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; - -// const auto & header = plan->getCurrentDataStream().header; -// for (size_t i = 0; i < sort_columns_size; ++i) -// sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); - -// auto final_step = std::make_unique( -// plan->getCurrentDataStream(), -// std::min(num_streams, settings.max_final_threads), -// sort_description, -// data.merging_params, -// partition_key_columns, -// max_block_size); - -// final_step->setStepDescription("Merge rows for FINAL"); -// plan->addStep(std::move(final_step)); - -// partition_plans.emplace_back(std::move(plan)); -// } - -// if (!lonely_parts.empty()) -// { -// RangesInDataParts new_parts; - -// size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size(); - -// const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( -// settings.merge_tree_min_rows_for_concurrent_read, -// settings.merge_tree_min_bytes_for_concurrent_read, -// data_settings->index_granularity, -// index_granularity_bytes, -// sum_marks_in_lonely_parts); - -// /// Reduce the number of num_streams_for_lonely_parts if the data is small. -// if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) -// num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); - -// ReadFromMergeTree::Settings step_settings -// { -// .max_block_size = max_block_size, -// .preferred_block_size_bytes = settings.preferred_block_size_bytes, -// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, -// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, -// .use_uncompressed_cache = use_uncompressed_cache, -// .reader_settings = reader_settings, -// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), -// }; - -// auto plan = std::make_unique(); -// auto step = std::make_unique( -// data, -// metadata_snapshot, -// query_id, -// column_names, -// std::move(lonely_parts), -// // std::move(index_stats), -// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, -// virt_columns, -// step_settings, -// num_streams_for_lonely_parts, -// ReadFromMergeTree::ReadType::Default); - -// plan->addStep(std::move(step)); - -// /// Drop temporary columns, added by 'sorting_key_expr' -// if (!out_projection) -// out_projection = createProjection(plan->getCurrentDataStream().header); - -// auto expression_step = std::make_unique( -// plan->getCurrentDataStream(), -// metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); - -// expression_step->setStepDescription("Calculate sorting key expression"); -// plan->addStep(std::move(expression_step)); - -// partition_plans.emplace_back(std::move(plan)); -// } - -// if (partition_plans.empty()) -// return {}; - -// if (partition_plans.size() == 1) -// return std::move(partition_plans.front()); - -// auto result_header = partition_plans.front()->getCurrentDataStream().header; -// DataStreams input_streams; -// for (const auto & partition_plan : partition_plans) -// input_streams.push_back(partition_plan->getCurrentDataStream()); - -// auto union_step = std::make_unique(std::move(input_streams), result_header); -// union_step->setStepDescription("Unite sources after FINAL"); -// QueryPlanPtr plan = std::make_unique(); -// plan->unitePlans(std::move(union_step), std::move(partition_plans)); -// return plan; -// } /// Calculates a set of mark ranges, that could possibly contain keys, required by condition. /// In other words, it removes subranges from whole range, that definitely could not contain required keys. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 077584039a1..00a347790a7 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -22,16 +22,6 @@ struct MergeTreeDataSelectSamplingData ActionsDAGPtr filter_expression; }; -// struct MergeTreeDataSelectCache -// { -// RangesInDataParts parts_with_ranges; -// MergeTreeDataSelectSamplingData sampling; -// std::unique_ptr index_stats; -// size_t sum_marks = 0; -// size_t sum_ranges = 0; -// bool use_cache = false; -// }; - using PartitionIdToMaxBlock = std::unordered_map; /** Executes SELECT queries on data from the merge tree. @@ -78,53 +68,8 @@ public: private: const MergeTreeData & data; - Poco::Logger * log; - // QueryPlanPtr spreadMarkRangesAmongStreams( - // RangesInDataParts && parts, - // size_t num_streams, - // const Names & column_names, - // const StorageMetadataPtr & metadata_snapshot, - // UInt64 max_block_size, - // bool use_uncompressed_cache, - // const SelectQueryInfo & query_info, - // const Names & virt_columns, - // const Settings & settings, - // const MergeTreeReaderSettings & reader_settings, - // const String & query_id) const; - - // /// out_projection - save projection only with columns, requested to read - // QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder( - // RangesInDataParts && parts, - // size_t num_streams, - // const Names & column_names, - // const StorageMetadataPtr & metadata_snapshot, - // UInt64 max_block_size, - // bool use_uncompressed_cache, - // const SelectQueryInfo & query_info, - // const ActionsDAGPtr & sorting_key_prefix_expr, - // const Names & virt_columns, - // const Settings & settings, - // const MergeTreeReaderSettings & reader_settings, - // ActionsDAGPtr & out_projection, - // const String & query_id, - // const InputOrderInfoPtr & input_order_info) const; - - // QueryPlanPtr spreadMarkRangesAmongStreamsFinal( - // RangesInDataParts && parts, - // size_t num_streams, - // const Names & column_names, - // const StorageMetadataPtr & metadata_snapshot, - // UInt64 max_block_size, - // bool use_uncompressed_cache, - // const SelectQueryInfo & query_info, - // const Names & virt_columns, - // const Settings & settings, - // const MergeTreeReaderSettings & reader_settings, - // ActionsDAGPtr & out_projection, - // const String & query_id) const; - /// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index. static size_t getApproximateTotalRowsToRead( const MergeTreeData::DataPartsVector & parts, @@ -186,6 +131,19 @@ private: Poco::Logger * log); public: + static size_t roundRowsOrBytesToMarks( + size_t rows_setting, + size_t bytes_setting, + size_t rows_granularity, + size_t bytes_granularity); + + static size_t minMarksForConcurrentRead( + size_t rows_setting, + size_t bytes_setting, + size_t rows_granularity, + size_t bytes_granularity, + size_t max_marks); + static std::optional> filterPartsByVirtualColumns( const MergeTreeData & data, MergeTreeData::DataPartsVector & parts, @@ -227,7 +185,10 @@ public: NamesAndTypesList available_real_columns, ContextPtr context); - static String checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, const ContextPtr & context); + static String checkLimits( + const MergeTreeData & data, + const RangesInDataParts & parts_with_ranges, + const ContextPtr & context); }; } From 91e8c2c75dbc598b23880c593fa09171e3dacaaa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 12:52:33 +0300 Subject: [PATCH 104/154] Remove MergingFinal step. --- src/Processors/QueryPlan/MergingFinal.cpp | 170 ------------------ src/Processors/QueryPlan/MergingFinal.h | 36 ---- src/Processors/ya.make | 1 - .../MergeTree/MergeTreeDataSelectExecutor.cpp | 1 - 4 files changed, 208 deletions(-) delete mode 100644 src/Processors/QueryPlan/MergingFinal.cpp delete mode 100644 src/Processors/QueryPlan/MergingFinal.h diff --git a/src/Processors/QueryPlan/MergingFinal.cpp b/src/Processors/QueryPlan/MergingFinal.cpp deleted file mode 100644 index c564a28d377..00000000000 --- a/src/Processors/QueryPlan/MergingFinal.cpp +++ /dev/null @@ -1,170 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -static ITransformingStep::Traits getTraits() -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = true, - .returns_single_stream = false, - .preserves_number_of_streams = false, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = true, - } - }; -} - -MergingFinal::MergingFinal( - const DataStream & input_stream, - size_t num_output_streams_, - SortDescription sort_description_, - MergeTreeData::MergingParams params_, - Names partition_key_columns_, - size_t max_block_size_) - : ITransformingStep(input_stream, input_stream.header, getTraits()) - , num_output_streams(num_output_streams_) - , sort_description(std::move(sort_description_)) - , merging_params(std::move(params_)) - , partition_key_columns(std::move(partition_key_columns_)) - , max_block_size(max_block_size_) -{ - /// TODO: check input_stream is partially sorted (each port) by the same description. -// output_stream->sort_description = sort_description; -// output_stream->sort_mode = DataStream::SortMode::Stream; -} - -void MergingFinal::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) -{ - const auto & header = pipeline.getHeader(); - size_t num_outputs = pipeline.getNumStreams(); - - auto get_merging_processor = [&]() -> MergingTransformPtr - { - switch (merging_params.mode) - { - case MergeTreeData::MergingParams::Ordinary: - { - return std::make_shared(header, num_outputs, - sort_description, max_block_size); - } - - case MergeTreeData::MergingParams::Collapsing: - return std::make_shared(header, num_outputs, - sort_description, merging_params.sign_column, true, max_block_size); - - case MergeTreeData::MergingParams::Summing: - return std::make_shared(header, num_outputs, - sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size); - - case MergeTreeData::MergingParams::Aggregating: - return std::make_shared(header, num_outputs, - sort_description, max_block_size); - - case MergeTreeData::MergingParams::Replacing: - return std::make_shared(header, num_outputs, - sort_description, merging_params.version_column, max_block_size); - - case MergeTreeData::MergingParams::VersionedCollapsing: - return std::make_shared(header, num_outputs, - sort_description, merging_params.sign_column, max_block_size); - - case MergeTreeData::MergingParams::Graphite: - throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); - } - - __builtin_unreachable(); - }; - - if (num_output_streams <= 1 || sort_description.empty()) - { - pipeline.addTransform(get_merging_processor()); - return; - } - - ColumnNumbers key_columns; - key_columns.reserve(sort_description.size()); - - for (auto & desc : sort_description) - { - if (!desc.column_name.empty()) - key_columns.push_back(header.getPositionByName(desc.column_name)); - else - key_columns.emplace_back(desc.column_number); - } - - pipeline.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, num_output_streams, key_columns); - }); - - pipeline.transform([&](OutputPortRawPtrs ports) - { - Processors transforms; - std::vector output_ports; - transforms.reserve(ports.size() + num_output_streams); - output_ports.reserve(ports.size()); - - for (auto & port : ports) - { - auto copier = std::make_shared(header, num_output_streams); - connect(*port, copier->getInputPort()); - output_ports.emplace_back(copier->getOutputs().begin()); - transforms.emplace_back(std::move(copier)); - } - - for (size_t i = 0; i < num_output_streams; ++i) - { - auto merge = get_merging_processor(); - merge->setSelectorPosition(i); - auto input = merge->getInputs().begin(); - - /// Connect i-th merge with i-th input port of every copier. - for (size_t j = 0; j < ports.size(); ++j) - { - connect(*output_ports[j], *input); - ++output_ports[j]; - ++input; - } - - transforms.emplace_back(std::move(merge)); - } - - return transforms; - }); -} - -void MergingFinal::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - settings.out << prefix << "Sort description: "; - dumpSortDescription(sort_description, input_streams.front().header, settings.out); - settings.out << '\n'; -} - -void MergingFinal::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add("Sort Description", explainSortDescription(sort_description, input_streams.front().header)); -} - -} diff --git a/src/Processors/QueryPlan/MergingFinal.h b/src/Processors/QueryPlan/MergingFinal.h deleted file mode 100644 index ed0394a62f4..00000000000 --- a/src/Processors/QueryPlan/MergingFinal.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once -#include -#include -#include - -namespace DB -{ - -/// Merge streams of data into single sorted stream. -class MergingFinal : public ITransformingStep -{ -public: - explicit MergingFinal( - const DataStream & input_stream, - size_t num_output_streams_, - SortDescription sort_description_, - MergeTreeData::MergingParams params_, - Names partition_key_columns_, - size_t max_block_size_); - - String getName() const override { return "MergingFinal"; } - - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; - -private: - size_t num_output_streams; - SortDescription sort_description; - MergeTreeData::MergingParams merging_params; - Names partition_key_columns; - size_t max_block_size; -}; - -} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 5ab9c79511f..86a40685d1f 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -112,7 +112,6 @@ SRCS( QueryPlan/LimitStep.cpp QueryPlan/MergeSortingStep.cpp QueryPlan/MergingAggregatedStep.cpp - QueryPlan/MergingFinal.cpp QueryPlan/MergingSortedStep.cpp QueryPlan/OffsetStep.cpp QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index baf545f7557..c719c7f996c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -30,7 +30,6 @@ #include #include #include -#include #include #include From 7b04181dd94ab9bb6d201aa7e84bc538cdb06405 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 28 May 2021 16:22:50 +0300 Subject: [PATCH 105/154] suppress msan cassandra --- contrib/cassandra | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cassandra b/contrib/cassandra index c097fb5c7e6..eb9b68dadbb 160000 --- a/contrib/cassandra +++ b/contrib/cassandra @@ -1 +1 @@ -Subproject commit c097fb5c7e63cc430016d9a8b240d8e63fbefa52 +Subproject commit eb9b68dadbb4417a2c132ad4a1c2fa76e65e6fc1 From 295a302bc8e3e22b2b5a7dff92bdbb6e7e113557 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 17:34:02 +0300 Subject: [PATCH 106/154] Remove settings from ReadFromMergeTree. --- .../QueryPlan/ReadFromMergeTree.cpp | 225 ++++++++++-------- src/Processors/QueryPlan/ReadFromMergeTree.h | 63 +++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 44 +--- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- 4 files changed, 161 insertions(+), 173 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 7e4a8ba5a55..4515db11067 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -49,38 +49,63 @@ struct ReadFromMergeTree::AnalysisResult ReadFromMergeTree::ReadType read_type = ReadFromMergeTree::ReadType::Default; }; +static MergeTreeReaderSettings getMergeTreeReaderSettings(const ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + return { + .min_bytes_to_use_direct_io = settings.min_bytes_to_use_direct_io, + .min_bytes_to_use_mmap_io = settings.min_bytes_to_use_mmap_io, + .mmap_cache = context->getMMappedFileCache(), + .max_read_buffer_size = settings.max_read_buffer_size, + .save_marks_in_cache = true, + .checksum_on_read = settings.checksum_on_read, + }; +} + +static const PrewhereInfoPtr & getPrewhereInfo(const SelectQueryInfo & query_info) +{ + return query_info.projection ? query_info.projection->prewhere_info + : query_info.prewhere_info; +} + ReadFromMergeTree::ReadFromMergeTree( - const SelectQueryInfo & query_info_, - std::shared_ptr max_block_numbers_to_read_, - ContextPtr context_, + MergeTreeData::DataPartsVector parts_, + Names real_column_names_, + Names virt_column_names_, const MergeTreeData & data_, + const SelectQueryInfo & query_info_, StorageMetadataPtr metadata_snapshot_, StorageMetadataPtr metadata_snapshot_base_, - Names real_column_names_, - MergeTreeData::DataPartsVector parts_, - PrewhereInfoPtr prewhere_info_, - Names virt_column_names_, - Settings settings_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_, + bool sample_factor_column_queried_, + std::shared_ptr max_block_numbers_to_read_, Poco::Logger * log_) : ISourceStep(DataStream{.header = MergeTreeBaseSelectProcessor::transformHeader( metadata_snapshot_->getSampleBlockForColumns(real_column_names_, data_.getVirtuals(), data_.getStorageID()), - prewhere_info_, + getPrewhereInfo(query_info_), data_.getPartitionValueType(), virt_column_names_)}) - , query_info(std::move(query_info_)) - , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) - , context(std::move(context_)) + , reader_settings(getMergeTreeReaderSettings(context_)) + , prepared_parts(std::move(parts_)) + , real_column_names(std::move(real_column_names_)) + , virt_column_names(std::move(virt_column_names_)) , data(data_) + , query_info(query_info_) + , prewhere_info(getPrewhereInfo(query_info)) , metadata_snapshot(std::move(metadata_snapshot_)) , metadata_snapshot_base(std::move(metadata_snapshot_base_)) - , real_column_names(std::move(real_column_names_)) - , prepared_parts(std::move(parts_)) - , prewhere_info(std::move(prewhere_info_)) - , virt_column_names(std::move(virt_column_names_)) - , settings(std::move(settings_)) + , context(std::move(context_)) + , max_block_size(max_block_size_) + , requested_num_streams(num_streams_) + , preferred_block_size_bytes(context->getSettingsRef().preferred_block_size_bytes) + , preferred_max_column_in_block_size_bytes(context->getSettingsRef().preferred_max_column_in_block_size_bytes) + , sample_factor_column_queried(sample_factor_column_queried_) + , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) , log(log_) { - if (settings.sample_factor_column_queried) + if (sample_factor_column_queried) { /// Only _sample_factor virtual column is added by ReadFromMergeTree /// Other virtual columns are added by MergeTreeBaseSelectProcessor. @@ -90,8 +115,11 @@ ReadFromMergeTree::ReadFromMergeTree( } Pipe ReadFromMergeTree::readFromPool( - RangesInDataParts parts_with_range, Names required_columns, - size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) + RangesInDataParts parts_with_range, + Names required_columns, + size_t max_streams, + size_t min_marks_for_concurrent_read, + bool use_uncompressed_cache) { Pipes pipes; size_t sum_marks = 0; @@ -103,8 +131,11 @@ Pipe ReadFromMergeTree::readFromPool( total_rows += part.getRowsCount(); } + const auto & settings = context->getSettingsRef(); + MergeTreeReadPool::BackoffSettings backoff_settings(settings); + auto pool = std::make_shared( - used_max_streams, + max_streams, sum_marks, min_marks_for_concurrent_read, std::move(parts_with_range), @@ -113,20 +144,20 @@ Pipe ReadFromMergeTree::readFromPool( prewhere_info, true, required_columns, - settings.backoff_settings, + backoff_settings, settings.preferred_block_size_bytes, false); auto * logger = &Poco::Logger::get(data.getLogName() + " (SelectExecutor)"); - LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, used_max_streams); + LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, max_streams); - for (size_t i = 0; i < used_max_streams; ++i) + for (size_t i = 0; i < max_streams; ++i) { auto source = std::make_shared( - i, pool, min_marks_for_concurrent_read, settings.max_block_size, + i, pool, min_marks_for_concurrent_read, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, data, metadata_snapshot, use_uncompressed_cache, - prewhere_info, settings.reader_settings, virt_column_names); + prewhere_info, reader_settings, virt_column_names); if (i == 0) { @@ -141,15 +172,22 @@ Pipe ReadFromMergeTree::readFromPool( } template -ProcessorPtr ReadFromMergeTree::createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache) +ProcessorPtr ReadFromMergeTree::createSource( + const RangesInDataPart & part, + const Names & required_columns, + bool use_uncompressed_cache) { return std::make_shared( - data, metadata_snapshot, part.data_part, settings.max_block_size, settings.preferred_block_size_bytes, - settings.preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, - prewhere_info, true, settings.reader_settings, virt_column_names, part.part_index_in_query); + data, metadata_snapshot, part.data_part, max_block_size, preferred_block_size_bytes, + preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, + prewhere_info, true, reader_settings, virt_column_names, part.part_index_in_query); } -Pipe ReadFromMergeTree::readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache) +Pipe ReadFromMergeTree::readInOrder( + RangesInDataParts parts_with_range, + Names required_columns, + ReadType read_type, + bool use_uncompressed_cache) { Pipes pipes; for (const auto & part : parts_with_range) @@ -176,10 +214,11 @@ Pipe ReadFromMergeTree::readInOrder(RangesInDataParts parts_with_range, Names re Pipe ReadFromMergeTree::read( RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, - size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) + size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) { - if (read_type == ReadType::Default && used_max_streams > 1) - return readFromPool(parts_with_range, required_columns, used_max_streams, min_marks_for_concurrent_read, use_uncompressed_cache); + if (read_type == ReadType::Default && max_streams > 1) + return readFromPool(parts_with_range, required_columns, max_streams, + min_marks_for_concurrent_read, use_uncompressed_cache); auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache); @@ -195,7 +234,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( RangesInDataParts && parts_with_ranges, const Names & column_names) { - const auto & q_settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); /// Count marks for each part. std::vector sum_marks_in_parts(parts_with_ranges.size()); @@ -219,35 +258,35 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( index_granularity_bytes = data_settings->index_granularity_bytes; const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - q_settings.merge_tree_max_rows_to_use_cache, - q_settings.merge_tree_max_bytes_to_use_cache, + settings.merge_tree_max_rows_to_use_cache, + settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, index_granularity_bytes); const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( - q_settings.merge_tree_min_rows_for_concurrent_read, - q_settings.merge_tree_min_bytes_for_concurrent_read, + settings.merge_tree_min_rows_for_concurrent_read, + settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, index_granularity_bytes, sum_marks); - bool use_uncompressed_cache = q_settings.use_uncompressed_cache; + bool use_uncompressed_cache = settings.use_uncompressed_cache; if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; if (0 == sum_marks) return {}; - size_t used_num_streams = settings.num_streams; - if (used_num_streams > 1) + size_t num_streams = requested_num_streams; + if (num_streams > 1) { /// Reduce the number of num_streams if the data is small. - if (sum_marks < used_num_streams * min_marks_for_concurrent_read && parts_with_ranges.size() < used_num_streams) - used_num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts_with_ranges.size()); + if (sum_marks < num_streams * min_marks_for_concurrent_read && parts_with_ranges.size() < num_streams) + num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts_with_ranges.size()); } return read(std::move(parts_with_ranges), column_names, ReadType::Default, - used_num_streams, min_marks_for_concurrent_read, use_uncompressed_cache); + num_streams, min_marks_for_concurrent_read, use_uncompressed_cache); } static ActionsDAGPtr createProjection(const Block & header) @@ -265,7 +304,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( ActionsDAGPtr & out_projection, const InputOrderInfoPtr & input_order_info) { - const auto & q_settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); size_t sum_marks = 0; size_t adaptive_parts = 0; std::vector sum_marks_in_parts(parts_with_ranges.size()); @@ -285,14 +324,14 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( index_granularity_bytes = data_settings->index_granularity_bytes; const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - q_settings.merge_tree_max_rows_to_use_cache, - q_settings.merge_tree_max_bytes_to_use_cache, + settings.merge_tree_max_rows_to_use_cache, + settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, index_granularity_bytes); const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( - q_settings.merge_tree_min_rows_for_concurrent_read, - q_settings.merge_tree_min_bytes_for_concurrent_read, + settings.merge_tree_min_rows_for_concurrent_read, + settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, index_granularity_bytes, sum_marks); @@ -307,7 +346,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( return {}; /// Let's split ranges to avoid reading much data. - auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size = settings.max_block_size] + auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size = max_block_size] (const auto & ranges, int direction) { MarkRanges new_ranges; @@ -352,12 +391,12 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( return new_ranges; }; - const size_t min_marks_per_stream = (sum_marks - 1) / settings.num_streams + 1; - bool need_preliminary_merge = (parts_with_ranges.size() > q_settings.read_in_order_two_level_merge_threshold); + const size_t min_marks_per_stream = (sum_marks - 1) / requested_num_streams + 1; + bool need_preliminary_merge = (parts_with_ranges.size() > settings.read_in_order_two_level_merge_threshold); Pipes pipes; - for (size_t i = 0; i < settings.num_streams && !parts_with_ranges.empty(); ++i) + for (size_t i = 0; i < requested_num_streams && !parts_with_ranges.empty(); ++i) { size_t need_marks = min_marks_per_stream; RangesInDataParts new_parts; @@ -398,7 +437,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( while (need_marks > 0) { if (part.ranges.empty()) - throw Exception("Unexpected end of ranges while spreading marks among streams", ErrorCodes::LOGICAL_ERROR); + throw Exception("Unexpected end of ranges while spreading marks among streams", + ErrorCodes::LOGICAL_ERROR); MarkRange & range = part.ranges.front(); @@ -423,7 +463,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( : ReadFromMergeTree::ReadType::InReverseOrder; pipes.emplace_back(read(std::move(new_parts), column_names, read_type, - settings.num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); + requested_num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); } if (need_preliminary_merge) @@ -451,7 +491,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( pipe.getHeader(), pipe.numOutputPorts(), sort_description, - settings.max_block_size); + max_block_size); pipe.addTransform(std::move(transform)); } @@ -573,7 +613,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( const Names & column_names, ActionsDAGPtr & out_projection) { - const auto & q_settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); const auto data_settings = data.getSettings(); size_t sum_marks = 0; size_t adaptive_parts = 0; @@ -591,8 +631,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( index_granularity_bytes = data_settings->index_granularity_bytes; const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - q_settings.merge_tree_max_rows_to_use_cache, - q_settings.merge_tree_max_bytes_to_use_cache, + settings.merge_tree_max_rows_to_use_cache, + settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, index_granularity_bytes); @@ -600,9 +640,9 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; - size_t used_num_streams = settings.num_streams; - if (used_num_streams > q_settings.max_final_threads) - used_num_streams = q_settings.max_final_threads; + size_t num_streams = requested_num_streams; + if (num_streams > settings.max_final_threads) + num_streams = settings.max_final_threads; /// If setting do_not_merge_across_partitions_select_final is true than we won't merge parts from different partitions. /// We have all parts in parts vector, where parts with same partition are nearby. @@ -613,7 +653,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( auto it = parts_with_range.begin(); parts_to_merge_ranges.push_back(it); - if (q_settings.do_not_merge_across_partitions_select_final) + if (settings.do_not_merge_across_partitions_select_final) { while (it != parts_with_range.end()) { @@ -623,7 +663,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( } /// We divide threads for each partition equally. But we will create at least the number of partitions threads. /// (So, the total number of threads could be more than initial num_streams. - used_num_streams /= (parts_to_merge_ranges.size() - 1); + num_streams /= (parts_to_merge_ranges.size() - 1); } else { @@ -650,7 +690,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// with level > 0 then we won't postprocess this part and if num_streams > 1 we /// can use parallel select on such parts. We save such parts in one vector and then use /// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select. - if (used_num_streams > 1 && q_settings.do_not_merge_across_partitions_select_final && + if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final && std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && parts_to_merge_ranges[range_index]->data_part->info.level > 0) { @@ -670,19 +710,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (new_parts.empty()) continue; - // ReadFromMergeTree::Settings step_settings - // { - // .max_block_size = max_block_size, - // .preferred_block_size_bytes = settings.preferred_block_size_bytes, - // .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - // .min_marks_for_concurrent_read = 0, /// this setting is not used for reading in order - // .use_uncompressed_cache = use_uncompressed_cache, - // .reader_settings = reader_settings, - // .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - // }; - pipe = read(std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder, - used_num_streams, 0, use_uncompressed_cache); + num_streams, 0, use_uncompressed_cache); /// Drop temporary columns, added by 'sorting_key_expr' if (!out_projection) @@ -699,7 +728,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition /// with level > 0 then we won't postprocess this part - if (q_settings.do_not_merge_across_partitions_select_final && + if (settings.do_not_merge_across_partitions_select_final && std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && parts_to_merge_ranges[range_index]->data_part->info.level > 0) { @@ -720,8 +749,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( addMergingFinal( pipe, - std::min(used_num_streams, q_settings.max_final_threads), - sort_description, data.merging_params, partition_key_columns, settings.max_block_size); + std::min(num_streams, settings.max_final_threads), + sort_description, data.merging_params, partition_key_columns, max_block_size); partition_pipes.emplace_back(std::move(pipe)); } @@ -730,11 +759,11 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( { RangesInDataParts new_parts; - size_t num_streams_for_lonely_parts = used_num_streams * lonely_parts.size(); + size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size(); const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( - q_settings.merge_tree_min_rows_for_concurrent_read, - q_settings.merge_tree_min_bytes_for_concurrent_read, + settings.merge_tree_min_rows_for_concurrent_read, + settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, index_granularity_bytes, sum_marks_in_lonely_parts); @@ -767,6 +796,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTreeData::DataPartsVector parts) const { AnalysisResult result; + const auto & settings = context->getSettingsRef(); size_t total_parts = parts.size(); @@ -807,7 +837,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre result.sampling = MergeTreeDataSelectExecutor::getSampling( select, parts, metadata_snapshot, key_condition, - data, log, settings.sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); + data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); if (result.sampling.read_nothing) return result; @@ -824,9 +854,9 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre query_info, context, key_condition, - settings.reader_settings, + reader_settings, log, - settings.num_streams, + requested_num_streams, result.index_stats, true); @@ -865,8 +895,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre ? query_info.input_order_info : (query_info.projection ? query_info.projection->input_order_info : nullptr); - const auto & q_settings = context->getSettingsRef(); - if ((q_settings.optimize_read_in_order || q_settings.optimize_aggregation_in_order) && input_order_info) + if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && input_order_info) result.read_type = (input_order_info->direction > 0) ? ReadType::InOrder : ReadType::InReverseOrder; @@ -906,7 +935,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build Pipe pipe; - const auto & q_settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); if (select.final()) { @@ -927,7 +956,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build column_names_to_read, result_projection); } - else if ((q_settings.optimize_read_in_order || q_settings.optimize_aggregation_in_order) && input_order_info) + else if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && input_order_info) { size_t prefix_size = input_order_info->order_key_prefix_descr.size(); auto order_key_prefix_ast = metadata_snapshot->getSortingKey().expression_list_ast->clone(); @@ -983,7 +1012,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build }; /// 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 (settings.sample_factor_column_queried) + if (sample_factor_column_queried) { ColumnWithTypeAndName column; column.name = "_sample_factor"; @@ -994,18 +1023,6 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build append_actions(std::move(adding_column)); } - // TODO There seems to be no place initializing remove_columns_actions - // if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) - // { - // auto remove_columns_action = std::make_shared( - // query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); - - // pipe.addSimpleTransform([&](const Block & header) - // { - // return std::make_shared(header, remove_columns_action); - // }); - // } - /// 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/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 0c3814c507f..07d45a71e0a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -40,22 +40,6 @@ public: using IndexStats = std::vector; - /// Part of settings which are needed for reading. - struct Settings - { - UInt64 max_block_size; - size_t num_streams; - size_t preferred_block_size_bytes; - size_t preferred_max_column_in_block_size_bytes; - //size_t min_marks_for_concurrent_read; - bool use_uncompressed_cache; - bool force_primary_key; - bool sample_factor_column_queried; - - MergeTreeReaderSettings reader_settings; - MergeTreeReadPool::BackoffSettings backoff_settings; - }; - enum class ReadType { /// By default, read will use MergeTreeReadPool and return pipe with num_streams outputs. @@ -72,17 +56,18 @@ public: }; ReadFromMergeTree( - const SelectQueryInfo & query_info_, - std::shared_ptr max_block_numbers_to_read_, - ContextPtr context_, + MergeTreeData::DataPartsVector parts_, + Names real_column_names_, + Names virt_column_names_, const MergeTreeData & data_, + const SelectQueryInfo & query_info_, StorageMetadataPtr metadata_snapshot_, StorageMetadataPtr metadata_snapshot_base_, - Names real_column_names_, - MergeTreeData::DataPartsVector parts_, - PrewhereInfoPtr prewhere_info_, - Names virt_column_names_, - Settings settings_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_, + bool sample_factor_column_queried_, + std::shared_ptr max_block_numbers_to_read_, Poco::Logger * log_ ); @@ -97,23 +82,33 @@ public: void describeIndexes(JSONBuilder::JSONMap & map) const override; private: - SelectQueryInfo query_info; - std::shared_ptr max_block_numbers_to_read; - ContextPtr context; + const MergeTreeReaderSettings reader_settings; + + MergeTreeData::DataPartsVector prepared_parts; + Names real_column_names; + Names virt_column_names; + const MergeTreeData & data; + SelectQueryInfo query_info; + PrewhereInfoPtr prewhere_info; + StorageMetadataPtr metadata_snapshot; StorageMetadataPtr metadata_snapshot_base; - Names real_column_names; - MergeTreeData::DataPartsVector prepared_parts; - PrewhereInfoPtr prewhere_info; - Names virt_column_names; - Settings settings; + ContextPtr context; + + const size_t max_block_size; + const size_t requested_num_streams; + const size_t preferred_block_size_bytes; + const size_t preferred_max_column_in_block_size_bytes; + const bool sample_factor_column_queried; + + std::shared_ptr max_block_numbers_to_read; Poco::Logger * log; - Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); - Pipe readFromPool(RangesInDataParts parts_with_ranges, Names required_columns, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); + Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); + Pipe readFromPool(RangesInDataParts parts_with_ranges, Names required_columns, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); Pipe readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache); template diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c719c7f996c..c12759bac97 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -381,7 +381,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( const ASTSelectQuery & select, - MergeTreeData::DataPartsVector & parts, + const MergeTreeData::DataPartsVector & parts, const StorageMetadataPtr & metadata_snapshot, KeyCondition & key_condition, const MergeTreeData & data, @@ -1189,43 +1189,19 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - const auto & settings = context->getSettingsRef(); - - MergeTreeReaderSettings reader_settings = - { - .min_bytes_to_use_direct_io = settings.min_bytes_to_use_direct_io, - .min_bytes_to_use_mmap_io = settings.min_bytes_to_use_mmap_io, - .mmap_cache = context->getMMappedFileCache(), - .max_read_buffer_size = settings.max_read_buffer_size, - .save_marks_in_cache = true, - .checksum_on_read = settings.checksum_on_read, - }; - - ReadFromMergeTree::Settings step_settings - { - .max_block_size = max_block_size, - .num_streams = num_streams, - .preferred_block_size_bytes = settings.preferred_block_size_bytes, - .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - .use_uncompressed_cache = settings.use_uncompressed_cache, - .force_primary_key = settings.force_primary_key, - .sample_factor_column_queried = sample_factor_column_queried, - .reader_settings = reader_settings, - .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - }; - auto read_from_merge_tree = std::make_unique( - query_info, - max_block_numbers_to_read, - context, + parts, + real_column_names, + virt_column_names, data, + query_info, metadata_snapshot, metadata_snapshot_base, - real_column_names, - parts, - query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - virt_column_names, - step_settings, + context, + max_block_size, + num_streams, + sample_factor_column_queried, + max_block_numbers_to_read, log ); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 00a347790a7..b43fb785573 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -176,7 +176,7 @@ public: static MergeTreeDataSelectSamplingData getSampling( const ASTSelectQuery & select, - MergeTreeData::DataPartsVector & parts, + const MergeTreeData::DataPartsVector & parts, const StorageMetadataPtr & metadata_snapshot, KeyCondition & key_condition, const MergeTreeData & data, From 095f06c8a35b876b64b247d77e8eec2dc27dc1c9 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 28 May 2021 17:37:05 +0300 Subject: [PATCH 107/154] raise timeout for test_host_ip_change/test.py::test_user_access_ip_change --- tests/integration/helpers/cluster.py | 9 ++++++--- .../configs/dictionaries/.gitkeep | 1 - 2 files changed, 6 insertions(+), 4 deletions(-) delete mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/configs/dictionaries/.gitkeep diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6287064b616..733af0cb8f9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -126,7 +126,7 @@ class ClickHouseCluster: """ def __init__(self, base_path, name=None, base_config_dir=None, server_bin_path=None, client_bin_path=None, - odbc_bridge_bin_path=None, library_bridge_bin_path=None, zookeeper_config_path=None, + odbc_bridge_bin_path=None, library_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None): for param in list(os.environ.keys()): print("ENV %40s %s" % (param, os.environ[param])) @@ -455,7 +455,10 @@ class ClickHouseCluster: run_and_check(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name]) node.ip_address = self.get_instance_ip(node.name) node.client = Client(node.ip_address, command=self.client_bin_path) - node.wait_for_start(start_timeout=20.0, connection_timeout=600.0) # seconds + print("Restart node with ip change") + # In builds with sanitizer the server can take a long time to start + node.wait_for_start(start_timeout=60.0, connection_timeout=600.0) # seconds + print("Restarted") return node def get_instance_ip(self, instance_name): @@ -1524,7 +1527,7 @@ class ClickHouseInstance: if self.stay_alive: entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND.replace("{main_config_file}", self.main_config_name) - + print("Entrypoint cmd: {}".format(entrypoint_cmd)) networks = app_net = ipv4_address = ipv6_address = net_aliases = net_alias1 = "" diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/dictionaries/.gitkeep b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/dictionaries/.gitkeep deleted file mode 100644 index 9568549054c..00000000000 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/dictionaries/.gitkeep +++ /dev/null @@ -1 +0,0 @@ -#Keep From 58fbc544cc5eee5c8d914d89895d483ca463ce15 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 20:16:09 +0300 Subject: [PATCH 108/154] Add more comments. --- .../QueryPlan/ReadFromMergeTree.cpp | 18 +++--- src/Processors/QueryPlan/ReadFromMergeTree.h | 5 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 33 ++++++----- .../MergeTree/MergeTreeDataSelectExecutor.h | 57 ++++++++++++------- 4 files changed, 66 insertions(+), 47 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 4515db11067..0672d7ed040 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -43,7 +44,6 @@ struct ReadFromMergeTree::AnalysisResult { RangesInDataParts parts_with_ranges; MergeTreeDataSelectSamplingData sampling; - String query_id; IndexStats index_stats; Names column_names_to_read; ReadFromMergeTree::ReadType read_type = ReadFromMergeTree::ReadType::Default; @@ -830,14 +830,14 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); const auto & select = query_info.query->as(); - auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; MergeTreeDataSelectExecutor::filterPartsByPartition( - metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read.get(), log, result.index_stats); + parts, part_values, metadata_snapshot_base, data, query_info, context, + max_block_numbers_to_read.get(), log, result.index_stats); result.sampling = MergeTreeDataSelectExecutor::getSampling( - select, parts, metadata_snapshot, key_condition, - data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); + select, metadata_snapshot->getColumns().getAllPhysical(), parts, key_condition, + data, metadata_snapshot, context, sample_factor_column_queried, log); if (result.sampling.read_nothing) return result; @@ -885,8 +885,6 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre sum_marks, sum_ranges); - result.query_id = MergeTreeDataSelectExecutor::checkLimits(data, result.parts_with_ranges, context); - ProfileEvents::increment(ProfileEvents::SelectedParts, result.parts_with_ranges.size()); ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks); @@ -905,6 +903,8 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) { auto result = selectRangesToRead(prepared_parts); + auto query_id_holder = MergeTreeDataSelectExecutor::checkLimits(data, result.parts_with_ranges, context); + if (result.parts_with_ranges.empty()) { pipeline.init(Pipe(std::make_shared(getOutputStream().header))); @@ -1048,8 +1048,8 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build processors.emplace_back(processor); // Attach QueryIdHolder if needed - if (!result.query_id.empty()) - pipe.addQueryIdHolder(std::make_shared(result.query_id, data)); + if (query_id_holder) + pipe.addQueryIdHolder(std::move(query_id_holder)); pipeline.init(std::move(pipe)); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 07d45a71e0a..6e1efffdb02 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -1,15 +1,14 @@ #pragma once #include -#include #include -#include -//#include namespace DB { using PartitionIdToMaxBlock = std::unordered_map; +class Pipe; + /// This step is created to read from MergeTree* table. /// For now, it takes a list of parts and creates source from it. class ReadFromMergeTree final : public ISourceStep diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c12759bac97..caf69b8ecfb 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -381,14 +381,14 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( const ASTSelectQuery & select, + NamesAndTypesList available_real_columns, const MergeTreeData::DataPartsVector & parts, - const StorageMetadataPtr & metadata_snapshot, KeyCondition & key_condition, const MergeTreeData & data, - Poco::Logger * log, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, bool sample_factor_column_queried, - NamesAndTypesList available_real_columns, - ContextPtr context) + Poco::Logger * log) { const Settings & settings = context->getSettingsRef(); /// Sampling. @@ -643,7 +643,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( std::optional> MergeTreeDataSelectExecutor::filterPartsByVirtualColumns( const MergeTreeData & data, - MergeTreeData::DataPartsVector & parts, + const MergeTreeData::DataPartsVector & parts, const ASTPtr & query, ContextPtr context) { @@ -666,13 +666,12 @@ std::optional> MergeTreeDataSelectExecutor::filterPar } void MergeTreeDataSelectExecutor::filterPartsByPartition( + MergeTreeData::DataPartsVector & parts, + const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const SelectQueryInfo & query_info, const ContextPtr & context, - const ContextPtr & query_context, - MergeTreeData::DataPartsVector & parts, - const std::optional> & part_values, const PartitionIdToMaxBlock * max_block_numbers_to_read, Poco::Logger * log, ReadFromMergeTree::IndexStats & index_stats) @@ -709,6 +708,7 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( } } + auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; PartFilterCounters part_filter_counters; if (query_context->getSettingsRef().allow_experimental_query_deduplication) selectPartsToReadWithUUIDFilter( @@ -766,7 +766,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd StorageMetadataPtr metadata_snapshot, const SelectQueryInfo & query_info, const ContextPtr & context, - KeyCondition & key_condition, + const KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, @@ -992,7 +992,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd return parts_with_ranges; } -String MergeTreeDataSelectExecutor::checkLimits( +std::shared_ptr MergeTreeDataSelectExecutor::checkLimits( const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, const ContextPtr & context) @@ -1032,7 +1032,10 @@ String MergeTreeDataSelectExecutor::checkLimits( } } - return query_id; + if (!query_id.empty()) + return std::make_shared(query_id, data); + + return nullptr; } static void selectColumnNames( @@ -1135,15 +1138,15 @@ size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead( } const auto & select = query_info.query->as(); - auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; ReadFromMergeTree::IndexStats index_stats; filterPartsByPartition( - metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read.get(), log, index_stats); + parts, part_values, metadata_snapshot_base, data, query_info, + context, max_block_numbers_to_read.get(), log, index_stats); auto sampling = MergeTreeDataSelectExecutor::getSampling( - select, parts, metadata_snapshot, key_condition, - data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); + select, metadata_snapshot->getColumns().getAllPhysical(), parts, key_condition, + data, metadata_snapshot, context, sample_factor_column_queried, log); if (sampling.read_nothing) return 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index b43fb785573..bd2a79f0aee 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -45,16 +45,7 @@ public: QueryProcessingStage::Enum processed_stage, std::shared_ptr max_block_numbers_to_read = nullptr) const; - size_t estimateNumMarksToRead( - MergeTreeData::DataPartsVector parts, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot_base, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - ContextPtr context, - unsigned num_streams, - std::shared_ptr max_block_numbers_to_read = nullptr) const; - + /// The same as read, but with specified set of parts. QueryPlanPtr readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names, @@ -66,6 +57,19 @@ public: unsigned num_streams, std::shared_ptr max_block_numbers_to_read = nullptr) const; + /// Get an estimation for the number of marks we are going to read. + /// Reads nothing. Secondary indexes are not used. + /// This method is used to select best projection for table. + size_t estimateNumMarksToRead( + MergeTreeData::DataPartsVector parts, + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot_base, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + ContextPtr context, + unsigned num_streams, + std::shared_ptr max_block_numbers_to_read = nullptr) const; + private: const MergeTreeData & data; Poco::Logger * log; @@ -131,12 +135,15 @@ private: Poco::Logger * log); public: + /// For given number rows and bytes, get the number of marks to read. + /// It is a minimal number of marks which contain so many rows and bytes. static size_t roundRowsOrBytesToMarks( size_t rows_setting, size_t bytes_setting, size_t rows_granularity, size_t bytes_granularity); + /// The same as roundRowsOrBytesToMarks, but return no more than max_marks. static size_t minMarksForConcurrentRead( size_t rows_setting, size_t bytes_setting, @@ -144,48 +151,58 @@ public: size_t bytes_granularity, size_t max_marks); + /// If possible, filter using expression on virtual columns. + /// Example: SELECT count() FROM table WHERE _part = 'part_name' + /// If expression found, return a set with allowed part names (std::nullopt otherwise). static std::optional> filterPartsByVirtualColumns( const MergeTreeData & data, - MergeTreeData::DataPartsVector & parts, + const MergeTreeData::DataPartsVector & parts, const ASTPtr & query, ContextPtr context); + /// Filter parts using minmax index and partition key. static void filterPartsByPartition( + MergeTreeData::DataPartsVector & parts, + const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const SelectQueryInfo & query_info, const ContextPtr & context, - const ContextPtr & query_context, - MergeTreeData::DataPartsVector & parts, - const std::optional> & part_values, const PartitionIdToMaxBlock * max_block_numbers_to_read, Poco::Logger * log, ReadFromMergeTree::IndexStats & index_stats); + /// Filter parts using primary key and secondary indexes. + /// For every part, select mark ranges to read. static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, const SelectQueryInfo & query_info, const ContextPtr & context, - KeyCondition & key_condition, + const KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, ReadFromMergeTree::IndexStats & index_stats, bool use_skip_indexes); + /// Create expression for sampling. + /// Also, calculate _sample_factor if needed. + /// Also, update key condition with selected sampling range. static MergeTreeDataSelectSamplingData getSampling( const ASTSelectQuery & select, + NamesAndTypesList available_real_columns, const MergeTreeData::DataPartsVector & parts, - const StorageMetadataPtr & metadata_snapshot, KeyCondition & key_condition, const MergeTreeData & data, - Poco::Logger * log, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, bool sample_factor_column_queried, - NamesAndTypesList available_real_columns, - ContextPtr context); + Poco::Logger * log); - static String checkLimits( + /// Check query limits: max_partitions_to_read, max_concurrent_queries. + /// Also, return QueryIdHolder. If not null, we should keep it until query finishes. + static std::shared_ptr checkLimits( const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, const ContextPtr & context); From d9cc77c4290bd854364f943685570106e012b735 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 28 May 2021 21:22:25 +0300 Subject: [PATCH 109/154] keep --- .../configs/dictionaries/.gitkeep | 1 + 1 file changed, 1 insertion(+) create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/configs/dictionaries/.gitkeep diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/dictionaries/.gitkeep b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/dictionaries/.gitkeep new file mode 100644 index 00000000000..9568549054c --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/dictionaries/.gitkeep @@ -0,0 +1 @@ +#Keep From 202e5773bf833aaf6d5d9c14d1859db677375773 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 28 May 2021 21:58:28 +0300 Subject: [PATCH 110/154] fix sentry --- base/daemon/BaseDaemon.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 01e700ebba3..a226b5e21c8 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -350,15 +350,16 @@ private: , calculated_binary_hash, daemon.stored_binary_hash); } #endif - - /// Write crash to system.crash_log table if available. - if (collectCrashLog) - collectCrashLog(sig, thread_num, query_id, stack_trace); + LOG_FATAL(log, "Here. {} {}", sig, SanitizerTrap); /// Send crash report to developers (if configured) if (sig != SanitizerTrap) SentryWriter::onFault(sig, error_message, stack_trace); + /// Write crash to system.crash_log table if available. + if (collectCrashLog) + collectCrashLog(sig, thread_num, query_id, stack_trace); + /// When everything is done, we will try to send these error messages to client. if (thread_ptr) thread_ptr->onFatalError(); From 00ce7feafbf109f8a38256bc656f93e30423c7f4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 28 May 2021 22:00:15 +0300 Subject: [PATCH 111/154] better --- base/daemon/BaseDaemon.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index a226b5e21c8..857a758bc71 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -350,7 +350,6 @@ private: , calculated_binary_hash, daemon.stored_binary_hash); } #endif - LOG_FATAL(log, "Here. {} {}", sig, SanitizerTrap); /// Send crash report to developers (if configured) if (sig != SanitizerTrap) From 2a631aaf0835d17da3aa3e79fc9dc5f3e3e1b728 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 28 May 2021 21:17:16 +0300 Subject: [PATCH 112/154] Final fixes --- base/daemon/BaseDaemon.cpp | 1 - base/daemon/SentryWriter.cpp | 1 - programs/client/Client.cpp | 3 +- programs/copier/Internals.h | 1 - src/Bridge/IBridgeHelper.cpp | 1 - src/Bridge/LibraryBridgeHelper.cpp | 1 - src/Bridge/XDBCBridgeHelper.h | 2 -- src/Common/Config/ConfigProcessor.h | 3 -- src/Common/Config/ConfigReloader.cpp | 8 ++--- .../{createFile.cpp => FileSystemHelpers.cpp} | 33 ++++++++++++++++--- src/Common/FileSystemHelpers.h | 14 ++++++++ src/Common/FileUpdatesTracker.h | 5 ++- src/Common/createFile.h | 9 ----- src/Common/filesystemHelpers.cpp | 2 -- src/Databases/DatabaseFactory.cpp | 2 -- src/Databases/DatabaseOnDisk.cpp | 7 ++-- .../MySQL/DatabaseConnectionMySQL.cpp | 2 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Dictionaries/DictionarySourceHelpers.h | 3 +- src/Dictionaries/FileDictionarySource.cpp | 10 ++---- src/Disks/DiskLocal.cpp | 9 ++--- src/Disks/IDiskRemote.cpp | 8 ++--- src/Disks/StoragePolicy.cpp | 2 -- src/Formats/FormatSchemaInfo.cpp | 11 ++----- src/Interpreters/DatabaseCatalog.cpp | 5 ++- .../ExternalLoaderXMLConfigRepository.cpp | 9 ++--- src/Interpreters/InterpreterDropQuery.cpp | 2 -- src/Server/HTTP/HTTPServerResponse.cpp | 4 +-- src/Server/HTTPHandler.cpp | 1 - src/Server/StaticRequestHandler.cpp | 5 ++- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 -- src/Storages/MergeTree/IMergeTreeReader.cpp | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 1 - .../MergeTree/MergeTreeDataPartCompact.cpp | 1 - .../MergeTree/MergeTreeDataPartInMemory.cpp | 1 - .../MergeTree/MergeTreeDataPartWide.cpp | 1 - .../MergeTree/MergeTreeDataSelectExecutor.cpp | 1 - .../MergeTree/MergeTreeDataWriter.cpp | 1 - .../MergeTree/MergeTreeMarksLoader.cpp | 1 - .../MergeTree/MergeTreeMutationEntry.cpp | 3 -- .../MergeTree/MergeTreeReaderCompact.cpp | 1 - .../MergeTree/MergedBlockOutputStream.cpp | 1 - src/Storages/MergeTree/checkDataPart.cpp | 1 - src/Storages/StorageJoin.cpp | 2 -- src/Storages/StorageXDBC.cpp | 1 - .../examples/remove_symlink_directory.cpp | 2 +- 47 files changed, 74 insertions(+), 115 deletions(-) rename src/Common/{createFile.cpp => FileSystemHelpers.cpp} (66%) create mode 100644 src/Common/FileSystemHelpers.h delete mode 100644 src/Common/createFile.h diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 310e656dac2..6aa65942445 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -57,7 +57,6 @@ #include #include #include -#include #include #if !defined(ARCADIA_BUILD) diff --git a/base/daemon/SentryWriter.cpp b/base/daemon/SentryWriter.cpp index 3719bb74c52..3571c64edd6 100644 --- a/base/daemon/SentryWriter.cpp +++ b/base/daemon/SentryWriter.cpp @@ -1,6 +1,5 @@ #include -#include #include #include diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 7dea4c9375b..233134f0529 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include #include @@ -88,7 +87,7 @@ #include #include #include -#include +#include #if !defined(ARCADIA_BUILD) # include diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index 7e45c0ea2ee..9e40d7ebd7b 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Bridge/IBridgeHelper.cpp b/src/Bridge/IBridgeHelper.cpp index 93ca359c578..b6f3446d0a6 100644 --- a/src/Bridge/IBridgeHelper.cpp +++ b/src/Bridge/IBridgeHelper.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index c589d0ce09e..7d378941d25 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Bridge/XDBCBridgeHelper.h b/src/Bridge/XDBCBridgeHelper.h index 299df6ff888..b170a55e438 100644 --- a/src/Bridge/XDBCBridgeHelper.h +++ b/src/Bridge/XDBCBridgeHelper.h @@ -5,10 +5,8 @@ #include #include #include -#include #include #include -#include #include #include #include diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 5b16bc0cb1b..e13e66547a3 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -15,12 +15,9 @@ #include #include #include -#include -#include #include #include #include - #include diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index b2aeb899e4d..e39ec7c590a 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -1,11 +1,12 @@ #include "ConfigReloader.h" #include -#include #include #include #include "ConfigProcessor.h" #include +#include + namespace fs = std::filesystem; @@ -170,10 +171,7 @@ struct ConfigReloader::FileWithTimestamp void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path_to_add) { if (!path_to_add.empty() && fs::exists(path_to_add)) - { - fs::file_time_type fs_time = fs::last_write_time(path_to_add); - files.emplace(path_to_add, fs::file_time_type::clock::to_time_t(fs_time)); - } + files.emplace(path_to_add, FS::getModificationTime(path_to_add)); } bool ConfigReloader::FilesChangesTracker::isDifferOrNewerThan(const FilesChangesTracker & rhs) diff --git a/src/Common/createFile.cpp b/src/Common/FileSystemHelpers.cpp similarity index 66% rename from src/Common/createFile.cpp rename to src/Common/FileSystemHelpers.cpp index 64e344f1f22..7862d2ccdf5 100644 --- a/src/Common/createFile.cpp +++ b/src/Common/FileSystemHelpers.cpp @@ -1,10 +1,12 @@ -#include "createFile.h" +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include +#include +#include namespace DB { @@ -61,4 +63,25 @@ bool canWrite(const std::string & path) DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); } +time_t getModificationTime(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + return st.st_mtime; + DB::throwFromErrnoWithPath("Cannot check modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + +Poco::Timestamp getModificationTimestamp(const std::string & path) +{ + return Poco::Timestamp::fromEpochTime(getModificationTime(path)); +} + +void setModificationTime(const std::string & path, time_t time) +{ + struct utimbuf tb; + tb.actime = time; + tb.modtime = time; + if (utime(path.c_str(), &tb) != 0) + DB::throwFromErrnoWithPath("Cannot set modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} } diff --git a/src/Common/FileSystemHelpers.h b/src/Common/FileSystemHelpers.h new file mode 100644 index 00000000000..bc11fd3acd1 --- /dev/null +++ b/src/Common/FileSystemHelpers.h @@ -0,0 +1,14 @@ +#pragma once +#include + +namespace FS +{ +bool createFile(const std::string & path); + +bool canRead(const std::string & path); +bool canWrite(const std::string & path); + +time_t getModificationTime(const std::string & path); +Poco::Timestamp getModificationTimestamp(const std::string & path); +void setModificationTime(const std::string & path, time_t time); +} diff --git a/src/Common/FileUpdatesTracker.h b/src/Common/FileUpdatesTracker.h index 471a62cf07a..5fcbbb0bf6c 100644 --- a/src/Common/FileUpdatesTracker.h +++ b/src/Common/FileUpdatesTracker.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -31,8 +32,6 @@ public: private: Poco::Timestamp getLastModificationTime() const { - fs::file_time_type fs_time = fs::last_write_time(path); - auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); - return Poco::Timestamp(micro_sec.count()); + return FS::getModificationTimestamp(path); } }; diff --git a/src/Common/createFile.h b/src/Common/createFile.h deleted file mode 100644 index f96d141d39e..00000000000 --- a/src/Common/createFile.h +++ /dev/null @@ -1,9 +0,0 @@ -#pragma once -#include - -namespace FS -{ -bool createFile(const std::string & path); -bool canRead(const std::string & path); -bool canWrite(const std::string & path); -} diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 39b8137f688..edf54961e16 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -6,8 +6,6 @@ # include #endif #include -#include -#include #include #include diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index d179adb18cf..e09860e79b6 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -11,8 +11,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f50adf54c3f..406ca2a381a 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include namespace fs = std::filesystem; @@ -542,10 +542,7 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n String table_metadata_path = getObjectMetadataPath(object_name); if (fs::exists(table_metadata_path)) - { - fs::file_time_type fs_time = fs::last_write_time(table_metadata_path); - return fs::file_time_type::clock::to_time_t(fs_time); - } + return FS::getModificationTime(table_metadata_path); else return static_cast(0); } diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 8901a3f77a1..46beccb1ee6 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -25,7 +25,7 @@ # include # include # include -# include +# include namespace fs = std::filesystem; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 5d75915ea91..7080b3293d7 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include namespace fs = std::filesystem; diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index 1febf921e07..0b799fa7cce 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -3,14 +3,13 @@ #include #include - -#include #include #include #include #include #include + namespace DB { diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index fd0583a4078..a516d6f79ee 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -1,9 +1,5 @@ #include "FileDictionarySource.h" - #include - -#include - #include #include #include @@ -14,6 +10,8 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" #include "DictionarySourceHelpers.h" +#include + namespace fs = std::filesystem; @@ -70,9 +68,7 @@ std::string FileDictionarySource::toString() const Poco::Timestamp FileDictionarySource::getLastModification() const { - fs::file_time_type fs_time = fs::last_write_time(filepath); - auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); - return Poco::Timestamp(micro_sec.count()); + return FS::getModificationTimestamp(filepath); } diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index e31731a7edc..93f60a7a665 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -7,8 +7,7 @@ #include #include #include -#include -#include +#include #include #include @@ -260,14 +259,12 @@ void DiskLocal::listFiles(const String & path, std::vector & file_names) void DiskLocal::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - fs::last_write_time(fs::path(disk_path) / path, static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); + FS::setModificationTime(fs::path(disk_path) / path, timestamp.epochTime()); } Poco::Timestamp DiskLocal::getLastModified(const String & path) { - fs::file_time_type fs_time = fs::last_write_time(fs::path(disk_path) / path); - auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); - return Poco::Timestamp(micro_sec.count()); + return FS::getModificationTimestamp(fs::path(disk_path) / path); } void DiskLocal::createHardLink(const String & src_path, const String & dst_path) diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index f32b046b5ec..691e53ac553 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -429,16 +430,13 @@ void IDiskRemote::listFiles(const String & path, std::vector & file_name void IDiskRemote::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - fs::last_write_time(fs::path(metadata_path) / path, - static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); + FS::setModificationTime(fs::path(metadata_path) / path, timestamp.epochTime()); } Poco::Timestamp IDiskRemote::getLastModified(const String & path) { - fs::file_time_type fs_time = fs::last_write_time(fs::path(metadata_path) / path); - auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); - return Poco::Timestamp(micro_sec.count()); + return FS::getModificationTimestamp(fs::path(metadata_path) / path); } diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index cff2685ca24..efedff9929f 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -8,8 +8,6 @@ #include -#include - namespace { diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 25ed6051382..bd1bd9367b2 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -74,14 +74,6 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & if (!path.has_extension() && !default_file_extension.empty()) path = path.parent_path() / (path.stem().string() + '.' + default_file_extension); - fs::path default_schema_directory_path(default_schema_directory()); - auto path_is_subdirectory_of = [](fs::path inner, const fs::path & outer) -> bool - { - while (inner != outer && inner != "/") - inner = inner.parent_path(); - return inner == outer; - }; - if (path.is_absolute()) { if (is_server) @@ -89,12 +81,13 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (path.has_parent_path() && !path_is_subdirectory_of(path, default_schema_directory_path)) + else if (!fs::weakly_canonical(path).string().starts_with(fs::weakly_canonical(default_schema_directory()).string())) { if (is_server) throw Exception( "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(), ErrorCodes::BAD_ARGUMENTS); + fs::path default_schema_directory_path(default_schema_directory()); if (default_schema_directory_path.is_absolute()) path = default_schema_directory_path; else diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 0794cb52a49..fe8b4348820 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -18,6 +17,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -782,8 +782,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr } addUUIDMapping(table_id.uuid); - fs::file_time_type fs_time = fs::last_write_time(dropped_metadata_path); - drop_time = fs::file_time_type::clock::to_time_t(fs_time); + drop_time = FS::getModificationTime(dropped_metadata_path); } std::lock_guard lock(tables_marked_dropped_mutex); diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index 4c6af554fae..49c95516c83 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -3,12 +3,11 @@ #include #include #include - #include -#include -#include +#include #include + namespace fs = std::filesystem; namespace DB @@ -21,9 +20,7 @@ ExternalLoaderXMLConfigRepository::ExternalLoaderXMLConfigRepository( Poco::Timestamp ExternalLoaderXMLConfigRepository::getUpdateTime(const std::string & definition_entity_name) { - fs::file_time_type fs_time = fs::last_write_time(definition_entity_name); - auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); - return Poco::Timestamp(micro_sec.count()); + return FS::getModificationTimestamp(definition_entity_name); } std::set ExternalLoaderXMLConfigRepository::getAllLoadablesDefinitionNames() diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 24cff90caea..a9d43c6e9cc 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -1,5 +1,3 @@ -#include - #include #include #include diff --git a/src/Server/HTTP/HTTPServerResponse.cpp b/src/Server/HTTP/HTTPServerResponse.cpp index db5cfb132e3..25e7604a515 100644 --- a/src/Server/HTTP/HTTPServerResponse.cpp +++ b/src/Server/HTTP/HTTPServerResponse.cpp @@ -1,11 +1,8 @@ #include - #include - #include #include #include -#include #include #include #include @@ -13,6 +10,7 @@ #include #include + namespace DB { diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 8aed5d20f74..907a778a07f 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -39,7 +39,6 @@ #include #include -#include #include #include #include diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index 4746ab6d1e0..e96f3a581a9 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -138,11 +138,10 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) if (startsWith(response_expression, file_prefix)) { - const auto & file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size()); + const auto & file_name = response_expression.substr(file_prefix.size() + 1, response_expression.size() - file_prefix.size()); fs::path user_files_absolute_path = fs::canonical(fs::path(server.context()->getUserFilesPath())); - /// Fixme: it does not work with fs::path(user_files_absolute_path) / file_name - String file_path = fs::canonical(user_files_absolute_path.string() + "/" + file_name); + String file_path = fs::weakly_canonical(user_files_absolute_path / file_name); if (!fs::exists(file_path)) throw Exception("Invalid file name " + file_path + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index a8a49680dd7..53640b41507 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -17,8 +17,6 @@ #include #include -#include - #include namespace zkutil diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 52d3e7ca9ab..14187564536 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1c9507d8cb1..fb144362b3c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3326,7 +3326,7 @@ MergeTreeData::getDetachedParts() const for (const auto & [path, disk] : getRelativeDataPathsWithDisks()) { - for (auto it = disk->iterateDirectory(path + MergeTreeData::DETACHED_DIR_NAME); it->isValid(); it->next()) + for (auto it = disk->iterateDirectory(fs::path(path) / MergeTreeData::DETACHED_DIR_NAME); it->isValid(); it->next()) { res.emplace_back(); auto & part = res.back(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 8f5c4b6a848..1df97dc9241 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 5c1a3b01804..7b1641a0537 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -2,7 +2,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 473f2e598a9..e929bfc6862 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 84378faa513..4eebd729dd8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -1,5 +1,4 @@ #include "MergeTreeDataPartWide.h" -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 4e11669b8d7..00bc4306e4b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -3,7 +3,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 3bf33d45802..0b05650b42c 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index c5a99b128e9..717179e5f26 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 49c4e93eb1d..2aefb3df2be 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -4,9 +4,6 @@ #include #include -#include -#include - #include diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index da28f75b57f..783a37cce60 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -2,7 +2,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index e8a790bce3f..6de01caa833 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -1,6 +1,5 @@ #include #include -#include #include diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 5de2b8707c3..8a234833da7 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index f0129e95495..7a448e1dde9 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -14,9 +14,7 @@ #include #include #include - #include /// toLower -#include #include #include diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index 88d54669889..9cffc32fda1 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/examples/remove_symlink_directory.cpp b/src/Storages/examples/remove_symlink_directory.cpp index a90083233e7..ae898edb49b 100644 --- a/src/Storages/examples/remove_symlink_directory.cpp +++ b/src/Storages/examples/remove_symlink_directory.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include namespace fs = std::filesystem; From 0a3589524ea5b5c412d5f55b9c27666fe5d14b99 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 29 May 2021 00:57:53 +0300 Subject: [PATCH 113/154] File movement --- programs/client/Client.cpp | 2 +- src/Common/Config/ConfigReloader.cpp | 2 +- src/Common/FileSystemHelpers.cpp | 87 ------------------- src/Common/FileSystemHelpers.h | 14 --- src/Common/FileUpdatesTracker.h | 2 +- src/Common/filesystemHelpers.cpp | 77 ++++++++++++++++ src/Common/filesystemHelpers.h | 12 +++ src/Databases/DatabaseOnDisk.cpp | 2 +- .../MySQL/DatabaseConnectionMySQL.cpp | 2 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Dictionaries/FileDictionarySource.cpp | 2 +- src/Disks/DiskLocal.cpp | 2 +- src/Disks/IDiskRemote.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 2 +- .../ExternalLoaderXMLConfigRepository.cpp | 2 +- .../examples/remove_symlink_directory.cpp | 2 +- 16 files changed, 101 insertions(+), 113 deletions(-) delete mode 100644 src/Common/FileSystemHelpers.cpp delete mode 100644 src/Common/FileSystemHelpers.h diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 233134f0529..89899b86eb0 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -87,7 +87,7 @@ #include #include #include -#include +#include #if !defined(ARCADIA_BUILD) # include diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index e39ec7c590a..01bb4f556d6 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -5,7 +5,7 @@ #include #include "ConfigProcessor.h" #include -#include +#include namespace fs = std::filesystem; diff --git a/src/Common/FileSystemHelpers.cpp b/src/Common/FileSystemHelpers.cpp deleted file mode 100644 index 7862d2ccdf5..00000000000 --- a/src/Common/FileSystemHelpers.cpp +++ /dev/null @@ -1,87 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ -extern const int PATH_ACCESS_DENIED; -extern const int CANNOT_CREATE_FILE; -} -} - -namespace FS -{ - -/// Copy from Poco::createFile -bool createFile(const std::string & path) -{ - int n = open(path.c_str(), O_WRONLY | O_CREAT | O_EXCL, S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP | S_IROTH | S_IWOTH); - if (n != -1) - { - close(n); - return true; - } - DB::throwFromErrnoWithPath("Cannot create file: " + path, path, DB::ErrorCodes::CANNOT_CREATE_FILE); -} - -bool canRead(const std::string & path) -{ - struct stat st; - if (stat(path.c_str(), &st) == 0) - { - if (st.st_uid == geteuid()) - return (st.st_mode & S_IRUSR) != 0; - else if (st.st_gid == getegid()) - return (st.st_mode & S_IRGRP) != 0; - else - return (st.st_mode & S_IROTH) != 0 || geteuid() == 0; - } - DB::throwFromErrnoWithPath("Cannot check read access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); -} - - -bool canWrite(const std::string & path) -{ - struct stat st; - if (stat(path.c_str(), &st) == 0) - { - if (st.st_uid == geteuid()) - return (st.st_mode & S_IWUSR) != 0; - else if (st.st_gid == getegid()) - return (st.st_mode & S_IWGRP) != 0; - else - return (st.st_mode & S_IWOTH) != 0 || geteuid() == 0; - } - DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); -} - -time_t getModificationTime(const std::string & path) -{ - struct stat st; - if (stat(path.c_str(), &st) == 0) - return st.st_mtime; - DB::throwFromErrnoWithPath("Cannot check modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); -} - -Poco::Timestamp getModificationTimestamp(const std::string & path) -{ - return Poco::Timestamp::fromEpochTime(getModificationTime(path)); -} - -void setModificationTime(const std::string & path, time_t time) -{ - struct utimbuf tb; - tb.actime = time; - tb.modtime = time; - if (utime(path.c_str(), &tb) != 0) - DB::throwFromErrnoWithPath("Cannot set modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); -} -} diff --git a/src/Common/FileSystemHelpers.h b/src/Common/FileSystemHelpers.h deleted file mode 100644 index bc11fd3acd1..00000000000 --- a/src/Common/FileSystemHelpers.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once -#include - -namespace FS -{ -bool createFile(const std::string & path); - -bool canRead(const std::string & path); -bool canWrite(const std::string & path); - -time_t getModificationTime(const std::string & path); -Poco::Timestamp getModificationTimestamp(const std::string & path); -void setModificationTime(const std::string & path, time_t time); -} diff --git a/src/Common/FileUpdatesTracker.h b/src/Common/FileUpdatesTracker.h index 5fcbbb0bf6c..9fe37c2a75d 100644 --- a/src/Common/FileUpdatesTracker.h +++ b/src/Common/FileUpdatesTracker.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace fs = std::filesystem; diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index edf54961e16..d7c556d1c6a 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -7,7 +7,13 @@ #endif #include #include +#include #include +#include +#include +#include +#include +#include namespace fs = std::filesystem; @@ -21,6 +27,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int CANNOT_STATVFS; extern const int PATH_ACCESS_DENIED; + extern const int CANNOT_CREATE_FILE; } @@ -125,3 +132,73 @@ bool pathStartsWith(const String & path, const String & prefix_path) } } + + +/// Copied from Poco::File +namespace FS +{ + +bool createFile(const std::string & path) +{ + int n = open(path.c_str(), O_WRONLY | O_CREAT | O_EXCL, S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP | S_IROTH | S_IWOTH); + if (n != -1) + { + close(n); + return true; + } + DB::throwFromErrnoWithPath("Cannot create file: " + path, path, DB::ErrorCodes::CANNOT_CREATE_FILE); +} + +bool canRead(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + { + if (st.st_uid == geteuid()) + return (st.st_mode & S_IRUSR) != 0; + else if (st.st_gid == getegid()) + return (st.st_mode & S_IRGRP) != 0; + else + return (st.st_mode & S_IROTH) != 0 || geteuid() == 0; + } + DB::throwFromErrnoWithPath("Cannot check read access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + + +bool canWrite(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + { + if (st.st_uid == geteuid()) + return (st.st_mode & S_IWUSR) != 0; + else if (st.st_gid == getegid()) + return (st.st_mode & S_IWGRP) != 0; + else + return (st.st_mode & S_IWOTH) != 0 || geteuid() == 0; + } + DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + +time_t getModificationTime(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + return st.st_mtime; + DB::throwFromErrnoWithPath("Cannot check modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + +Poco::Timestamp getModificationTimestamp(const std::string & path) +{ + return Poco::Timestamp::fromEpochTime(getModificationTime(path)); +} + +void setModificationTime(const std::string & path, time_t time) +{ + struct utimbuf tb; + tb.actime = time; + tb.modtime = time; + if (utime(path.c_str(), &tb) != 0) + DB::throwFromErrnoWithPath("Cannot set modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} +} diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 0ffbf19643d..b7525a64fae 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -36,3 +36,15 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p bool pathStartsWith(const String & path, const String & prefix_path); } + +namespace FS +{ +bool createFile(const std::string & path); + +bool canRead(const std::string & path); +bool canWrite(const std::string & path); + +time_t getModificationTime(const std::string & path); +Poco::Timestamp getModificationTimestamp(const std::string & path); +void setModificationTime(const std::string & path, time_t time); +} diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 406ca2a381a..e1b5f51a8fc 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include namespace fs = std::filesystem; diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 46beccb1ee6..ccc1c058e74 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -25,7 +25,7 @@ # include # include # include -# include +# include namespace fs = std::filesystem; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 7080b3293d7..53d34fb07b1 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include namespace fs = std::filesystem; diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index a516d6f79ee..a22263575f4 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -10,7 +10,7 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" #include "DictionarySourceHelpers.h" -#include +#include namespace fs = std::filesystem; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 93f60a7a665..cae166d2110 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index 691e53ac553..b30e9613ed8 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index fe8b4348820..17edc68dcc3 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index 49c95516c83..00c31110ea4 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/examples/remove_symlink_directory.cpp b/src/Storages/examples/remove_symlink_directory.cpp index ae898edb49b..db436c0a608 100644 --- a/src/Storages/examples/remove_symlink_directory.cpp +++ b/src/Storages/examples/remove_symlink_directory.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include namespace fs = std::filesystem; From 386f31fddf790bb88f280f67391e0ba3e0447ae5 Mon Sep 17 00:00:00 2001 From: Ahmed Dardery Date: Sun, 30 May 2021 14:08:19 +0200 Subject: [PATCH 114/154] fixed missing "weight" in definition --- .../aggregate-functions/reference/quantiletdigestweighted.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md index 32d174136e0..a726938d07f 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md @@ -18,7 +18,7 @@ When using multiple `quantile*` functions with different levels in a query, the **Syntax** ``` sql -quantileTDigest(level)(expr) +quantileTDigest(level)(expr, weight) ``` Alias: `medianTDigest`. From 69816e6effcde4ac55481d49a3319be33523956d Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 29 May 2021 08:44:07 +0300 Subject: [PATCH 115/154] Fix checks --- programs/client/Client.cpp | 2 +- src/Common/filesystemHelpers.cpp | 1 - src/Common/ya.make | 1 - src/Dictionaries/FileDictionarySource.cpp | 4 ---- src/Dictionaries/LibraryDictionarySource.cpp | 2 -- src/Disks/DiskLocal.cpp | 1 - src/Formats/FormatSchemaInfo.cpp | 8 ++++---- src/Server/StaticRequestHandler.cpp | 4 +++- .../examples/get_current_inserts_in_replicated.cpp | 3 ++- 9 files changed, 10 insertions(+), 16 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 89899b86eb0..cd8162e1af2 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -278,7 +278,7 @@ private: /// Set path for format schema files if (config().has("format_schema_path")) - context->setFormatSchemaPath(fs::absolute(config().getString("format_schema_path"))); + context->setFormatSchemaPath(fs::weakly_canonical(config().getString("format_schema_path"))); /// Initialize query_id_formats if any if (config().has("query_id_formats")) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index d7c556d1c6a..4855500b776 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include diff --git a/src/Common/ya.make b/src/Common/ya.make index c91f794e97c..57b60e9cce5 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -95,7 +95,6 @@ SRCS( ZooKeeper/ZooKeeperNodeCache.cpp checkStackSize.cpp clearPasswordFromCommandLine.cpp - createFile.cpp createHardLink.cpp escapeForFileName.cpp filesystemHelpers.cpp diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index a22263575f4..239c13e71c2 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -1,5 +1,4 @@ #include "FileDictionarySource.h" -#include #include #include #include @@ -10,11 +9,8 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" #include "DictionarySourceHelpers.h" -#include -namespace fs = std::filesystem; - namespace DB { static const UInt64 max_block_size = 8192; diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 70022a9b721..0b8b52a2d67 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -7,8 +7,6 @@ #include #include #include -#include -#include #include #include diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index cae166d2110..768d1c2354a 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index bd1bd9367b2..1b406f639ea 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -81,12 +81,12 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (!fs::weakly_canonical(path).string().starts_with(fs::weakly_canonical(default_schema_directory()).string())) + else if (path.has_parent_path() && !fs::weakly_canonical(path).string().starts_with(fs::weakly_canonical(default_schema_directory()).string())) { if (is_server) - throw Exception( - "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(), - ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: {} ({} not in {})", + path.string()); fs::path default_schema_directory_path(default_schema_directory()); if (default_schema_directory_path.is_absolute()) path = default_schema_directory_path; diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index e96f3a581a9..19b91ae9c42 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -138,7 +138,9 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) if (startsWith(response_expression, file_prefix)) { - const auto & file_name = response_expression.substr(file_prefix.size() + 1, response_expression.size() - file_prefix.size()); + auto file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size()); + if (file_name.starts_with('/')) + file_name = file_name.substr(1); fs::path user_files_absolute_path = fs::canonical(fs::path(server.context()->getUserFilesPath())); String file_path = fs::weakly_canonical(user_files_absolute_path / file_name); diff --git a/src/Storages/examples/get_current_inserts_in_replicated.cpp b/src/Storages/examples/get_current_inserts_in_replicated.cpp index 62dcce37d4e..56364997643 100644 --- a/src/Storages/examples/get_current_inserts_in_replicated.cpp +++ b/src/Storages/examples/get_current_inserts_in_replicated.cpp @@ -4,11 +4,12 @@ #include #include #include - #include +#include #include +namespace fs = std::filesystem; using namespace DB; From a206b76e5b5620fc76aaff0a9a11797ce7505ce8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 31 May 2021 10:31:45 +0800 Subject: [PATCH 116/154] Fix empty part set with force_use_projection = 1 --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 14 +++++++++----- .../01710_force_use_projection.reference | 1 + .../0_stateless/01710_force_use_projection.sql | 17 +++++++++++++++++ 3 files changed, 27 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01710_force_use_projection.reference create mode 100644 tests/queries/0_stateless/01710_force_use_projection.sql diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 20cadb53098..e2f17915624 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -144,11 +144,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( const auto & settings = context->getSettingsRef(); if (!query_info.projection) { - if (settings.allow_experimental_projection_optimization && settings.force_optimize_projection - && !metadata_snapshot->projections.empty()) - throw Exception("No projection is used when allow_experimental_projection_optimization = 1", ErrorCodes::PROJECTION_NOT_USED); - - return readFromParts( + auto plan = readFromParts( data.getDataPartsVector(), column_names_to_return, metadata_snapshot, @@ -159,6 +155,14 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( num_streams, max_block_numbers_to_read, query_info.merge_tree_data_select_cache.get()); + + if (plan->isInitialized() && settings.allow_experimental_projection_optimization && settings.force_optimize_projection + && !metadata_snapshot->projections.empty()) + throw Exception( + "No projection is used when allow_experimental_projection_optimization = 1 and force_optimize_projection = 1", + ErrorCodes::PROJECTION_NOT_USED); + + return plan; } LOG_DEBUG( diff --git a/tests/queries/0_stateless/01710_force_use_projection.reference b/tests/queries/0_stateless/01710_force_use_projection.reference new file mode 100644 index 00000000000..f1ad7de51f0 --- /dev/null +++ b/tests/queries/0_stateless/01710_force_use_projection.reference @@ -0,0 +1 @@ +3 1 diff --git a/tests/queries/0_stateless/01710_force_use_projection.sql b/tests/queries/0_stateless/01710_force_use_projection.sql new file mode 100644 index 00000000000..8931c65e34e --- /dev/null +++ b/tests/queries/0_stateless/01710_force_use_projection.sql @@ -0,0 +1,17 @@ +drop table if exists tp; + +create table tp (d1 Int32, d2 Int32, eventcnt Int64, projection p (select sum(eventcnt) group by d1)) engine = MergeTree order by (d1, d2); + +set allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + +select sum(eventcnt) eventcnt, d1 from tp group by d1; + +select avg(eventcnt) eventcnt, d1 from tp group by d1; + +insert into tp values (1, 2, 3); + +select sum(eventcnt) eventcnt, d1 from tp group by d1; + +select avg(eventcnt) eventcnt, d1 from tp group by d1; -- { serverError 584 } + +drop table tp; From 3be331edeb3119b59bf40cabcd96f1fe7e513d34 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 31 May 2021 12:24:09 +0800 Subject: [PATCH 117/154] Better exception for invalid projection creation --- src/Parsers/ASTProjectionSelectQuery.cpp | 2 -- src/Storages/ProjectionsDescription.cpp | 6 +++--- .../01710_projection_group_by_order_by.reference | 0 .../0_stateless/01710_projection_group_by_order_by.sql | 5 +++++ 4 files changed, 8 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01710_projection_group_by_order_by.reference create mode 100644 tests/queries/0_stateless/01710_projection_group_by_order_by.sql diff --git a/src/Parsers/ASTProjectionSelectQuery.cpp b/src/Parsers/ASTProjectionSelectQuery.cpp index 58943ed0430..cc0d387b933 100644 --- a/src/Parsers/ASTProjectionSelectQuery.cpp +++ b/src/Parsers/ASTProjectionSelectQuery.cpp @@ -134,8 +134,6 @@ ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const if (groupBy()) select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, groupBy()->clone()); // Get rid of orderBy. It's used for projection definition only - if (orderBy()) - select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, orderBy()->clone()); return node; } diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index b2a06018054..dd48b23ecc3 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -172,15 +172,15 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const metadata.sorting_key = KeyDescription::getSortingKeyFromAST({}, metadata.columns, query_context, {}); metadata.primary_key = KeyDescription::getKeyFromAST({}, metadata.columns, query_context); } - if (query_select.orderBy()) + if (query.orderBy()) throw Exception( "When aggregation is used in projection, ORDER BY cannot be specified", ErrorCodes::ILLEGAL_PROJECTION); } else { result.type = ProjectionDescription::Type::Normal; - metadata.sorting_key = KeyDescription::getSortingKeyFromAST(query_select.orderBy(), metadata.columns, query_context, {}); - metadata.primary_key = KeyDescription::getKeyFromAST(query_select.orderBy(), metadata.columns, query_context); + metadata.sorting_key = KeyDescription::getSortingKeyFromAST(query.orderBy(), metadata.columns, query_context, {}); + metadata.primary_key = KeyDescription::getKeyFromAST(query.orderBy(), metadata.columns, query_context); } metadata.primary_key.definition_ast = nullptr; result.metadata = std::make_shared(metadata); diff --git a/tests/queries/0_stateless/01710_projection_group_by_order_by.reference b/tests/queries/0_stateless/01710_projection_group_by_order_by.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_projection_group_by_order_by.sql b/tests/queries/0_stateless/01710_projection_group_by_order_by.sql new file mode 100644 index 00000000000..9370e9d36ce --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_group_by_order_by.sql @@ -0,0 +1,5 @@ +drop table if exists tp; + +create table tp (type Int32, eventcnt UInt64, projection p (select sum(eventcnt), type group by type order by sum(eventcnt))) engine = MergeTree order by type; -- { serverError 583 } + +drop table if exists tp; From 054fe1cf2fd5ce1cacc074f19b22be9b6a080d66 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 May 2021 14:06:32 +0300 Subject: [PATCH 118/154] Fix --- src/Formats/FormatSchemaInfo.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 1b406f639ea..2605c0bdf04 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -74,6 +74,7 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & if (!path.has_extension() && !default_file_extension.empty()) path = path.parent_path() / (path.stem().string() + '.' + default_file_extension); + fs::path default_schema_directory_path(default_schema_directory()); if (path.is_absolute()) { if (is_server) @@ -81,17 +82,13 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (path.has_parent_path() && !fs::weakly_canonical(path).string().starts_with(fs::weakly_canonical(default_schema_directory()).string())) + else if (path.has_parent_path() && !fs::weakly_canonical(default_schema_directory_path / path).string().starts_with(fs::weakly_canonical(default_schema_directory_path).string())) { if (is_server) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: {} ({} not in {})", path.string()); - fs::path default_schema_directory_path(default_schema_directory()); - if (default_schema_directory_path.is_absolute()) - path = default_schema_directory_path; - else - path /= default_schema_directory_path; + path = default_schema_directory_path / path; schema_path = path.filename(); schema_directory = path.parent_path() / ""; } From 2fe75e33eea801c5941f1db046a74916e09e6e15 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 May 2021 16:03:30 +0300 Subject: [PATCH 119/154] Replace Poco::Event with std future in synchronous calls in ZooKeeper client --- src/Common/ZooKeeper/IKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeper.cpp | 255 +++++++++++------- src/Common/ZooKeeper/ZooKeeper.h | 43 +-- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- 5 files changed, 196 insertions(+), 110 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 5e11687eab5..d69ab34ea43 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -411,7 +411,7 @@ public: * - whenever you receive exception with ZSESSIONEXPIRED code or method isExpired returns true, * the ZooKeeper instance is no longer usable - you may only destroy it and probably create another. * - whenever session is expired or ZooKeeper instance is destroying, all callbacks are notified with special event. - * - data for callbacks must be alive when ZooKeeper instance is alive. + * - data for callbacks must be alive when ZooKeeper instance is alive, so try to avoid capturing references in callbacks, it's error-prone. */ class IKeeper { diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 43092f719ec..6d64aff99bf 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -238,24 +238,25 @@ Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings Coordination::Stat * stat, Coordination::WatchCallback watch_callback) { - Coordination::Error code = Coordination::Error::ZOK; - Poco::Event event; + auto future_result = tryAsyncGetChildrenNoThrow(path, watch_callback); - auto callback = [&](const Coordination::ListResponse & response) + if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { - SCOPE_EXIT(event.set()); - code = response.error; + impl->finalize(); + return Coordination::Error::ZOPERATIONTIMEOUT; + } + else + { + auto response = future_result.get(); + Coordination::Error code = response.error; if (code == Coordination::Error::ZOK) { res = response.names; if (stat) *stat = response.stat; } - }; - - impl->list(path, callback, watch_callback); - event.wait(); - return code; + return code; + } } Strings ZooKeeper::getChildren( @@ -298,20 +299,21 @@ Coordination::Error ZooKeeper::tryGetChildrenWatch(const std::string & path, Str Coordination::Error ZooKeeper::createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created) { - Coordination::Error code = Coordination::Error::ZOK; - Poco::Event event; + auto future_result = tryAsyncCreateNoThrow(path, data, mode); - auto callback = [&](const Coordination::CreateResponse & response) + if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { - SCOPE_EXIT(event.set()); - code = response.error; + impl->finalize(); + return Coordination::Error::ZOPERATIONTIMEOUT; + } + else + { + auto response = future_result.get(); + Coordination::Error code = response.error; if (code == Coordination::Error::ZOK) path_created = response.path_created; - }; - - impl->create(path, data, mode & 1, mode & 2, {}, callback); /// TODO better mode - event.wait(); - return code; + return code; + } } std::string ZooKeeper::create(const std::string & path, const std::string & data, int32_t mode) @@ -366,19 +368,19 @@ void ZooKeeper::createAncestors(const std::string & path) Coordination::Error ZooKeeper::removeImpl(const std::string & path, int32_t version) { - Coordination::Error code = Coordination::Error::ZOK; - Poco::Event event; + auto future_result = tryAsyncRemoveNoThrow(path, version); - auto callback = [&](const Coordination::RemoveResponse & response) + + if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { - SCOPE_EXIT(event.set()); - if (response.error != Coordination::Error::ZOK) - code = response.error; - }; - - impl->remove(path, version, callback); - event.wait(); - return code; + impl->finalize(); + return Coordination::Error::ZOPERATIONTIMEOUT; + } + else + { + auto response = future_result.get(); + return response.error; + } } void ZooKeeper::remove(const std::string & path, int32_t version) @@ -399,26 +401,22 @@ Coordination::Error ZooKeeper::tryRemove(const std::string & path, int32_t versi Coordination::Error ZooKeeper::existsImpl(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback) { - Coordination::Error code = Coordination::Error::ZOK; - Poco::Event event; + auto future_result = tryAsyncExistsNoThrow(path, watch_callback); - auto callback = [&](const Coordination::ExistsResponse & response) - { - SCOPE_EXIT(event.set()); - code = response.error; - if (code == Coordination::Error::ZOK && stat) - *stat = response.stat; - }; - - impl->exists(path, callback, watch_callback); - - if (!event.tryWait(operation_timeout_ms)) + if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { impl->finalize(); return Coordination::Error::ZOPERATIONTIMEOUT; } + else + { + auto response = future_result.get(); + Coordination::Error code = response.error; + if (code == Coordination::Error::ZOK && stat) + *stat = response.stat; - return code; + return code; + } } bool ZooKeeper::exists(const std::string & path, Coordination::Stat * stat, const EventPtr & watch) @@ -437,29 +435,25 @@ bool ZooKeeper::existsWatch(const std::string & path, Coordination::Stat * stat, Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback) { - Coordination::Error code = Coordination::Error::ZOK; - Poco::Event event; + auto future_result = tryAsyncGetNoThrow(path, watch_callback); - auto callback = [&](const Coordination::GetResponse & response) + if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { - SCOPE_EXIT(event.set()); - code = response.error; + impl->finalize(); + return Coordination::Error::ZOPERATIONTIMEOUT; + } + else + { + auto response = future_result.get(); + Coordination::Error code = response.error; if (code == Coordination::Error::ZOK) { res = response.data; if (stat) *stat = response.stat; } - }; - - impl->get(path, callback, watch_callback); - if (!event.tryWait(operation_timeout_ms)) - { - impl->finalize(); - return Coordination::Error::ZOPERATIONTIMEOUT; + return code; } - - return code; } @@ -514,24 +508,22 @@ bool ZooKeeper::tryGetWatch( Coordination::Error ZooKeeper::setImpl(const std::string & path, const std::string & data, int32_t version, Coordination::Stat * stat) { - Coordination::Error code = Coordination::Error::ZOK; - Poco::Event event; + auto future_result = tryAsyncSetNoThrow(path, data, version); - auto callback = [&](const Coordination::SetResponse & response) - { - SCOPE_EXIT(event.set()); - code = response.error; - if (code == Coordination::Error::ZOK && stat) - *stat = response.stat; - }; - - impl->set(path, data, version, callback); - if (!event.tryWait(operation_timeout_ms)) + if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { impl->finalize(); return Coordination::Error::ZOPERATIONTIMEOUT; } - return code; + else + { + auto response = future_result.get(); + Coordination::Error code = response.error; + if (code == Coordination::Error::ZOK && stat) + *stat = response.stat; + + return code; + } } void ZooKeeper::set(const std::string & path, const std::string & data, int32_t version, Coordination::Stat * stat) @@ -568,23 +560,20 @@ Coordination::Error ZooKeeper::multiImpl(const Coordination::Requests & requests if (requests.empty()) return Coordination::Error::ZOK; - Coordination::Error code = Coordination::Error::ZOK; - Poco::Event event; + auto future_result = tryAsyncMultiNoThrow(requests); - auto callback = [&](const Coordination::MultiResponse & response) - { - SCOPE_EXIT(event.set()); - code = response.error; - responses = response.responses; - }; - - impl->multi(requests, callback); - if (!event.tryWait(operation_timeout_ms)) + if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { impl->finalize(); return Coordination::Error::ZOPERATIONTIMEOUT; } - return code; + else + { + auto response = future_result.get(); + Coordination::Error code = response.error; + responses = response.responses; + return code; + } } Coordination::Responses ZooKeeper::multi(const Coordination::Requests & requests) @@ -769,8 +758,21 @@ std::future ZooKeeper::asyncCreate(const std::stri return future; } +std::future ZooKeeper::tryAsyncCreateNoThrow(const std::string & path, const std::string & data, int32_t mode) +{ + auto promise = std::make_shared>(); + auto future = promise->get_future(); -std::future ZooKeeper::asyncGet(const std::string & path) + auto callback = [promise, path](const Coordination::CreateResponse & response) mutable + { + promise->set_value(response); + }; + + impl->create(path, data, mode & 1, mode & 2, {}, std::move(callback)); + return future; +} + +std::future ZooKeeper::asyncGet(const std::string & path, Coordination::WatchCallback watch_callback) { auto promise = std::make_shared>(); auto future = promise->get_future(); @@ -783,7 +785,21 @@ std::future ZooKeeper::asyncGet(const std::string & p promise->set_value(response); }; - impl->get(path, std::move(callback), {}); + impl->get(path, std::move(callback), watch_callback); + return future; +} + +std::future ZooKeeper::tryAsyncGetNoThrow(const std::string & path, Coordination::WatchCallback watch_callback) +{ + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise, path](const Coordination::GetResponse & response) mutable + { + promise->set_value(response); + }; + + impl->get(path, std::move(callback), watch_callback); return future; } @@ -805,7 +821,7 @@ std::future ZooKeeper::asyncTryGet(const std::string return future; } -std::future ZooKeeper::asyncExists(const std::string & path) +std::future ZooKeeper::asyncExists(const std::string & path, Coordination::WatchCallback watch_callback) { auto promise = std::make_shared>(); auto future = promise->get_future(); @@ -818,7 +834,21 @@ std::future ZooKeeper::asyncExists(const std::stri promise->set_value(response); }; - impl->exists(path, std::move(callback), {}); + impl->exists(path, std::move(callback), watch_callback); + return future; +} + +std::future ZooKeeper::tryAsyncExistsNoThrow(const std::string & path, Coordination::WatchCallback watch_callback) +{ + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise, path](const Coordination::ExistsResponse & response) mutable + { + promise->set_value(response); + }; + + impl->exists(path, std::move(callback), watch_callback); return future; } @@ -839,7 +869,22 @@ std::future ZooKeeper::asyncSet(const std::string & p return future; } -std::future ZooKeeper::asyncGetChildren(const std::string & path) + +std::future ZooKeeper::tryAsyncSetNoThrow(const std::string & path, const std::string & data, int32_t version) +{ + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise, path](const Coordination::SetResponse & response) mutable + { + promise->set_value(response); + }; + + impl->set(path, data, version, std::move(callback)); + return future; +} + +std::future ZooKeeper::asyncGetChildren(const std::string & path, Coordination::WatchCallback watch_callback) { auto promise = std::make_shared>(); auto future = promise->get_future(); @@ -852,7 +897,21 @@ std::future ZooKeeper::asyncGetChildren(const std::s promise->set_value(response); }; - impl->list(path, std::move(callback), {}); + impl->list(path, std::move(callback), watch_callback); + return future; +} + +std::future ZooKeeper::tryAsyncGetChildrenNoThrow(const std::string & path, Coordination::WatchCallback watch_callback) +{ + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise, path](const Coordination::ListResponse & response) mutable + { + promise->set_value(response); + }; + + impl->list(path, std::move(callback), watch_callback); return future; } @@ -895,7 +954,21 @@ std::future ZooKeeper::asyncTryRemove(const std::s return future; } -std::future ZooKeeper::tryAsyncMulti(const Coordination::Requests & ops) +std::future ZooKeeper::tryAsyncRemoveNoThrow(const std::string & path, int32_t version) +{ + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise, path](const Coordination::RemoveResponse & response) mutable + { + promise->set_value(response); + }; + + impl->remove(path, version, std::move(callback)); + return future; +} + +std::future ZooKeeper::tryAsyncMultiNoThrow(const Coordination::Requests & ops) { auto promise = std::make_shared>(); auto future = promise->get_future(); diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 4a65ff070f7..e855a450b96 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -39,9 +39,6 @@ constexpr size_t MULTI_BATCH_SIZE = 100; /// watch notification. /// Callback-based watch interface is also provided. /// -/// Read-only methods retry retry_num times if recoverable errors like OperationTimeout -/// or ConnectionLoss are encountered. -/// /// Modifying methods do not retry, because it leads to problems of the double-delete type. /// /// Methods with names not starting at try- raise KeeperException on any error. @@ -220,39 +217,55 @@ public: /// auto result1 = future1.get(); /// auto result2 = future2.get(); /// - /// Future should not be destroyed before the result is gotten. + /// NoThrow versions never throw any exception on future.get(), even on SessionExpired error. using FutureCreate = std::future; FutureCreate asyncCreate(const std::string & path, const std::string & data, int32_t mode); + /// Like the previous one but don't throw any exceptions on future.get() + FutureCreate tryAsyncCreateNoThrow(const std::string & path, const std::string & data, int32_t mode); using FutureGet = std::future; - FutureGet asyncGet(const std::string & path); - - FutureGet asyncTryGet(const std::string & path); + FutureGet asyncGet(const std::string & path, Coordination::WatchCallback watch_callback = {}); + /// Like the previous one but don't throw any exceptions on future.get() + FutureGet tryAsyncGetNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); using FutureExists = std::future; - FutureExists asyncExists(const std::string & path); + FutureExists asyncExists(const std::string & path, Coordination::WatchCallback watch_callback = {}); + /// Like the previous one but don't throw any exceptions on future.get() + FutureExists tryAsyncExistsNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); using FutureGetChildren = std::future; - FutureGetChildren asyncGetChildren(const std::string & path); + FutureGetChildren asyncGetChildren(const std::string & path, Coordination::WatchCallback watch_callback = {}); + /// Like the previous one but don't throw any exceptions on future.get() + FutureGetChildren tryAsyncGetChildrenNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); using FutureSet = std::future; FutureSet asyncSet(const std::string & path, const std::string & data, int32_t version = -1); + /// Like the previous one but don't throw any exceptions on future.get() + FutureSet tryAsyncSetNoThrow(const std::string & path, const std::string & data, int32_t version = -1); using FutureRemove = std::future; FutureRemove asyncRemove(const std::string & path, int32_t version = -1); + /// Like the previous one but don't throw any exceptions on future.get() + FutureRemove tryAsyncRemoveNoThrow(const std::string & path, int32_t version = -1); + using FutureMulti = std::future; + FutureMulti asyncMulti(const Coordination::Requests & ops); + /// Like the previous one but don't throw any exceptions on future.get() + FutureMulti tryAsyncMultiNoThrow(const Coordination::Requests & ops); + + /// Very specific methods introduced without following general style. Implements + /// some custom throw/no throw logic on future.get(). + /// /// Doesn't throw in the following cases: /// * The node doesn't exist /// * The versions do not match /// * The node has children FutureRemove asyncTryRemove(const std::string & path, int32_t version = -1); - using FutureMulti = std::future; - FutureMulti asyncMulti(const Coordination::Requests & ops); - - /// Like the previous one but don't throw any exceptions on future.get() - FutureMulti tryAsyncMulti(const Coordination::Requests & ops); + /// Doesn't throw in the following cases: + /// * The node doesn't exist + FutureGet asyncTryGet(const std::string & path); void finalize(); @@ -262,7 +275,7 @@ private: void init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_); - /// The following methods don't throw exceptions but return error codes. + /// The following methods don't any throw exceptions but return error codes. Coordination::Error createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created); Coordination::Error removeImpl(const std::string & path, int32_t version); Coordination::Error getImpl( diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index d3496d99cef..9e994b3f7f4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -291,7 +291,7 @@ void ReplicatedMergeTreeCleanupThread::markLostReplicas(const std::unordered_map std::vector futures; for (size_t i = 0; i < candidate_lost_replicas.size(); ++i) - futures.emplace_back(zookeeper->tryAsyncMulti(requests[i])); + futures.emplace_back(zookeeper->tryAsyncMultiNoThrow(requests[i])); for (size_t i = 0; i < candidate_lost_replicas.size(); ++i) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 319e04f2424..91246cf5ac9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6144,7 +6144,7 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const St { Coordination::Requests ops; removePartFromZooKeeper(part_names[i], ops, exists_resp.stat.numChildren > 0); - remove_futures.emplace_back(zookeeper->tryAsyncMulti(ops)); + remove_futures.emplace_back(zookeeper->tryAsyncMultiNoThrow(ops)); } } @@ -6205,7 +6205,7 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper( { Coordination::Requests ops; removePartFromZooKeeper(part_names[i], ops, exists_resp.stat.numChildren > 0); - remove_futures.emplace_back(zookeeper->tryAsyncMulti(ops)); + remove_futures.emplace_back(zookeeper->tryAsyncMultiNoThrow(ops)); } else { From ad357ed97096b002881af3c1fdb0a4f3488881dd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 May 2021 16:04:26 +0300 Subject: [PATCH 120/154] Fix 00953_zookeeper_suetin_deduplication_bug --- .../0_stateless/00953_zookeeper_suetin_deduplication_bug.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh index 71ca29bfd96..ca3f4efe743 100755 --- a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh +++ b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh @@ -6,6 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +CLICKHOUSE_TEST_ZOOKEEPER_PREFIX="${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}" + $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS elog;" From ccebba13e15dfb067c6ca52dd42eeb8dcf268360 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 May 2021 16:11:32 +0300 Subject: [PATCH 121/154] Comment --- src/Common/ZooKeeper/IKeeper.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index d69ab34ea43..30d816aad15 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -428,6 +428,9 @@ public: /// /// After the method is executed successfully, you must wait for callbacks /// (don't destroy callback data before it will be called). + /// TODO: The above line is the description of an error-prone interface. It's better + /// to replace callbacks with std::future results, so the caller shouldn't think about + /// lifetime of the callback data. /// /// All callbacks are executed sequentially (the execution of callbacks is serialized). /// From 0035997e02ade74a11f6ee97ed2f011bae475283 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 31 May 2021 16:31:03 +0300 Subject: [PATCH 122/154] fix race on enqueue query --- src/Databases/DatabaseReplicated.cpp | 20 +++------- src/Databases/DatabaseReplicated.h | 2 +- src/Databases/DatabaseReplicatedWorker.cpp | 45 ++++++++++++++++++++-- src/Databases/DatabaseReplicatedWorker.h | 3 ++ 4 files changed, 51 insertions(+), 19 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index d36fe45f748..2630c896a7a 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -273,19 +273,11 @@ bool DatabaseReplicated::createDatabaseNodesInZooKeeper(const zkutil::ZooKeeperP __builtin_unreachable(); } -void DatabaseReplicated::createEmptyLogEntry(Coordination::Requests & ops, const ZooKeeperPtr & current_zookeeper) +void DatabaseReplicated::createEmptyLogEntry(const ZooKeeperPtr & current_zookeeper) { /// On replica creation add empty entry to log. Can be used to trigger some actions on other replicas (e.g. update cluster info). DDLLogEntry entry{}; - - String query_path_prefix = zookeeper_path + "/log/query-"; - String counter_prefix = zookeeper_path + "/counter/cnt-"; - String counter_path = current_zookeeper->create(counter_prefix, "", zkutil::CreateMode::EphemeralSequential); - String query_path = query_path_prefix + counter_path.substr(counter_prefix.size()); - - ops.emplace_back(zkutil::makeCreateRequest(query_path, entry.toString(), zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(query_path + "/committed", getFullReplicaName(), zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeRemoveRequest(counter_path, -1)); + DatabaseReplicatedDDLWorker::enqueueQueryImpl(current_zookeeper, entry, this, true); } void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPtr & current_zookeeper) @@ -296,8 +288,8 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent)); - createEmptyLogEntry(ops, current_zookeeper); current_zookeeper->multi(ops); + createEmptyLogEntry(current_zookeeper); } void DatabaseReplicated::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach) @@ -659,10 +651,8 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node void DatabaseReplicated::drop(ContextPtr context_) { auto current_zookeeper = getZooKeeper(); - Coordination::Requests ops; - ops.emplace_back(zkutil::makeSetRequest(replica_path, DROPPED_MARK, -1)); - createEmptyLogEntry(ops, current_zookeeper); - current_zookeeper->multi(ops); + current_zookeeper->set(replica_path, DROPPED_MARK, -1); + createEmptyLogEntry(current_zookeeper); DatabaseAtomic::drop(context_); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index b930d27c19b..e6c3fb00eb6 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -78,7 +78,7 @@ private: ClusterPtr getClusterImpl() const; void setCluster(ClusterPtr && new_cluster); - void createEmptyLogEntry(Coordination::Requests & ops, const ZooKeeperPtr & current_zookeeper); + void createEmptyLogEntry(const ZooKeeperPtr & current_zookeeper); String zookeeper_path; String shard_name; diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 9ae4d026bf0..548f7f6f882 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -69,25 +70,63 @@ void DatabaseReplicatedDDLWorker::initializeReplication() String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry) { auto zookeeper = getAndSetZooKeeper(); - const String query_path_prefix = queue_dir + "/query-"; + return enqueueQueryImpl(zookeeper, entry, database); +} + +String DatabaseReplicatedDDLWorker::enqueueQueryImpl(const ZooKeeperPtr & zookeeper, DDLLogEntry & entry, + DatabaseReplicated * const database, bool committed) +{ + const String query_path_prefix = database->zookeeper_path + "/log/query-"; /// We cannot create sequential node and it's ephemeral child in a single transaction, so allocate sequential number another way String counter_prefix = database->zookeeper_path + "/counter/cnt-"; - String counter_path = zookeeper->create(counter_prefix, "", zkutil::CreateMode::EphemeralSequential); + String counter_lock_path = database->zookeeper_path + "/counter_lock"; + + String counter_path; + size_t iters = 1000; + while (--iters) + { + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(counter_lock_path, database->getFullReplicaName(), zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeCreateRequest(counter_prefix, "", zkutil::CreateMode::EphemeralSequential)); + Coordination::Responses res; + + Coordination::Error code = zookeeper->tryMulti(ops, res); + if (code == Coordination::Error::ZOK) + { + counter_path = dynamic_cast(*res.back()).path_created; + break; + } + else if (code != Coordination::Error::ZNODEEXISTS) + zkutil::KeeperMultiException::check(code, ops, res); + } + + if (iters == 0) + throw Exception(ErrorCodes::UNFINISHED, + "Cannot enqueue query, because some replica are trying to enqueue another query. " + "It may happen on high queries rate or, in rare cases, after connection loss. Client should retry."); + String node_path = query_path_prefix + counter_path.substr(counter_prefix.size()); + /// Now create task in queue Coordination::Requests ops; /// Query is not committed yet, but we have to write it into log to avoid reordering ops.emplace_back(zkutil::makeCreateRequest(node_path, entry.toString(), zkutil::CreateMode::Persistent)); /// '/try' will be replaced with '/committed' or will be removed due to expired session or other error - ops.emplace_back(zkutil::makeCreateRequest(node_path + "/try", database->getFullReplicaName(), zkutil::CreateMode::Ephemeral)); + if (committed) + ops.emplace_back(zkutil::makeCreateRequest(node_path + "/committed", database->getFullReplicaName(), zkutil::CreateMode::Persistent)); + else + ops.emplace_back(zkutil::makeCreateRequest(node_path + "/try", database->getFullReplicaName(), zkutil::CreateMode::Ephemeral)); /// We don't need it anymore ops.emplace_back(zkutil::makeRemoveRequest(counter_path, -1)); + /// Unlock counters + ops.emplace_back(zkutil::makeRemoveRequest(counter_lock_path, -1)); /// Create status dirs ops.emplace_back(zkutil::makeCreateRequest(node_path + "/active", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(node_path + "/finished", "", zkutil::CreateMode::Persistent)); zookeeper->multi(ops); + return node_path; } diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 16ad100b81a..4020906f9b2 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -29,6 +29,9 @@ public: void shutdown() override; + static String enqueueQueryImpl(const ZooKeeperPtr & zookeeper, DDLLogEntry & entry, + DatabaseReplicated * const database, bool committed = false); + private: bool initializeMainThread() override; void initializeReplication(); From 572dcbcbe26790ec3fae5e19c75c12936153b63b Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 31 May 2021 16:38:33 +0300 Subject: [PATCH 123/154] Update StorageMaterializedView.cpp --- src/Storages/StorageMaterializedView.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 67bd6b21c3f..f0ac0aebb4e 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -431,7 +431,12 @@ Strings StorageMaterializedView::getDataPaths() const ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type) { - return has_inner_table ? getTargetTable()->getActionLock(type) : ActionLock{}; + if (has_inner_table) + { + if (auto target_table = tryGetTargetTable()) + return target_table->getActionLock(type); + } + return ActionLock{}; } void registerStorageMaterializedView(StorageFactory & factory) From 639b4f7388ff10f064d4401c31d0d112dec11397 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 31 May 2021 17:06:03 +0300 Subject: [PATCH 124/154] Update 40_bug-report.md --- .github/ISSUE_TEMPLATE/40_bug-report.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/.github/ISSUE_TEMPLATE/40_bug-report.md b/.github/ISSUE_TEMPLATE/40_bug-report.md index 97137366189..c3edd8f933e 100644 --- a/.github/ISSUE_TEMPLATE/40_bug-report.md +++ b/.github/ISSUE_TEMPLATE/40_bug-report.md @@ -15,6 +15,17 @@ A clear and concise description of what works not as it is supposed to. **Does it reproduce on recent release?** [The list of releases](https://github.com/ClickHouse/ClickHouse/blob/master/utils/list-versions/version_date.tsv) +**Enable crash reporting** + +If possible, change "enabled" to true in "send_crash_reports" section in `config.xml`: + +``` + + + + false +``` + **How to reproduce** * Which ClickHouse server version to use * Which interface to use, if matters From 3cb26bca96e200c9d0864ec4e77fe7d8c12cf33a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 31 May 2021 17:06:28 +0300 Subject: [PATCH 125/154] Update 40_bug-report.md --- .github/ISSUE_TEMPLATE/40_bug-report.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/ISSUE_TEMPLATE/40_bug-report.md b/.github/ISSUE_TEMPLATE/40_bug-report.md index c3edd8f933e..5c8611d47e6 100644 --- a/.github/ISSUE_TEMPLATE/40_bug-report.md +++ b/.github/ISSUE_TEMPLATE/40_bug-report.md @@ -10,9 +10,11 @@ assignees: '' You have to provide the following information whenever possible. **Describe the bug** + A clear and concise description of what works not as it is supposed to. **Does it reproduce on recent release?** + [The list of releases](https://github.com/ClickHouse/ClickHouse/blob/master/utils/list-versions/version_date.tsv) **Enable crash reporting** @@ -27,6 +29,7 @@ If possible, change "enabled" to true in "send_crash_reports" section in `config ``` **How to reproduce** + * Which ClickHouse server version to use * Which interface to use, if matters * Non-default settings, if any @@ -35,10 +38,13 @@ If possible, change "enabled" to true in "send_crash_reports" section in `config * Queries to run that lead to unexpected result **Expected behavior** + A clear and concise description of what you expected to happen. **Error message and/or stacktrace** + If applicable, add screenshots to help explain your problem. **Additional context** + Add any other context about the problem here. From f3c81f4c9bd0574f621af7a3efd11871044b09a4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 31 May 2021 17:30:02 +0300 Subject: [PATCH 126/154] disable test under msan --- base/daemon/BaseDaemon.cpp | 8 ++++---- tests/integration/test_send_crash_reports/test.py | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 857a758bc71..01e700ebba3 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -351,14 +351,14 @@ private: } #endif - /// Send crash report to developers (if configured) - if (sig != SanitizerTrap) - SentryWriter::onFault(sig, error_message, stack_trace); - /// Write crash to system.crash_log table if available. if (collectCrashLog) collectCrashLog(sig, thread_num, query_id, stack_trace); + /// Send crash report to developers (if configured) + if (sig != SanitizerTrap) + SentryWriter::onFault(sig, error_message, stack_trace); + /// When everything is done, we will try to send these error messages to client. if (thread_ptr) thread_ptr->onFatalError(); diff --git a/tests/integration/test_send_crash_reports/test.py b/tests/integration/test_send_crash_reports/test.py index 3f88f719fe4..ab52879c7c3 100644 --- a/tests/integration/test_send_crash_reports/test.py +++ b/tests/integration/test_send_crash_reports/test.py @@ -24,7 +24,7 @@ def started_node(): def test_send_segfault(started_node): - if started_node.is_built_with_thread_sanitizer(): + if started_node.is_built_with_thread_sanitizer() or started_node.is_built_with_memory_sanitizer(): pytest.skip("doesn't fit in timeouts for stacktrace generation") started_node.copy_file_to_container(os.path.join(SCRIPT_DIR, "fake_sentry_server.py"), "/fake_sentry_server.py") From 481b37cd6897885e987682b6ee69241d51da0c9f Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Mon, 31 May 2021 16:32:43 +0200 Subject: [PATCH 127/154] fix more copy & paste --- .../aggregate-functions/reference/quantiletdigestweighted.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md index a726938d07f..70f30f3a480 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md @@ -18,10 +18,10 @@ When using multiple `quantile*` functions with different levels in a query, the **Syntax** ``` sql -quantileTDigest(level)(expr, weight) +quantileTDigestWeighted(level)(expr, weight) ``` -Alias: `medianTDigest`. +Alias: `medianTDigestWeighted`. **Arguments** From afc1fe7f3d7970f70700f45d637bf1dc5e1c9201 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 May 2021 17:49:02 +0300 Subject: [PATCH 128/154] Make ContextPtr const by default. --- programs/benchmark/Benchmark.cpp | 2 +- programs/client/Client.cpp | 2 +- programs/copier/ClusterCopier.h | 6 +-- programs/keeper/Keeper.h | 4 +- programs/local/LocalServer.cpp | 4 +- programs/local/LocalServer.h | 6 +-- programs/obfuscator/Obfuscator.cpp | 2 +- programs/server/Server.h | 4 +- src/Common/CurrentThread.h | 2 +- src/Core/ExternalTable.h | 4 +- src/Core/MySQL/Authentication.cpp | 4 +- src/Core/MySQL/Authentication.h | 6 +-- src/Core/PostgreSQLProtocol.h | 10 ++-- .../PushingToViewsBlockOutputStream.h | 4 +- src/Databases/DatabaseAtomic.cpp | 2 +- src/Databases/DatabaseAtomic.h | 2 +- src/Databases/DatabaseLazy.cpp | 2 +- src/Databases/DatabaseLazy.h | 2 +- src/Databases/DatabaseOnDisk.cpp | 2 +- src/Databases/DatabaseOnDisk.h | 2 +- src/Databases/DatabaseOrdinary.cpp | 4 +- src/Databases/DatabaseOrdinary.h | 2 +- src/Databases/DatabaseReplicated.cpp | 2 +- src/Databases/DatabaseReplicated.h | 2 +- src/Databases/IDatabase.h | 2 +- .../MySQL/DatabaseConnectionMySQL.cpp | 2 +- src/Databases/MySQL/DatabaseConnectionMySQL.h | 2 +- .../MySQL/DatabaseMaterializeMySQL.cpp | 2 +- .../MySQL/DatabaseMaterializeMySQL.h | 2 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 6 +-- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 2 +- src/Dictionaries/ClickHouseDictionarySource.h | 2 +- src/Dictionaries/DictionarySourceHelpers.cpp | 2 +- src/Dictionaries/DictionarySourceHelpers.h | 2 +- .../ClusterProxy/executeQuery.cpp | 2 +- src/Interpreters/ClusterProxy/executeQuery.h | 2 +- src/Interpreters/Context.cpp | 47 +++++++++-------- src/Interpreters/Context.h | 52 +++++++++---------- src/Interpreters/Context_fwd.h | 7 ++- src/Interpreters/DDLTask.cpp | 4 +- src/Interpreters/DDLTask.h | 4 +- src/Interpreters/DDLWorker.h | 2 +- src/Interpreters/DatabaseCatalog.cpp | 6 +-- src/Interpreters/DatabaseCatalog.h | 6 +-- .../IInterpreterUnionOrSelectQuery.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 +- src/Interpreters/InterpreterCreateQuery.h | 4 +- .../InterpreterCreateQuotaQuery.h | 4 +- src/Interpreters/InterpreterCreateRoleQuery.h | 4 +- .../InterpreterCreateRowPolicyQuery.h | 4 +- .../InterpreterCreateSettingsProfileQuery.h | 4 +- src/Interpreters/InterpreterCreateUserQuery.h | 4 +- .../InterpreterDropAccessEntityQuery.h | 4 +- src/Interpreters/InterpreterDropQuery.cpp | 2 +- src/Interpreters/InterpreterDropQuery.h | 4 +- .../InterpreterExternalDDLQuery.cpp | 4 +- .../InterpreterExternalDDLQuery.h | 4 +- src/Interpreters/InterpreterFactory.cpp | 2 +- src/Interpreters/InterpreterFactory.h | 2 +- src/Interpreters/InterpreterGrantQuery.h | 4 +- src/Interpreters/InterpreterKillQueryQuery.h | 4 +- src/Interpreters/InterpreterSetQuery.h | 4 +- src/Interpreters/InterpreterSetRoleQuery.h | 4 +- .../InterpreterShowAccessEntitiesQuery.cpp | 4 +- .../InterpreterShowAccessEntitiesQuery.h | 4 +- .../InterpreterShowPrivilegesQuery.cpp | 2 +- .../InterpreterShowPrivilegesQuery.h | 4 +- .../InterpreterShowProcesslistQuery.h | 6 +-- .../InterpreterShowTablesQuery.cpp | 4 +- src/Interpreters/InterpreterShowTablesQuery.h | 4 +- src/Interpreters/InterpreterSystemQuery.cpp | 8 +-- src/Interpreters/InterpreterSystemQuery.h | 8 +-- src/Interpreters/MutationsInterpreter.cpp | 2 +- src/Interpreters/MutationsInterpreter.h | 2 +- .../MySQL/InterpretersMySQLDDLQuery.h | 6 +-- src/Interpreters/ThreadStatusExt.cpp | 2 +- src/Interpreters/executeQuery.cpp | 10 ++-- src/Interpreters/executeQuery.h | 6 +-- src/Interpreters/loadMetadata.cpp | 8 +-- src/Interpreters/loadMetadata.h | 4 +- src/Processors/Pipe.h | 4 +- src/Processors/QueryPipeline.h | 2 +- .../QueryPlan/ReadFromPreparedSource.cpp | 2 +- .../QueryPlan/ReadFromPreparedSource.h | 4 +- src/Server/GRPCServer.cpp | 6 +-- src/Server/HTTPHandler.cpp | 14 ++--- src/Server/HTTPHandler.h | 22 ++++---- src/Server/IServer.h | 2 +- src/Server/MySQLHandler.h | 2 +- src/Server/PostgreSQLHandler.h | 2 +- src/Server/TCPHandler.h | 4 +- .../DistributedBlockOutputStream.h | 2 +- src/Storages/Kafka/KafkaBlockOutputStream.h | 4 +- src/Storages/LiveView/StorageLiveView.h | 2 +- .../LiveView/TemporaryLiveViewCleaner.cpp | 6 +-- .../LiveView/TemporaryLiveViewCleaner.h | 6 +-- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.h | 4 +- src/Storages/RabbitMQ/StorageRabbitMQ.h | 2 +- src/Storages/StorageFactory.cpp | 8 +-- src/Storages/StorageFactory.h | 12 ++--- src/Storages/StorageMaterializedView.cpp | 4 +- src/Storages/StorageMaterializedView.h | 2 +- src/Storages/StorageMerge.cpp | 2 +- src/Storages/StorageMerge.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- 110 files changed, 265 insertions(+), 259 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 1d2b579db3a..82c93eef9ff 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -159,7 +159,7 @@ private: bool print_stacktrace; const Settings & settings; SharedContextHolder shared_context; - ContextPtr global_context; + ContextMutablePtr global_context; QueryProcessingStage::Enum query_processing_stage; /// Don't execute new queries after timelimit or SIGINT or exception diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 098f7e689c5..23adeca03a8 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -181,7 +181,7 @@ private: bool has_vertical_output_suffix = false; /// Is \G present at the end of the query string? SharedContextHolder shared_context = Context::createShared(); - ContextPtr context = Context::createGlobal(shared_context.get()); + ContextMutablePtr context = Context::createGlobal(shared_context.get()); /// Buffer that reads from stdin in batch mode. ReadBufferFromFileDescriptor std_in{STDIN_FILENO}; diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index e875ca7df2e..085fa2ece06 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -12,14 +12,14 @@ namespace DB { -class ClusterCopier : WithContext +class ClusterCopier : WithMutableContext { public: ClusterCopier(const String & task_path_, const String & host_id_, const String & proxy_database_name_, - ContextPtr context_) - : WithContext(context_), + ContextMutablePtr context_) + : WithMutableContext(context_), task_zookeeper_path(task_path_), host_id(host_id_), working_database_name(proxy_database_name_), diff --git a/programs/keeper/Keeper.h b/programs/keeper/Keeper.h index e80fe10b61c..f5b97dacf7d 100644 --- a/programs/keeper/Keeper.h +++ b/programs/keeper/Keeper.h @@ -32,7 +32,7 @@ public: return BaseDaemon::logger(); } - ContextPtr context() const override + ContextMutablePtr context() const override { return global_context; } @@ -58,7 +58,7 @@ protected: std::string getDefaultConfigFileName() const override; private: - ContextPtr global_context; + ContextMutablePtr global_context; Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const; diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 043cc596e2b..d8648a4a470 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -100,7 +100,7 @@ void LocalServer::initialize(Poco::Util::Application & self) } } -void LocalServer::applyCmdSettings(ContextPtr context) +void LocalServer::applyCmdSettings(ContextMutablePtr context) { context->applySettingsChanges(cmd_settings.changes()); } @@ -645,7 +645,7 @@ void LocalServer::init(int argc, char ** argv) argsToConfig(arguments, config(), 100); } -void LocalServer::applyCmdOptions(ContextPtr context) +void LocalServer::applyCmdOptions(ContextMutablePtr context) { context->setDefaultFormat(config().getString("output-format", config().getString("format", "TSV"))); applyCmdSettings(context); diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index c5e9d5716dd..cf8886d9652 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -36,8 +36,8 @@ private: std::string getInitialCreateTableQuery(); void tryInitPath(); - void applyCmdOptions(ContextPtr context); - void applyCmdSettings(ContextPtr context); + void applyCmdOptions(ContextMutablePtr context); + void applyCmdSettings(ContextMutablePtr context); void processQueries(); void setupUsers(); void cleanup(); @@ -45,7 +45,7 @@ private: protected: SharedContextHolder shared_context; - ContextPtr global_context; + ContextMutablePtr global_context; /// Settings specified via command line args Settings cmd_settings; diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index fb6817fbf80..f68b255158c 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1133,7 +1133,7 @@ try } SharedContextHolder shared_context = Context::createShared(); - ContextPtr context = Context::createGlobal(shared_context.get()); + auto context = Context::createGlobal(shared_context.get()); context->makeGlobalContext(); ReadBufferFromFileDescriptor file_in(STDIN_FILENO); diff --git a/programs/server/Server.h b/programs/server/Server.h index c698108767c..45e5fccd51d 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -40,7 +40,7 @@ public: return BaseDaemon::logger(); } - ContextPtr context() const override + ContextMutablePtr context() const override { return global_context; } @@ -64,7 +64,7 @@ protected: std::string getDefaultCorePath() const override; private: - ContextPtr global_context; + ContextMutablePtr global_context; Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const; using CreateServerFunc = std::function; diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index 069f9cf2af7..c6ac7e9933a 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -87,7 +87,7 @@ public: /// Initializes query with current thread as master thread in constructor, and detaches it in destructor struct QueryScope { - explicit QueryScope(ContextPtr query_context); + explicit QueryScope(ContextMutablePtr query_context); ~QueryScope(); void logPeakMemoryUsage(); diff --git a/src/Core/ExternalTable.h b/src/Core/ExternalTable.h index fcefa3d7fe3..f682bf9b27f 100644 --- a/src/Core/ExternalTable.h +++ b/src/Core/ExternalTable.h @@ -80,10 +80,10 @@ public: /// Parsing of external table used when sending tables via http /// The `handlePart` function will be called for each table passed, /// so it's also necessary to call `clean` at the end of the `handlePart`. -class ExternalTablesHandler : public HTMLForm::PartHandler, BaseExternalTable, WithContext +class ExternalTablesHandler : public HTMLForm::PartHandler, BaseExternalTable, WithMutableContext { public: - ExternalTablesHandler(ContextPtr context_, const Poco::Net::NameValueCollection & params_) : WithContext(context_), params(params_) {} + ExternalTablesHandler(ContextMutablePtr context_, const Poco::Net::NameValueCollection & params_) : WithMutableContext(context_), params(params_) {} void handlePart(const Poco::Net::MessageHeader & header, ReadBuffer & stream) override; diff --git a/src/Core/MySQL/Authentication.cpp b/src/Core/MySQL/Authentication.cpp index b8dff9972c7..0258f9f94af 100644 --- a/src/Core/MySQL/Authentication.cpp +++ b/src/Core/MySQL/Authentication.cpp @@ -73,7 +73,7 @@ Native41::Native41(const String & password, const String & auth_plugin_data) } void Native41::authenticate( - const String & user_name, std::optional auth_response, ContextPtr context, + const String & user_name, std::optional auth_response, ContextMutablePtr context, std::shared_ptr packet_endpoint, bool, const Poco::Net::SocketAddress & address) { if (!auth_response) @@ -136,7 +136,7 @@ Sha256Password::Sha256Password(RSA & public_key_, RSA & private_key_, Poco::Logg } void Sha256Password::authenticate( - const String & user_name, std::optional auth_response, ContextPtr context, + const String & user_name, std::optional auth_response, ContextMutablePtr context, std::shared_ptr packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) { if (!auth_response) diff --git a/src/Core/MySQL/Authentication.h b/src/Core/MySQL/Authentication.h index 5358e2da737..acbda2bdb58 100644 --- a/src/Core/MySQL/Authentication.h +++ b/src/Core/MySQL/Authentication.h @@ -32,7 +32,7 @@ public: virtual String getAuthPluginData() = 0; virtual void authenticate( - const String & user_name, std::optional auth_response, ContextPtr context, + const String & user_name, std::optional auth_response, ContextMutablePtr context, std::shared_ptr packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) = 0; }; @@ -49,7 +49,7 @@ public: String getAuthPluginData() override { return scramble; } void authenticate( - const String & user_name, std::optional auth_response, ContextPtr context, + const String & user_name, std::optional auth_response, ContextMutablePtr context, std::shared_ptr packet_endpoint, bool /* is_secure_connection */, const Poco::Net::SocketAddress & address) override; private: @@ -69,7 +69,7 @@ public: String getAuthPluginData() override { return scramble; } void authenticate( - const String & user_name, std::optional auth_response, ContextPtr context, + const String & user_name, std::optional auth_response, ContextMutablePtr context, std::shared_ptr packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) override; private: diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index 9e1afcd187c..7c5800f5a8f 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -802,7 +802,7 @@ protected: static void setPassword( const String & user_name, const String & password, - ContextPtr context, + ContextMutablePtr context, Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) { @@ -821,7 +821,7 @@ protected: public: virtual void authenticate( const String & user_name, - ContextPtr context, + ContextMutablePtr context, Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) = 0; @@ -835,7 +835,7 @@ class NoPasswordAuth : public AuthenticationMethod public: void authenticate( const String & user_name, - ContextPtr context, + ContextMutablePtr context, Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) override { @@ -853,7 +853,7 @@ class CleartextPasswordAuth : public AuthenticationMethod public: void authenticate( const String & user_name, - ContextPtr context, + ContextMutablePtr context, Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) override { @@ -896,7 +896,7 @@ public: void authenticate( const String & user_name, - ContextPtr context, + ContextMutablePtr context, Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) { diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 2ae941efc2e..552a0a3452a 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -54,8 +54,8 @@ private: }; std::vector views; - ContextPtr select_context; - ContextPtr insert_context; + ContextMutablePtr select_context; + ContextMutablePtr insert_context; void process(const Block & block, ViewInfo & view); }; diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index e939f76f372..88ead6c9b4c 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -416,7 +416,7 @@ UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const return UUIDHelpers::Nil; } -void DatabaseAtomic::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach) +void DatabaseAtomic::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool force_attach) { /// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken if (has_force_restore_data_flag) diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index a6acd10b656..c5dedf00d23 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -47,7 +47,7 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override; - void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override; + void loadStoredObjects(ContextMutablePtr context, bool has_force_restore_data_flag, bool force_attach) override; /// Atomic database cannot be detached if there is detached table which still in use void assertCanBeDetached(bool cleanup) override; diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 80179de104a..f77133f68e7 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -35,7 +35,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, void DatabaseLazy::loadStoredObjects( - ContextPtr local_context, + ContextMutablePtr local_context, bool /* has_force_restore_data_flag */, bool /*force_attach*/) { diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 99a71b342fa..949869f4509 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -27,7 +27,7 @@ public: bool canContainDistributedTables() const override { return false; } void loadStoredObjects( - ContextPtr context, + ContextMutablePtr context, bool has_force_restore_data_flag, bool force_attach) override; void createTable( diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 3db5314bbd3..af0ec35f5a2 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -45,7 +45,7 @@ std::pair createTableFromAST( ASTCreateQuery ast_create_query, const String & database_name, const String & table_data_path_relative, - ContextPtr context, + ContextMutablePtr context, bool has_force_restore_data_flag) { ast_create_query.attach = true; diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index 677465e306e..73b69dd43a5 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -16,7 +16,7 @@ std::pair createTableFromAST( ASTCreateQuery ast_create_query, const String & database_name, const String & table_data_path_relative, - ContextPtr context, + ContextMutablePtr context, bool has_force_restore_data_flag); /** Get the string with the table definition based on the CREATE query. diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 4a150972573..eccee68da40 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -35,7 +35,7 @@ static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; namespace { void tryAttachTable( - ContextPtr context, + ContextMutablePtr context, const ASTCreateQuery & query, DatabaseOrdinary & database, const String & database_name, @@ -84,7 +84,7 @@ DatabaseOrdinary::DatabaseOrdinary( { } -void DatabaseOrdinary::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool /*force_attach*/) +void DatabaseOrdinary::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool /*force_attach*/) { /** Tables load faster if they are loaded in sorted (by name) order. * Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order, diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index 8a32bb10991..4d68890cc2b 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -20,7 +20,7 @@ public: String getEngineName() const override { return "Ordinary"; } - void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override; + void loadStoredObjects(ContextMutablePtr context, bool has_force_restore_data_flag, bool force_attach) override; void alterTable( ContextPtr context, diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index d36fe45f748..8d27595338a 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -300,7 +300,7 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt current_zookeeper->multi(ops); } -void DatabaseReplicated::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach) +void DatabaseReplicated::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool force_attach) { tryConnectToZooKeeperAndInitDatabase(force_attach); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index b930d27c19b..b1fe0e58b0e 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -57,7 +57,7 @@ public: void drop(ContextPtr /*context*/) override; - void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override; + void loadStoredObjects(ContextMutablePtr context, bool has_force_restore_data_flag, bool force_attach) override; void shutdown() override; friend struct DatabaseReplicatedTask; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index ea07e51bffd..ba5fa974d5c 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -122,7 +122,7 @@ public: /// Load a set of existing tables. /// You can call only once, right after the object is created. - virtual void loadStoredObjects(ContextPtr /*context*/, bool /*has_force_restore_data_flag*/, bool /*force_attach*/ = false) {} + virtual void loadStoredObjects(ContextMutablePtr /*context*/, bool /*has_force_restore_data_flag*/, bool /*force_attach*/ = false) {} /// Check the existence of the table. virtual bool isTableExist(const String & name, ContextPtr context) const = 0; diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 9b71fe537ec..50b1c77c87e 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -399,7 +399,7 @@ String DatabaseConnectionMySQL::getMetadataPath() const return metadata_path; } -void DatabaseConnectionMySQL::loadStoredObjects(ContextPtr, bool, bool /*force_attach*/) +void DatabaseConnectionMySQL::loadStoredObjects(ContextMutablePtr, bool, bool /*force_attach*/) { std::lock_guard lock{mutex}; diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.h b/src/Databases/MySQL/DatabaseConnectionMySQL.h index c1936e51999..32741185cfa 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.h +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.h @@ -74,7 +74,7 @@ public: void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; - void loadStoredObjects(ContextPtr, bool, bool force_attach) override; + void loadStoredObjects(ContextMutablePtr, bool, bool force_attach) override; StoragePtr detachTable(const String & table_name) override; diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 62a66b22c93..6e82f88c574 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -92,7 +92,7 @@ void DatabaseMaterializeMySQL::setException(const std::exception_ptr & exc } template -void DatabaseMaterializeMySQL::loadStoredObjects(ContextPtr context_, bool has_force_restore_data_flag, bool force_attach) +void DatabaseMaterializeMySQL::loadStoredObjects(ContextMutablePtr context_, bool has_force_restore_data_flag, bool force_attach) { Base::loadStoredObjects(context_, has_force_restore_data_flag, force_attach); if (!force_attach) diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index cd9fe640239..74a3c06e6f0 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -43,7 +43,7 @@ protected: public: String getEngineName() const override { return "MaterializeMySQL"; } - void loadStoredObjects(ContextPtr context_, bool has_force_restore_data_flag, bool force_attach) override; + void loadStoredObjects(ContextMutablePtr context_, bool has_force_restore_data_flag, bool force_attach) override; void createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) override; diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index a7ebcb0d3f2..e5783be4624 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -41,7 +41,7 @@ namespace ErrorCodes static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; -static ContextPtr createQueryContext(ContextPtr context) +static ContextMutablePtr createQueryContext(ContextPtr context) { Settings new_query_settings = context->getSettings(); new_query_settings.insert_allow_materialized_columns = true; @@ -59,7 +59,7 @@ static ContextPtr createQueryContext(ContextPtr context) return query_context; } -static BlockIO tryToExecuteQuery(const String & query_to_execute, ContextPtr query_context, const String & database, const String & comment) +static BlockIO tryToExecuteQuery(const String & query_to_execute, ContextMutablePtr query_context, const String & database, const String & comment) { try { @@ -281,7 +281,7 @@ static inline void cleanOutdatedTables(const String & database_name, ContextPtr } static inline BlockOutputStreamPtr -getTableOutput(const String & database_name, const String & table_name, ContextPtr query_context, bool insert_materialized = false) +getTableOutput(const String & database_name, const String & table_name, ContextMutablePtr query_context, bool insert_materialized = false) { const StoragePtr & storage = DatabaseCatalog::instance().getTable(StorageID(database_name, table_name), query_context); diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 55813e519e4..1920c047b8e 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -280,7 +280,7 @@ void DatabasePostgreSQL::drop(ContextPtr /*context*/) } -void DatabasePostgreSQL::loadStoredObjects(ContextPtr /* context */, bool, bool /*force_attach*/) +void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, bool, bool /*force_attach*/) { { std::lock_guard lock{mutex}; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index c5054883f2c..3c3a67ec9d3 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -47,7 +47,7 @@ public: bool empty() const override; - void loadStoredObjects(ContextPtr, bool, bool force_attach) override; + void loadStoredObjects(ContextMutablePtr, bool, bool force_attach) override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override; diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index a16b2f18680..cf954a9620d 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -79,7 +79,7 @@ private: mutable std::string invalidate_query_response; ExternalQueryBuilder query_builder; Block sample_block; - ContextPtr context; + ContextMutablePtr context; ConnectionPoolWithFailoverPtr pool; const std::string load_all_query; Poco::Logger * log = &Poco::Logger::get("ClickHouseDictionarySource"); diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 6ba4f63be66..723a0c338fd 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -67,7 +67,7 @@ Block blockForKeys( return block; } -ContextPtr copyContextAndApplySettings( +ContextMutablePtr copyContextAndApplySettings( const std::string & config_prefix, ContextConstPtr context, const Poco::Util::AbstractConfiguration & config) diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index aa0219449e3..2f39f951f8d 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -37,7 +37,7 @@ Block blockForKeys( const std::vector & requested_rows); /// Used for applying settings to copied context in some register[...]Source functions -ContextPtr copyContextAndApplySettings( +ContextMutablePtr copyContextAndApplySettings( const std::string & config_prefix, ContextConstPtr context, const Poco::Util::AbstractConfiguration & config); diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 5284756a4ff..2af65e0fd87 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes namespace ClusterProxy { -ContextPtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log) +ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log) { Settings new_settings = settings; new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time); diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 46525335803..c9efedfc422 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -31,7 +31,7 @@ class IStreamFactory; /// - optimize_skip_unused_shards_nesting /// /// @return new Context with adjusted settings -ContextPtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log = nullptr); +ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log = nullptr); /// Execute a distributed query, creating a vector of BlockInputStreams, from which the result can be read. /// `stream_factory` object encapsulates the logic of creating streams for a different type of query diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7d9c2c27ee2..b6b10906c6b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -140,7 +140,7 @@ public: /// Find existing session or create a new. std::shared_ptr acquireSession( const String & session_id, - ContextPtr context, + ContextMutablePtr context, std::chrono::steady_clock::duration timeout, bool throw_if_not_found) { @@ -513,7 +513,7 @@ SharedContextHolder::SharedContextHolder(std::unique_ptr shar void SharedContextHolder::reset() { shared.reset(); } -ContextPtr Context::createGlobal(ContextSharedPart * shared) +ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) { auto res = std::shared_ptr(new Context); res->shared = shared; @@ -530,19 +530,19 @@ SharedContextHolder Context::createShared() return SharedContextHolder(std::make_unique()); } -ContextPtr Context::createCopy(const ContextConstPtr & other) +ContextMutablePtr Context::createCopy(const ContextPtr & other) { return std::shared_ptr(new Context(*other)); } -ContextPtr Context::createCopy(const ContextWeakConstPtr & other) +ContextMutablePtr Context::createCopy(const ContextWeakConstPtr & other) { auto ptr = other.lock(); if (!ptr) throw Exception("Can't copy an expired context", ErrorCodes::LOGICAL_ERROR); return createCopy(ptr); } -ContextPtr Context::createCopy(const ContextPtr & other) +ContextMutablePtr Context::createCopy(const ContextMutablePtr & other) { return createCopy(std::const_pointer_cast(other)); } @@ -1079,7 +1079,7 @@ void Context::addViewSource(const StoragePtr & storage) } -StoragePtr Context::getViewSource() +StoragePtr Context::getViewSource() const { return view_source; } @@ -1313,7 +1313,7 @@ void Context::setMacros(std::unique_ptr && macros) shared->macros.set(std::move(macros)); } -ContextPtr Context::getQueryContext() const +ContextMutablePtr Context::getQueryContext() const { auto ptr = query_context.lock(); if (!ptr) throw Exception("There is no query or query context has expired", ErrorCodes::THERE_IS_NO_QUERY); @@ -1326,21 +1326,21 @@ bool Context::isInternalSubquery() const return ptr && ptr.get() != this; } -ContextPtr Context::getSessionContext() const +ContextMutablePtr Context::getSessionContext() const { auto ptr = session_context.lock(); if (!ptr) throw Exception("There is no session or session context has expired", ErrorCodes::THERE_IS_NO_SESSION); return ptr; } -ContextPtr Context::getGlobalContext() const +ContextMutablePtr Context::getGlobalContext() const { auto ptr = global_context.lock(); if (!ptr) throw Exception("There is no global context or global context has expired", ErrorCodes::LOGICAL_ERROR); return ptr; } -ContextPtr Context::getBufferContext() const +ContextMutablePtr Context::getBufferContext() const { if (!buffer_context) throw Exception("There is no buffer context", ErrorCodes::LOGICAL_ERROR); return buffer_context; @@ -1871,7 +1871,7 @@ std::shared_ptr Context::tryGetCluster(const std::string & cluster_name } -void Context::reloadClusterConfig() +void Context::reloadClusterConfig() const { while (true) { @@ -1958,7 +1958,7 @@ bool Context::hasTraceCollector() const } -std::shared_ptr Context::getQueryLog() +std::shared_ptr Context::getQueryLog() const { auto lock = getLock(); @@ -1969,7 +1969,7 @@ std::shared_ptr Context::getQueryLog() } -std::shared_ptr Context::getQueryThreadLog() +std::shared_ptr Context::getQueryThreadLog() const { auto lock = getLock(); @@ -1980,7 +1980,7 @@ std::shared_ptr Context::getQueryThreadLog() } -std::shared_ptr Context::getPartLog(const String & part_database) +std::shared_ptr Context::getPartLog(const String & part_database) const { auto lock = getLock(); @@ -1998,7 +1998,7 @@ std::shared_ptr Context::getPartLog(const String & part_database) } -std::shared_ptr Context::getTraceLog() +std::shared_ptr Context::getTraceLog() const { auto lock = getLock(); @@ -2009,7 +2009,7 @@ std::shared_ptr Context::getTraceLog() } -std::shared_ptr Context::getTextLog() +std::shared_ptr Context::getTextLog() const { auto lock = getLock(); @@ -2020,7 +2020,7 @@ std::shared_ptr Context::getTextLog() } -std::shared_ptr Context::getMetricLog() +std::shared_ptr Context::getMetricLog() const { auto lock = getLock(); @@ -2042,7 +2042,7 @@ std::shared_ptr Context::getAsynchronousMetricLog() const } -std::shared_ptr Context::getOpenTelemetrySpanLog() +std::shared_ptr Context::getOpenTelemetrySpanLog() const { auto lock = getLock(); @@ -2640,11 +2640,14 @@ ZooKeeperMetadataTransactionPtr Context::getZooKeeperMetadataTransaction() const return metadata_transaction; } -PartUUIDsPtr Context::getPartUUIDs() +PartUUIDsPtr Context::getPartUUIDs() const { auto lock = getLock(); if (!part_uuids) - part_uuids = std::make_shared(); + /// For context itself, only this initialization is not const. + /// We could have done in constructor. + /// TODO: probably, remove this from Context. + const_cast(part_uuids) = std::make_shared(); return part_uuids; } @@ -2663,11 +2666,11 @@ void Context::setReadTaskCallback(ReadTaskCallback && callback) next_task_callback = callback; } -PartUUIDsPtr Context::getIgnoredPartUUIDs() +PartUUIDsPtr Context::getIgnoredPartUUIDs() const { auto lock = getLock(); if (!ignored_part_uuids) - ignored_part_uuids = std::make_shared(); + const_cast(ignored_part_uuids) = std::make_shared(); return ignored_part_uuids; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 5089d2c0288..97cb1b980f1 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -252,12 +252,12 @@ private: StoragePtr view_source; /// Temporary StorageValues used to generate alias columns for materialized views Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id. - ContextWeakPtr query_context; - ContextWeakPtr session_context; /// Session context or nullptr. Could be equal to this. - ContextWeakPtr global_context; /// Global context. Could be equal to this. + ContextWeakMutablePtr query_context; + ContextWeakMutablePtr session_context; /// Session context or nullptr. Could be equal to this. + ContextWeakMutablePtr global_context; /// Global context. Could be equal to this. /// XXX: move this stuff to shared part instead. - ContextPtr buffer_context; /// Buffer context. Could be equal to this. + ContextMutablePtr buffer_context; /// Buffer context. Could be equal to this. public: // Top-level OpenTelemetry trace context for the query. Makes sense only for a query context. @@ -293,10 +293,10 @@ private: public: /// Create initial Context with ContextShared and etc. - static ContextPtr createGlobal(ContextSharedPart * shared); - static ContextPtr createCopy(const ContextWeakConstPtr & other); - static ContextPtr createCopy(const ContextConstPtr & other); - static ContextPtr createCopy(const ContextPtr & other); + static ContextMutablePtr createGlobal(ContextSharedPart * shared); + static ContextMutablePtr createCopy(const ContextWeakConstPtr & other); + static ContextMutablePtr createCopy(const ContextMutablePtr & other); + static ContextMutablePtr createCopy(const ContextPtr & other); static SharedContextHolder createShared(); void copyFrom(const ContextPtr & other); @@ -459,7 +459,7 @@ public: StoragePtr executeTableFunction(const ASTPtr & table_expression); void addViewSource(const StoragePtr & storage); - StoragePtr getViewSource(); + StoragePtr getViewSource() const; String getCurrentDatabase() const; String getCurrentQueryId() const { return client_info.current_query_id; } @@ -555,14 +555,14 @@ public: /// For methods below you may need to acquire the context lock by yourself. - ContextPtr getQueryContext() const; + ContextMutablePtr getQueryContext() const; bool hasQueryContext() const { return !query_context.expired(); } bool isInternalSubquery() const; - ContextPtr getSessionContext() const; + ContextMutablePtr getSessionContext() const; bool hasSessionContext() const { return !session_context.expired(); } - ContextPtr getGlobalContext() const; + ContextMutablePtr getGlobalContext() const; bool hasGlobalContext() const { return !global_context.expired(); } bool isGlobalContext() const { @@ -570,10 +570,10 @@ public: return ptr && ptr.get() == this; } - ContextPtr getBufferContext() const; + ContextMutablePtr getBufferContext() const; - void setQueryContext(ContextPtr context_) { query_context = context_; } - void setSessionContext(ContextPtr context_) { session_context = context_; } + void setQueryContext(ContextMutablePtr context_) { query_context = context_; } + void setSessionContext(ContextMutablePtr context_) { session_context = context_; } void makeQueryContext() { query_context = shared_from_this(); } void makeSessionContext() { session_context = shared_from_this(); } @@ -668,7 +668,7 @@ public: void setClustersConfig(const ConfigurationPtr & config, const String & config_name = "remote_servers"); /// Sets custom cluster, but doesn't update configuration void setCluster(const String & cluster_name, const std::shared_ptr & cluster); - void reloadClusterConfig(); + void reloadClusterConfig() const; Compiler & getCompiler(); @@ -681,17 +681,17 @@ public: bool hasTraceCollector() const; /// Nullptr if the query log is not ready for this moment. - std::shared_ptr getQueryLog(); - std::shared_ptr getQueryThreadLog(); - std::shared_ptr getTraceLog(); - std::shared_ptr getTextLog(); - std::shared_ptr getMetricLog(); + std::shared_ptr getQueryLog() const; + std::shared_ptr getQueryThreadLog() const; + std::shared_ptr getTraceLog() const; + std::shared_ptr getTextLog() const; + std::shared_ptr getMetricLog() const; std::shared_ptr getAsynchronousMetricLog() const; - std::shared_ptr getOpenTelemetrySpanLog(); + std::shared_ptr getOpenTelemetrySpanLog() const; /// Returns an object used to log operations with parts if it possible. /// Provide table name to make required checks. - std::shared_ptr getPartLog(const String & part_database); + std::shared_ptr getPartLog(const String & part_database) const; const MergeTreeSettings & getMergeTreeSettings() const; const MergeTreeSettings & getReplicatedMergeTreeSettings() const; @@ -778,8 +778,8 @@ public: MySQLWireContext mysql; - PartUUIDsPtr getPartUUIDs(); - PartUUIDsPtr getIgnoredPartUUIDs(); + PartUUIDsPtr getPartUUIDs() const; + PartUUIDsPtr getIgnoredPartUUIDs() const; ReadTaskCallback getReadTaskCallback() const; void setReadTaskCallback(ReadTaskCallback && callback); @@ -820,7 +820,7 @@ struct NamedSession { NamedSessionKey key; UInt64 close_cycle = 0; - ContextPtr context; + ContextMutablePtr context; std::chrono::steady_clock::duration timeout; NamedSessions & parent; diff --git a/src/Interpreters/Context_fwd.h b/src/Interpreters/Context_fwd.h index d7232717bb2..af18e5a7fdd 100644 --- a/src/Interpreters/Context_fwd.h +++ b/src/Interpreters/Context_fwd.h @@ -22,10 +22,12 @@ class Context; /// Most used types have shorter names /// TODO: in the first part of refactoring all the context pointers are non-const. -using ContextPtr = std::shared_ptr; +using ContextPtr = std::shared_ptr; using ContextConstPtr = std::shared_ptr; -using ContextWeakPtr = std::weak_ptr; +using ContextMutablePtr = std::shared_ptr; +using ContextWeakPtr = std::weak_ptr; using ContextWeakConstPtr = std::weak_ptr; +using ContextWeakMutablePtr = std::weak_ptr; template struct WithContextImpl @@ -50,5 +52,6 @@ protected: using WithContext = WithContextImpl<>; using WithConstContext = WithContextImpl; +using WithMutableContext = WithContextImpl; } diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 087e5504672..4fb44738d8d 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -145,7 +145,7 @@ void DDLTaskBase::parseQueryFromEntry(ContextPtr context) query = parseQuery(parser_query, begin, end, description, 0, context->getSettingsRef().max_parser_depth); } -ContextPtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/) +ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/) { auto query_context = Context::createCopy(from_context); query_context->makeQueryContext(); @@ -355,7 +355,7 @@ void DatabaseReplicatedTask::parseQueryFromEntry(ContextPtr context) } } -ContextPtr DatabaseReplicatedTask::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper) +ContextMutablePtr DatabaseReplicatedTask::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper) { auto query_context = DDLTaskBase::makeQueryContext(from_context, zookeeper); query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 5fafb42bfd2..2dd9857b78a 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -98,7 +98,7 @@ struct DDLTaskBase virtual String getShardID() const = 0; - virtual ContextPtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper); + virtual ContextMutablePtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper); inline String getActiveNodePath() const { return entry_path + "/active/" + host_id_str; } inline String getFinishedNodePath() const { return entry_path + "/finished/" + host_id_str; } @@ -136,7 +136,7 @@ struct DatabaseReplicatedTask : public DDLTaskBase String getShardID() const override; void parseQueryFromEntry(ContextPtr context) override; - ContextPtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper) override; + ContextMutablePtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper) override; DatabaseReplicated * database; }; diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index e92644f0f99..45218226fee 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -110,7 +110,7 @@ protected: void runMainThread(); void runCleanupThread(); - ContextPtr context; + ContextMutablePtr context; Poco::Logger * log; std::string host_fqdn; /// current host domain name diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 09166103155..7dfc7b93c96 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -528,13 +528,13 @@ void DatabaseCatalog::updateUUIDMapping(const UUID & uuid, DatabasePtr database, std::unique_ptr DatabaseCatalog::database_catalog; -DatabaseCatalog::DatabaseCatalog(ContextPtr global_context_) - : WithContext(global_context_), log(&Poco::Logger::get("DatabaseCatalog")) +DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_) + : WithMutableContext(global_context_), log(&Poco::Logger::get("DatabaseCatalog")) { TemporaryLiveViewCleaner::init(global_context_); } -DatabaseCatalog & DatabaseCatalog::init(ContextPtr global_context_) +DatabaseCatalog & DatabaseCatalog::init(ContextMutablePtr global_context_) { if (database_catalog) { diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 783c511fa08..e20259fb973 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -117,13 +117,13 @@ using TemporaryTablesMapping = std::map database_catalog; - explicit DatabaseCatalog(ContextPtr global_context_); + explicit DatabaseCatalog(ContextMutablePtr global_context_); void assertDatabaseExistsUnlocked(const String & database_name) const; void assertDatabaseDoesntExistUnlocked(const String & database_name) const; diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index 72723d68161..0b07f27e14a 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -32,7 +32,7 @@ public: protected: ASTPtr query_ptr; - ContextPtr context; + ContextMutablePtr context; Block result_header; SelectQueryOptions options; size_t max_streams = 1; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6cb823aae3c..229ffa2af03 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -89,8 +89,8 @@ namespace ErrorCodes namespace fs = std::filesystem; -InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, ContextPtr context_) - : WithContext(context_), query_ptr(query_ptr_) +InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) { } diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 990b87f02be..45f0bbd7cf8 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -22,10 +22,10 @@ using DatabasePtr = std::shared_ptr; /** Allows to create new table or database, * or create an object for existing table or database. */ -class InterpreterCreateQuery : public IInterpreter, WithContext +class InterpreterCreateQuery : public IInterpreter, WithMutableContext { public: - InterpreterCreateQuery(const ASTPtr & query_ptr_, ContextPtr context_); + InterpreterCreateQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); BlockIO execute() override; diff --git a/src/Interpreters/InterpreterCreateQuotaQuery.h b/src/Interpreters/InterpreterCreateQuotaQuery.h index d8edd24b2d9..7fb60cde1f6 100644 --- a/src/Interpreters/InterpreterCreateQuotaQuery.h +++ b/src/Interpreters/InterpreterCreateQuotaQuery.h @@ -10,10 +10,10 @@ namespace DB class ASTCreateQuotaQuery; struct Quota; -class InterpreterCreateQuotaQuery : public IInterpreter, WithContext +class InterpreterCreateQuotaQuery : public IInterpreter, WithMutableContext { public: - InterpreterCreateQuotaQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterCreateQuotaQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; diff --git a/src/Interpreters/InterpreterCreateRoleQuery.h b/src/Interpreters/InterpreterCreateRoleQuery.h index 18b3f946837..ee8ff8ff7fb 100644 --- a/src/Interpreters/InterpreterCreateRoleQuery.h +++ b/src/Interpreters/InterpreterCreateRoleQuery.h @@ -10,10 +10,10 @@ namespace DB class ASTCreateRoleQuery; struct Role; -class InterpreterCreateRoleQuery : public IInterpreter, WithContext +class InterpreterCreateRoleQuery : public IInterpreter, WithMutableContext { public: - InterpreterCreateRoleQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterCreateRoleQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; diff --git a/src/Interpreters/InterpreterCreateRowPolicyQuery.h b/src/Interpreters/InterpreterCreateRowPolicyQuery.h index 10167bac669..8adfe6b0855 100644 --- a/src/Interpreters/InterpreterCreateRowPolicyQuery.h +++ b/src/Interpreters/InterpreterCreateRowPolicyQuery.h @@ -10,10 +10,10 @@ namespace DB class ASTCreateRowPolicyQuery; struct RowPolicy; -class InterpreterCreateRowPolicyQuery : public IInterpreter, WithContext +class InterpreterCreateRowPolicyQuery : public IInterpreter, WithMutableContext { public: - InterpreterCreateRowPolicyQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterCreateRowPolicyQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; diff --git a/src/Interpreters/InterpreterCreateSettingsProfileQuery.h b/src/Interpreters/InterpreterCreateSettingsProfileQuery.h index 9ef1f0354a9..aa9264fbf22 100644 --- a/src/Interpreters/InterpreterCreateSettingsProfileQuery.h +++ b/src/Interpreters/InterpreterCreateSettingsProfileQuery.h @@ -10,10 +10,10 @@ namespace DB class ASTCreateSettingsProfileQuery; struct SettingsProfile; -class InterpreterCreateSettingsProfileQuery : public IInterpreter, WithContext +class InterpreterCreateSettingsProfileQuery : public IInterpreter, WithMutableContext { public: - InterpreterCreateSettingsProfileQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterCreateSettingsProfileQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; diff --git a/src/Interpreters/InterpreterCreateUserQuery.h b/src/Interpreters/InterpreterCreateUserQuery.h index e9f4e82e767..7d357924d35 100644 --- a/src/Interpreters/InterpreterCreateUserQuery.h +++ b/src/Interpreters/InterpreterCreateUserQuery.h @@ -10,10 +10,10 @@ namespace DB class ASTCreateUserQuery; struct User; -class InterpreterCreateUserQuery : public IInterpreter, WithContext +class InterpreterCreateUserQuery : public IInterpreter, WithMutableContext { public: - InterpreterCreateUserQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterCreateUserQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; diff --git a/src/Interpreters/InterpreterDropAccessEntityQuery.h b/src/Interpreters/InterpreterDropAccessEntityQuery.h index 7f0f6348610..0ee478e904e 100644 --- a/src/Interpreters/InterpreterDropAccessEntityQuery.h +++ b/src/Interpreters/InterpreterDropAccessEntityQuery.h @@ -9,10 +9,10 @@ namespace DB class AccessRightsElements; -class InterpreterDropAccessEntityQuery : public IInterpreter, WithContext +class InterpreterDropAccessEntityQuery : public IInterpreter, WithMutableContext { public: - InterpreterDropAccessEntityQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterDropAccessEntityQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 24cff90caea..332a52a1d1f 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -42,7 +42,7 @@ static DatabasePtr tryGetDatabase(const String & database_name, bool if_exists) } -InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) +InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) { } diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index 8e8d577deec..16a08e95a3b 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -16,10 +16,10 @@ class AccessRightsElements; * or remove information about table (just forget) from server (DETACH), * or just clear all data in table (TRUNCATE). */ -class InterpreterDropQuery : public IInterpreter, WithContext +class InterpreterDropQuery : public IInterpreter, WithMutableContext { public: - InterpreterDropQuery(const ASTPtr & query_ptr_, ContextPtr context_); + InterpreterDropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); /// Drop table or database. BlockIO execute() override; diff --git a/src/Interpreters/InterpreterExternalDDLQuery.cpp b/src/Interpreters/InterpreterExternalDDLQuery.cpp index 8f9f0cf9ddb..e91b95a988d 100644 --- a/src/Interpreters/InterpreterExternalDDLQuery.cpp +++ b/src/Interpreters/InterpreterExternalDDLQuery.cpp @@ -26,8 +26,8 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -InterpreterExternalDDLQuery::InterpreterExternalDDLQuery(const ASTPtr & query_, ContextPtr context_) - : WithContext(context_), query(query_) +InterpreterExternalDDLQuery::InterpreterExternalDDLQuery(const ASTPtr & query_, ContextMutablePtr context_) + : WithMutableContext(context_), query(query_) { } diff --git a/src/Interpreters/InterpreterExternalDDLQuery.h b/src/Interpreters/InterpreterExternalDDLQuery.h index 15a842a2611..d084e68d7c8 100644 --- a/src/Interpreters/InterpreterExternalDDLQuery.h +++ b/src/Interpreters/InterpreterExternalDDLQuery.h @@ -6,10 +6,10 @@ namespace DB { -class InterpreterExternalDDLQuery : public IInterpreter, WithContext +class InterpreterExternalDDLQuery : public IInterpreter, WithMutableContext { public: - InterpreterExternalDDLQuery(const ASTPtr & query_, ContextPtr context_); + InterpreterExternalDDLQuery(const ASTPtr & query_, ContextMutablePtr context_); BlockIO execute() override; diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 4af8b6ffa7d..79cda364c42 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -92,7 +92,7 @@ namespace ErrorCodes } -std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextPtr context, const SelectQueryOptions & options) +std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMutablePtr context, const SelectQueryOptions & options) { OpenTelemetrySpanHolder span("InterpreterFactory::get()"); diff --git a/src/Interpreters/InterpreterFactory.h b/src/Interpreters/InterpreterFactory.h index c122fe11b7d..774cbd1cb0f 100644 --- a/src/Interpreters/InterpreterFactory.h +++ b/src/Interpreters/InterpreterFactory.h @@ -16,7 +16,7 @@ class InterpreterFactory public: static std::unique_ptr get( ASTPtr & query, - ContextPtr context, + ContextMutablePtr context, const SelectQueryOptions & options = {}); }; diff --git a/src/Interpreters/InterpreterGrantQuery.h b/src/Interpreters/InterpreterGrantQuery.h index f5939ff3cb7..abaddcc599b 100644 --- a/src/Interpreters/InterpreterGrantQuery.h +++ b/src/Interpreters/InterpreterGrantQuery.h @@ -12,10 +12,10 @@ class ASTGrantQuery; struct User; struct Role; -class InterpreterGrantQuery : public IInterpreter, WithContext +class InterpreterGrantQuery : public IInterpreter, WithMutableContext { public: - InterpreterGrantQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterGrantQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; diff --git a/src/Interpreters/InterpreterKillQueryQuery.h b/src/Interpreters/InterpreterKillQueryQuery.h index 5ffd9a525a2..9284d777ca7 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.h +++ b/src/Interpreters/InterpreterKillQueryQuery.h @@ -10,10 +10,10 @@ namespace DB class AccessRightsElements; -class InterpreterKillQueryQuery final : public IInterpreter, WithContext +class InterpreterKillQueryQuery final : public IInterpreter, WithMutableContext { public: - InterpreterKillQueryQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) { } + InterpreterKillQueryQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) { } BlockIO execute() override; diff --git a/src/Interpreters/InterpreterSetQuery.h b/src/Interpreters/InterpreterSetQuery.h index 31519be6f29..9bd49708421 100644 --- a/src/Interpreters/InterpreterSetQuery.h +++ b/src/Interpreters/InterpreterSetQuery.h @@ -11,10 +11,10 @@ class ASTSetQuery; /** Change one or several settings for the session or just for the current context. */ -class InterpreterSetQuery : public IInterpreter, WithContext +class InterpreterSetQuery : public IInterpreter, WithMutableContext { public: - InterpreterSetQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterSetQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} /** Usual SET query. Set setting for the session. */ diff --git a/src/Interpreters/InterpreterSetRoleQuery.h b/src/Interpreters/InterpreterSetRoleQuery.h index 70ba3c381ab..0a489f45fcd 100644 --- a/src/Interpreters/InterpreterSetRoleQuery.h +++ b/src/Interpreters/InterpreterSetRoleQuery.h @@ -11,10 +11,10 @@ class ASTSetRoleQuery; struct RolesOrUsersSet; struct User; -class InterpreterSetRoleQuery : public IInterpreter, WithContext +class InterpreterSetRoleQuery : public IInterpreter, WithMutableContext { public: - InterpreterSetRoleQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterSetRoleQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; diff --git a/src/Interpreters/InterpreterShowAccessEntitiesQuery.cpp b/src/Interpreters/InterpreterShowAccessEntitiesQuery.cpp index c2c2305f976..41b986e43a2 100644 --- a/src/Interpreters/InterpreterShowAccessEntitiesQuery.cpp +++ b/src/Interpreters/InterpreterShowAccessEntitiesQuery.cpp @@ -17,8 +17,8 @@ namespace ErrorCodes using EntityType = IAccessEntity::Type; -InterpreterShowAccessEntitiesQuery::InterpreterShowAccessEntitiesQuery(const ASTPtr & query_ptr_, ContextPtr context_) - : WithContext(context_), query_ptr(query_ptr_) +InterpreterShowAccessEntitiesQuery::InterpreterShowAccessEntitiesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) { } diff --git a/src/Interpreters/InterpreterShowAccessEntitiesQuery.h b/src/Interpreters/InterpreterShowAccessEntitiesQuery.h index 7224f0d593b..35511a38d8f 100644 --- a/src/Interpreters/InterpreterShowAccessEntitiesQuery.h +++ b/src/Interpreters/InterpreterShowAccessEntitiesQuery.h @@ -7,10 +7,10 @@ namespace DB { -class InterpreterShowAccessEntitiesQuery : public IInterpreter, WithContext +class InterpreterShowAccessEntitiesQuery : public IInterpreter, WithMutableContext { public: - InterpreterShowAccessEntitiesQuery(const ASTPtr & query_ptr_, ContextPtr context_); + InterpreterShowAccessEntitiesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); BlockIO execute() override; diff --git a/src/Interpreters/InterpreterShowPrivilegesQuery.cpp b/src/Interpreters/InterpreterShowPrivilegesQuery.cpp index c566d31e2fc..201c1cfece8 100644 --- a/src/Interpreters/InterpreterShowPrivilegesQuery.cpp +++ b/src/Interpreters/InterpreterShowPrivilegesQuery.cpp @@ -4,7 +4,7 @@ namespace DB { -InterpreterShowPrivilegesQuery::InterpreterShowPrivilegesQuery(const ASTPtr & query_ptr_, ContextPtr context_) +InterpreterShowPrivilegesQuery::InterpreterShowPrivilegesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : query_ptr(query_ptr_), context(context_) { } diff --git a/src/Interpreters/InterpreterShowPrivilegesQuery.h b/src/Interpreters/InterpreterShowPrivilegesQuery.h index 75989263405..852d5173eb1 100644 --- a/src/Interpreters/InterpreterShowPrivilegesQuery.h +++ b/src/Interpreters/InterpreterShowPrivilegesQuery.h @@ -11,7 +11,7 @@ class Context; class InterpreterShowPrivilegesQuery : public IInterpreter { public: - InterpreterShowPrivilegesQuery(const ASTPtr & query_ptr_, ContextPtr context_); + InterpreterShowPrivilegesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); BlockIO execute() override; @@ -20,7 +20,7 @@ public: private: ASTPtr query_ptr; - ContextPtr context; + ContextMutablePtr context; }; } diff --git a/src/Interpreters/InterpreterShowProcesslistQuery.h b/src/Interpreters/InterpreterShowProcesslistQuery.h index 5eedb67595e..31454882a89 100644 --- a/src/Interpreters/InterpreterShowProcesslistQuery.h +++ b/src/Interpreters/InterpreterShowProcesslistQuery.h @@ -9,11 +9,11 @@ namespace DB /** Return list of currently executing queries. */ -class InterpreterShowProcesslistQuery : public IInterpreter, WithContext +class InterpreterShowProcesslistQuery : public IInterpreter, WithMutableContext { public: - InterpreterShowProcesslistQuery(const ASTPtr & query_ptr_, ContextPtr context_) - : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterShowProcesslistQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 901999f004f..609df1404ca 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -18,8 +18,8 @@ namespace ErrorCodes } -InterpreterShowTablesQuery::InterpreterShowTablesQuery(const ASTPtr & query_ptr_, ContextPtr context_) - : WithContext(context_), query_ptr(query_ptr_) +InterpreterShowTablesQuery::InterpreterShowTablesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) { } diff --git a/src/Interpreters/InterpreterShowTablesQuery.h b/src/Interpreters/InterpreterShowTablesQuery.h index b61be568e35..16fc9ef2cf4 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.h +++ b/src/Interpreters/InterpreterShowTablesQuery.h @@ -13,10 +13,10 @@ class Context; /** Return a list of tables or databases meets specified conditions. * Interprets a query through replacing it to SELECT query from system.tables or system.databases. */ -class InterpreterShowTablesQuery : public IInterpreter, WithContext +class InterpreterShowTablesQuery : public IInterpreter, WithMutableContext { public: - InterpreterShowTablesQuery(const ASTPtr & query_ptr_, ContextPtr context_); + InterpreterShowTablesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); BlockIO execute() override; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index e7babb9b83f..7dab12eb803 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -192,8 +192,8 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, } -InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextPtr context_) - : WithContext(context_), query_ptr(query_ptr_->clone()), log(&Poco::Logger::get("InterpreterSystemQuery")) +InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_->clone()), log(&Poco::Logger::get("InterpreterSystemQuery")) { } @@ -424,7 +424,7 @@ BlockIO InterpreterSystemQuery::execute() } -StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, ContextPtr system_context, bool need_ddl_guard) +StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, ContextMutablePtr system_context, bool need_ddl_guard) { getContext()->checkAccess(AccessType::SYSTEM_RESTART_REPLICA, replica); @@ -469,7 +469,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, return table; } -void InterpreterSystemQuery::restartReplicas(ContextPtr system_context) +void InterpreterSystemQuery::restartReplicas(ContextMutablePtr system_context) { std::vector replica_names; auto & catalog = DatabaseCatalog::instance(); diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 341611e0af1..297f7225a92 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -30,10 +30,10 @@ class ASTSystemQuery; * - start/stop actions for all existing tables. * Note that the actions for tables that will be created after this query will not be affected. */ -class InterpreterSystemQuery : public IInterpreter, WithContext +class InterpreterSystemQuery : public IInterpreter, WithMutableContext { public: - InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextPtr context_); + InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); BlockIO execute() override; @@ -45,9 +45,9 @@ private: /// Tries to get a replicated table and restart it /// Returns pointer to a newly created table if the restart was successful - StoragePtr tryRestartReplica(const StorageID & replica, ContextPtr context, bool need_ddl_guard = true); + StoragePtr tryRestartReplica(const StorageID & replica, ContextMutablePtr context, bool need_ddl_guard = true); - void restartReplicas(ContextPtr system_context); + void restartReplicas(ContextMutablePtr system_context); void syncReplica(ASTSystemQuery & query); void dropReplica(ASTSystemQuery & query); bool dropReplicaImpl(ASTSystemQuery & query, const StoragePtr & table); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 2332dada770..ca0b8257f6e 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -179,7 +179,7 @@ bool isStorageTouchedByMutations( const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, - ContextPtr context_copy) + ContextMutablePtr context_copy) { if (commands.empty()) return false; diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 7e2f910466b..0d91da5613c 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -23,7 +23,7 @@ bool isStorageTouchedByMutations( const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, - ContextPtr context_copy + ContextMutablePtr context_copy ); ASTPtr getPartitionAndPredicateExpressionForMutationCommand( diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h index 3202612ac94..1efbe8ee744 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h @@ -55,12 +55,12 @@ namespace MySQLInterpreter }; template -class InterpreterMySQLDDLQuery : public IInterpreter, WithContext +class InterpreterMySQLDDLQuery : public IInterpreter, WithMutableContext { public: InterpreterMySQLDDLQuery( - const ASTPtr & query_ptr_, ContextPtr context_, const String & mapped_to_database_, const String & mysql_database_) - : WithContext(context_), query_ptr(query_ptr_), mapped_to_database(mapped_to_database_), mysql_database(mysql_database_) + const ASTPtr & query_ptr_, ContextMutablePtr context_, const String & mapped_to_database_, const String & mysql_database_) + : WithMutableContext(context_), query_ptr(query_ptr_), mapped_to_database(mapped_to_database_), mysql_database(mysql_database_) { } diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index cff6fb9cbcd..8590b3c94f3 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -508,7 +508,7 @@ void CurrentThread::detachQueryIfNotDetached() } -CurrentThread::QueryScope::QueryScope(ContextPtr query_context) +CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context) { CurrentThread::initializeQuery(); CurrentThread::attachQueryContext(query_context); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 542ed7ca0f9..a78c810d5d4 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -335,7 +335,7 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr } } -static void setQuerySpecificSettings(ASTPtr & ast, ContextPtr context) +static void setQuerySpecificSettings(ASTPtr & ast, ContextMutablePtr context) { if (auto * ast_insert_into = dynamic_cast(ast.get())) { @@ -347,7 +347,7 @@ static void setQuerySpecificSettings(ASTPtr & ast, ContextPtr context) static std::tuple executeQueryImpl( const char * begin, const char * end, - ContextPtr context, + ContextMutablePtr context, bool internal, QueryProcessingStage::Enum stage, bool has_query_tail, @@ -910,7 +910,7 @@ static std::tuple executeQueryImpl( BlockIO executeQuery( const String & query, - ContextPtr context, + ContextMutablePtr context, bool internal, QueryProcessingStage::Enum stage, bool may_have_embedded_data) @@ -935,7 +935,7 @@ BlockIO executeQuery( BlockIO executeQuery( const String & query, - ContextPtr context, + ContextMutablePtr context, bool internal, QueryProcessingStage::Enum stage, bool may_have_embedded_data, @@ -954,7 +954,7 @@ void executeQuery( ReadBuffer & istr, WriteBuffer & ostr, bool allow_into_outfile, - ContextPtr context, + ContextMutablePtr context, std::function set_result_details) { PODArray parse_buf; diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index bdb1f877ce3..6448b26a652 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -16,7 +16,7 @@ void executeQuery( ReadBuffer & istr, /// Where to read query from (and data for INSERT, if present). WriteBuffer & ostr, /// Where to write query output to. bool allow_into_outfile, /// If true and the query contains INTO OUTFILE section, redirect output to that file. - ContextPtr context, /// DB, tables, data types, storage engines, functions, aggregate functions... + ContextMutablePtr context, /// DB, tables, data types, storage engines, functions, aggregate functions... std::function set_result_details /// If a non-empty callback is passed, it will be called with the query id, the content-type, the format, and the timezone. ); @@ -37,7 +37,7 @@ void executeQuery( /// must be done separately. BlockIO executeQuery( const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out. - ContextPtr context, /// DB, tables, data types, storage engines, functions, aggregate functions... + ContextMutablePtr context, /// DB, tables, data types, storage engines, functions, aggregate functions... bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList. QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, /// To which stage the query must be executed. bool may_have_embedded_data = false /// If insert query may have embedded data @@ -46,7 +46,7 @@ BlockIO executeQuery( /// Old interface with allow_processors flag. For compatibility. BlockIO executeQuery( const String & query, - ContextPtr context, + ContextMutablePtr context, bool internal, QueryProcessingStage::Enum stage, bool may_have_embedded_data, diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 79076e57328..9f1934e2e2f 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -25,7 +25,7 @@ namespace DB static void executeCreateQuery( const String & query, - ContextPtr context, + ContextMutablePtr context, const String & database, const String & file_name, bool has_force_restore_data_flag) @@ -46,7 +46,7 @@ static void executeCreateQuery( static void loadDatabase( - ContextPtr context, + ContextMutablePtr context, const String & database, const String & database_path, bool force_restore_data) @@ -84,7 +84,7 @@ static void loadDatabase( } -void loadMetadata(ContextPtr context, const String & default_database_name) +void loadMetadata(ContextMutablePtr context, const String & default_database_name) { Poco::Logger * log = &Poco::Logger::get("loadMetadata"); @@ -168,7 +168,7 @@ void loadMetadata(ContextPtr context, const String & default_database_name) } -void loadMetadataSystem(ContextPtr context) +void loadMetadataSystem(ContextMutablePtr context) { String path = context->getPath() + "metadata/" + DatabaseCatalog::SYSTEM_DATABASE; String metadata_file = path + ".sql"; diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index 047def84bba..cf038a42855 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -8,9 +8,9 @@ namespace DB /// Load tables from system database. Only real tables like query_log, part_log. /// You should first load system database, then attach system tables that you need into it, then load other databases. -void loadMetadataSystem(ContextPtr context); +void loadMetadataSystem(ContextMutablePtr context); /// Load tables from databases and add them to context. Database 'system' is ignored. Use separate function to load system tables. -void loadMetadata(ContextPtr context, const String & default_database_name = {}); +void loadMetadata(ContextMutablePtr context, const String & default_database_name = {}); } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index f69c2562d7f..dc3be3289fc 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -110,7 +110,7 @@ public: /// Do not allow to change the table while the processors of pipe are alive. void addTableLock(TableLockHolder lock) { holder.table_locks.emplace_back(std::move(lock)); } /// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible. - void addInterpreterContext(std::shared_ptr context) { holder.interpreter_context.emplace_back(std::move(context)); } + void addInterpreterContext(std::shared_ptr context) { holder.interpreter_context.emplace_back(std::move(context)); } void addStorageHolder(StoragePtr storage) { holder.storage_holders.emplace_back(std::move(storage)); } void addQueryIdHolder(std::shared_ptr query_id_holder) { holder.query_id_holder = std::move(query_id_holder); } /// For queries with nested interpreters (i.e. StorageDistributed) @@ -129,7 +129,7 @@ private: /// Some processors may implicitly use Context or temporary Storage created by Interpreter. /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, /// because QueryPipeline is alive until query is finished. - std::vector> interpreter_context; + std::vector> interpreter_context; std::vector storage_holders; std::vector table_locks; std::vector> query_plans; diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 90c7e370880..1585f2532ff 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -119,7 +119,7 @@ public: const Block & getHeader() const { return pipe.getHeader(); } void addTableLock(TableLockHolder lock) { pipe.addTableLock(std::move(lock)); } - void addInterpreterContext(std::shared_ptr context) { pipe.addInterpreterContext(std::move(context)); } + void addInterpreterContext(std::shared_ptr context) { pipe.addInterpreterContext(std::move(context)); } void addStorageHolder(StoragePtr storage) { pipe.addStorageHolder(std::move(storage)); } void addQueryPlan(std::unique_ptr plan) { pipe.addQueryPlan(std::move(plan)); } void setLimits(const StreamLocalLimits & limits) { pipe.setLimits(limits); } diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index d7711abb3e1..0d1a0fdc619 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -4,7 +4,7 @@ namespace DB { -ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptr context_) +ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptr context_) : ISourceStep(DataStream{.header = pipe_.getHeader()}) , pipe(std::move(pipe_)) , context(std::move(context_)) diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.h b/src/Processors/QueryPlan/ReadFromPreparedSource.h index 69297c3c9ae..0d4e9653806 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -9,7 +9,7 @@ namespace DB class ReadFromPreparedSource : public ISourceStep { public: - explicit ReadFromPreparedSource(Pipe pipe_, std::shared_ptr context_ = nullptr); + explicit ReadFromPreparedSource(Pipe pipe_, std::shared_ptr context_ = nullptr); String getName() const override { return "ReadFromPreparedSource"; } @@ -17,7 +17,7 @@ public: private: Pipe pipe; - std::shared_ptr context; + std::shared_ptr context; }; class ReadFromStorageStep : public ReadFromPreparedSource diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 6f0f2d30123..2d6de423c5b 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -521,7 +521,7 @@ namespace Poco::Logger * log = nullptr; std::shared_ptr session; - ContextPtr query_context; + ContextMutablePtr query_context; std::optional query_scope; String query_text; ASTPtr ast; @@ -932,8 +932,8 @@ namespace String format = external_table.format(); if (format.empty()) format = "TabSeparated"; - ContextPtr external_table_context = query_context; - ContextPtr temp_context; + ContextMutablePtr external_table_context = query_context; + ContextMutablePtr temp_context; if (!external_table.settings().empty()) { temp_context = Context::createCopy(query_context); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 8aed5d20f74..39dc764b958 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -277,7 +277,7 @@ HTTPHandler::~HTTPHandler() bool HTTPHandler::authenticateUser( - ContextPtr context, + ContextMutablePtr context, HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response) @@ -441,7 +441,7 @@ bool HTTPHandler::authenticateUser( void HTTPHandler::processQuery( - ContextPtr context, + ContextMutablePtr context, HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response, @@ -937,7 +937,7 @@ DynamicQueryHandler::DynamicQueryHandler(IServer & server_, const std::string & { } -bool DynamicQueryHandler::customizeQueryParam(ContextPtr context, const std::string & key, const std::string & value) +bool DynamicQueryHandler::customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) { if (key == param_name) return true; /// do nothing @@ -953,7 +953,7 @@ bool DynamicQueryHandler::customizeQueryParam(ContextPtr context, const std::str return false; } -std::string DynamicQueryHandler::getQuery(HTTPServerRequest & request, HTMLForm & params, ContextPtr context) +std::string DynamicQueryHandler::getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) { if (likely(!startsWith(request.getContentType(), "multipart/form-data"))) { @@ -992,7 +992,7 @@ PredefinedQueryHandler::PredefinedQueryHandler( { } -bool PredefinedQueryHandler::customizeQueryParam(ContextPtr context, const std::string & key, const std::string & value) +bool PredefinedQueryHandler::customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) { if (receive_params.count(key)) { @@ -1003,7 +1003,7 @@ bool PredefinedQueryHandler::customizeQueryParam(ContextPtr context, const std:: return false; } -void PredefinedQueryHandler::customizeContext(HTTPServerRequest & request, ContextPtr context) +void PredefinedQueryHandler::customizeContext(HTTPServerRequest & request, ContextMutablePtr context) { /// If in the configuration file, the handler's header is regex and contains named capture group /// We will extract regex named capture groups as query parameters @@ -1039,7 +1039,7 @@ void PredefinedQueryHandler::customizeContext(HTTPServerRequest & request, Conte } } -std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLForm & params, ContextPtr context) +std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) { if (unlikely(startsWith(request.getContentType(), "multipart/form-data"))) { diff --git a/src/Server/HTTPHandler.h b/src/Server/HTTPHandler.h index 4715949cb87..2149a7ca55c 100644 --- a/src/Server/HTTPHandler.h +++ b/src/Server/HTTPHandler.h @@ -33,11 +33,11 @@ public: void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override; /// This method is called right before the query execution. - virtual void customizeContext(HTTPServerRequest & /* request */, ContextPtr /* context */) {} + virtual void customizeContext(HTTPServerRequest & /* request */, ContextMutablePtr /* context */) {} - virtual bool customizeQueryParam(ContextPtr context, const std::string & key, const std::string & value) = 0; + virtual bool customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) = 0; - virtual std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextPtr context) = 0; + virtual std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) = 0; private: struct Output @@ -73,7 +73,7 @@ private: // The request_context and the request_credentials instances may outlive a single request/response loop. // This happens only when the authentication mechanism requires more than a single request/response exchange (e.g., SPNEGO). - ContextPtr request_context; + ContextMutablePtr request_context; std::unique_ptr request_credentials; // Returns true when the user successfully authenticated, @@ -82,14 +82,14 @@ private: // the request_context and request_credentials instances are preserved. // Throws an exception if authentication failed. bool authenticateUser( - ContextPtr context, + ContextMutablePtr context, HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response); /// Also initializes 'used_output'. void processQuery( - ContextPtr context, + ContextMutablePtr context, HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response, @@ -113,9 +113,9 @@ private: public: explicit DynamicQueryHandler(IServer & server_, const std::string & param_name_ = "query"); - std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextPtr context) override; + std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) override; - bool customizeQueryParam(ContextPtr context, const std::string &key, const std::string &value) override; + bool customizeQueryParam(ContextMutablePtr context, const std::string &key, const std::string &value) override; }; class PredefinedQueryHandler : public HTTPHandler @@ -130,11 +130,11 @@ public: IServer & server_, const NameSet & receive_params_, const std::string & predefined_query_ , const CompiledRegexPtr & url_regex_, const std::unordered_map & header_name_with_regex_); - virtual void customizeContext(HTTPServerRequest & request, ContextPtr context) override; + virtual void customizeContext(HTTPServerRequest & request, ContextMutablePtr context) override; - std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextPtr context) override; + std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) override; - bool customizeQueryParam(ContextPtr context, const std::string & key, const std::string & value) override; + bool customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) override; }; } diff --git a/src/Server/IServer.h b/src/Server/IServer.h index 80736fda3ea..c55b045d2a5 100644 --- a/src/Server/IServer.h +++ b/src/Server/IServer.h @@ -28,7 +28,7 @@ public: virtual Poco::Logger & logger() const = 0; /// Returns global application's context. - virtual ContextPtr context() const = 0; + virtual ContextMutablePtr context() const = 0; /// Returns true if shutdown signaled. virtual bool isCancelled() const = 0; diff --git a/src/Server/MySQLHandler.h b/src/Server/MySQLHandler.h index f5fb82b5bef..e681ad2e6f6 100644 --- a/src/Server/MySQLHandler.h +++ b/src/Server/MySQLHandler.h @@ -56,7 +56,7 @@ private: protected: Poco::Logger * log; - ContextPtr connection_context; + ContextMutablePtr connection_context; std::shared_ptr packet_endpoint; diff --git a/src/Server/PostgreSQLHandler.h b/src/Server/PostgreSQLHandler.h index 0f114d388fb..9aaad1d7aa7 100644 --- a/src/Server/PostgreSQLHandler.h +++ b/src/Server/PostgreSQLHandler.h @@ -37,7 +37,7 @@ private: Poco::Logger * log = &Poco::Logger::get("PostgreSQLHandler"); IServer & server; - ContextPtr connection_context; + ContextMutablePtr connection_context; bool ssl_enabled = false; Int32 connection_id = 0; Int32 secret_key = 0; diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 8387ca5f254..287f849ce4c 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -132,8 +132,8 @@ private: UInt64 client_version_patch = 0; UInt64 client_tcp_protocol_version = 0; - ContextPtr connection_context; - ContextPtr query_context; + ContextMutablePtr connection_context; + ContextMutablePtr query_context; size_t unknown_packet_in_send_data = 0; diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.h b/src/Storages/Distributed/DistributedBlockOutputStream.h index f574702f35f..a17e7d6565b 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.h +++ b/src/Storages/Distributed/DistributedBlockOutputStream.h @@ -84,7 +84,7 @@ private: /// Returns the number of blocks was written for each cluster node. Uses during exception handling. std::string getCurrentStateDescription(); - ContextPtr context; + ContextMutablePtr context; StorageDistributed & storage; StorageMetadataPtr metadata_snapshot; ASTPtr query_ast; diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.h b/src/Storages/Kafka/KafkaBlockOutputStream.h index 715ed39b8d6..9f413ae527f 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.h +++ b/src/Storages/Kafka/KafkaBlockOutputStream.h @@ -12,7 +12,7 @@ public: explicit KafkaBlockOutputStream( StorageKafka & storage_, const StorageMetadataPtr & metadata_snapshot_, - const std::shared_ptr & context_); + const std::shared_ptr & context_); Block getHeader() const override; @@ -25,7 +25,7 @@ public: private: StorageKafka & storage; StorageMetadataPtr metadata_snapshot; - const std::shared_ptr context; + const std::shared_ptr context; ProducerBufferPtr buffer; BlockOutputStreamPtr child; }; diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index df09316f333..e787b7bf939 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -191,7 +191,7 @@ private: ASTPtr inner_query; /// stored query : SELECT * FROM ( SELECT a FROM A) ASTPtr inner_subquery; /// stored query's innermost subquery if any ASTPtr inner_blocks_query; /// query over the mergeable blocks to produce final result - ContextPtr live_view_context; + ContextMutablePtr live_view_context; Poco::Logger * log; diff --git a/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp index 7294b82f10d..69369cbc1a3 100644 --- a/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp +++ b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp @@ -16,7 +16,7 @@ namespace ErrorCodes namespace { - void executeDropQuery(const StorageID & storage_id, ContextPtr context) + void executeDropQuery(const StorageID & storage_id, ContextMutablePtr context) { if (!DatabaseCatalog::instance().isTableExist(storage_id, context)) return; @@ -42,7 +42,7 @@ namespace std::unique_ptr TemporaryLiveViewCleaner::the_instance; -void TemporaryLiveViewCleaner::init(ContextPtr global_context_) +void TemporaryLiveViewCleaner::init(ContextMutablePtr global_context_) { if (the_instance) throw Exception("TemporaryLiveViewCleaner already initialized", ErrorCodes::LOGICAL_ERROR); @@ -63,7 +63,7 @@ void TemporaryLiveViewCleaner::shutdown() the_instance.reset(); } -TemporaryLiveViewCleaner::TemporaryLiveViewCleaner(ContextPtr global_context_) : WithContext(global_context_) +TemporaryLiveViewCleaner::TemporaryLiveViewCleaner(ContextMutablePtr global_context_) : WithMutableContext(global_context_) { } diff --git a/src/Storages/LiveView/TemporaryLiveViewCleaner.h b/src/Storages/LiveView/TemporaryLiveViewCleaner.h index 9b31bf9c999..3fe0079a46f 100644 --- a/src/Storages/LiveView/TemporaryLiveViewCleaner.h +++ b/src/Storages/LiveView/TemporaryLiveViewCleaner.h @@ -14,7 +14,7 @@ struct StorageID; /// This class removes temporary live views in the background thread when it's possible. /// There should only a single instance of this class. -class TemporaryLiveViewCleaner : WithContext +class TemporaryLiveViewCleaner : WithMutableContext { public: static TemporaryLiveViewCleaner & instance() { return *the_instance; } @@ -23,7 +23,7 @@ public: void addView(const std::shared_ptr & view); /// Should be called once. - static void init(ContextPtr global_context_); + static void init(ContextMutablePtr global_context_); static void shutdown(); void startup(); @@ -31,7 +31,7 @@ public: private: friend std::unique_ptr::deleter_type; - TemporaryLiveViewCleaner(ContextPtr global_context_); + TemporaryLiveViewCleaner(ContextMutablePtr global_context_); ~TemporaryLiveViewCleaner(); void backgroundThreadFunc(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0ed3f89b2c3..90540b71543 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -136,7 +136,7 @@ MergeTreeData::MergeTreeData( const StorageID & table_id_, const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, - ContextPtr context_, + ContextMutablePtr context_, const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr storage_settings_, @@ -144,7 +144,7 @@ MergeTreeData::MergeTreeData( bool attach, BrokenPartCallback broken_part_callback_) : IStorage(table_id_) - , WithContext(context_->getGlobalContext()) + , WithMutableContext(context_->getGlobalContext()) , merging_params(merging_params_) , require_part_metadata(require_part_metadata_) , relative_data_path(relative_data_path_) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index a80d7eae139..5cef1773246 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -113,7 +113,7 @@ namespace ErrorCodes /// - MergeTreeDataWriter /// - MergeTreeDataMergerMutator -class MergeTreeData : public IStorage, public WithContext +class MergeTreeData : public IStorage, public WithMutableContext { public: /// Function to call if the part is suspected to contain corrupt data. @@ -353,7 +353,7 @@ public: MergeTreeData(const StorageID & table_id_, const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, - ContextPtr context_, + ContextMutablePtr context_, const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr settings_, diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 6f1724c6c2e..129e231e46a 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -79,7 +79,7 @@ protected: std::unique_ptr rabbitmq_settings_); private: - ContextPtr rabbitmq_context; + ContextMutablePtr rabbitmq_context; std::unique_ptr rabbitmq_settings; const String exchange_name; diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 836d8bae7f3..5ca423b449a 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -31,7 +31,7 @@ static void checkAllTypesAreAllowedInTable(const NamesAndTypesList & names_and_t } -ContextPtr StorageFactory::Arguments::getContext() const +ContextMutablePtr StorageFactory::Arguments::getContext() const { auto ptr = context.lock(); if (!ptr) @@ -39,7 +39,7 @@ ContextPtr StorageFactory::Arguments::getContext() const return ptr; } -ContextPtr StorageFactory::Arguments::getLocalContext() const +ContextMutablePtr StorageFactory::Arguments::getLocalContext() const { auto ptr = local_context.lock(); if (!ptr) @@ -59,8 +59,8 @@ void StorageFactory::registerStorage(const std::string & name, CreatorFn creator StoragePtr StorageFactory::get( const ASTCreateQuery & query, const String & relative_data_path, - ContextPtr local_context, - ContextPtr context, + ContextMutablePtr local_context, + ContextMutablePtr context, const ColumnsDescription & columns, const ConstraintsDescription & constraints, bool has_force_restore_data_flag) const diff --git a/src/Storages/StorageFactory.h b/src/Storages/StorageFactory.h index e04e5459dda..bdc57bfdc6d 100644 --- a/src/Storages/StorageFactory.h +++ b/src/Storages/StorageFactory.h @@ -39,16 +39,16 @@ public: /// Relative to from server config (possibly of some of some for *MergeTree) const String & relative_data_path; const StorageID & table_id; - ContextWeakPtr local_context; - ContextWeakPtr context; + ContextWeakMutablePtr local_context; + ContextWeakMutablePtr context; const ColumnsDescription & columns; const ConstraintsDescription & constraints; bool attach; bool has_force_restore_data_flag; const String & comment; - ContextPtr getContext() const; - ContextPtr getLocalContext() const; + ContextMutablePtr getContext() const; + ContextMutablePtr getLocalContext() const; }; /// Analog of the IStorage::supports*() helpers @@ -81,8 +81,8 @@ public: StoragePtr get( const ASTCreateQuery & query, const String & relative_data_path, - ContextPtr local_context, - ContextPtr context, + ContextMutablePtr local_context, + ContextMutablePtr context, const ColumnsDescription & columns, const ConstraintsDescription & constraints, bool has_force_restore_data_flag) const; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 67bd6b21c3f..bc2c6d98c4d 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -54,7 +54,7 @@ StorageMaterializedView::StorageMaterializedView( const ASTCreateQuery & query, const ColumnsDescription & columns_, bool attach_) - : IStorage(table_id_), WithContext(local_context->getGlobalContext()) + : IStorage(table_id_), WithMutableContext(local_context->getGlobalContext()) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -228,7 +228,7 @@ static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, if (auto txn = current_context->getZooKeeperMetadataTransaction()) { /// For Replicated database - drop_context->setQueryContext(current_context); + drop_context->setQueryContext(std::const_pointer_cast(current_context)); drop_context->initZooKeeperMetadataTransaction(txn, true); } InterpreterDropQuery drop_interpreter(ast_drop_query, drop_context); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 8f9c8a9d3f1..8e2b8bdc9f5 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -12,7 +12,7 @@ namespace DB { -class StorageMaterializedView final : public ext::shared_ptr_helper, public IStorage, WithContext +class StorageMaterializedView final : public ext::shared_ptr_helper, public IStorage, WithMutableContext { friend struct ext::shared_ptr_helper; public: diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 5a84e0c3901..15d520c13aa 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -332,7 +332,7 @@ Pipe StorageMerge::createSources( const Block & header, const StorageWithLockAndName & storage_with_lock, Names & real_column_names, - ContextPtr modified_context, + ContextMutablePtr modified_context, size_t streams_num, bool has_table_virtual_column, bool concat_streams) diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 7c5a50af486..b9d44bfa27e 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -92,7 +92,7 @@ protected: const Block & header, const StorageWithLockAndName & storage_with_lock, Names & real_column_names, - ContextPtr modified_context, + ContextMutablePtr modified_context, size_t streams_num, bool has_table_virtual_column, bool concat_streams = false); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index d656c7c088d..c0ec041d7c6 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -62,7 +62,7 @@ StorageMergeTree::StorageMergeTree( const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, bool attach, - ContextPtr context_, + ContextMutablePtr context_, const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr storage_settings_, diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 53199e1595a..3bd4ef2ebec 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -250,7 +250,7 @@ protected: const String & relative_data_path_, const StorageInMemoryMetadata & metadata, bool attach, - ContextPtr context_, + ContextMutablePtr context_, const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr settings_, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 319e04f2424..debed255c9f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -253,7 +253,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const StorageID & table_id_, const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, - ContextPtr context_, + ContextMutablePtr context_, const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr settings_, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 4e697f2d1f2..28fe1541e92 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -685,7 +685,7 @@ protected: const StorageID & table_id_, const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, - ContextPtr context_, + ContextMutablePtr context_, const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr settings_, From 18988a60ad76b330f7665388a998be74f078465e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 May 2021 18:03:45 +0300 Subject: [PATCH 129/154] Fix mutation wait --- src/Storages/StorageReplicatedMergeTree.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 319e04f2424..a83a2f36428 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -523,10 +523,11 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( if (wait_event->tryWait(1000)) continue; - /// Here we check mutation for errors or kill on local replica. If they happen on this replica + /// Here we check mutation for errors on local replica. If they happen on this replica /// they will happen on each replica, so we can check only in-memory info. auto mutation_status = queue.getIncompleteMutationsStatus(mutation_id); - if (!mutation_status || !mutation_status->latest_fail_reason.empty()) + /// If mutation status is empty, than local replica may just not loaded it into memory. + if (mutation_status && !mutation_status->latest_fail_reason.empty()) break; } From c2aaa29a17c15390a8ad782c8dbcec4eccf06fd0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 May 2021 19:07:45 +0300 Subject: [PATCH 130/154] Fix build. --- src/Common/tests/gtest_global_context.h | 2 +- .../tests/gtest_transform_query_for_external_database.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/tests/gtest_global_context.h b/src/Common/tests/gtest_global_context.h index 30ebf1dbca9..9bd7c2490d6 100644 --- a/src/Common/tests/gtest_global_context.h +++ b/src/Common/tests/gtest_global_context.h @@ -5,7 +5,7 @@ struct ContextHolder { DB::SharedContextHolder shared_context; - DB::ContextPtr context; + DB::ContextMutablePtr context; ContextHolder() : shared_context(DB::Context::createShared()) diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 6735206862d..e61123a3aa8 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -22,7 +22,7 @@ struct State { State(const State&) = delete; - ContextPtr context; + ContextMutablePtr context; static const State & instance() { From b9aa4045080adfa261f916b2161288a0725c2eb6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 May 2021 21:25:43 +0300 Subject: [PATCH 131/154] Review fixes. --- .../QueryPlan/ReadFromMergeTree.cpp | 194 ++++++++---------- src/Storages/MergeTree/MergeTreeData.cpp | 2 - .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 - 3 files changed, 82 insertions(+), 119 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 0672d7ed040..fd5de98b4c0 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -230,63 +230,85 @@ Pipe ReadFromMergeTree::read( return pipe; } +namespace +{ + +struct PartRangesReadInfo +{ + std::vector sum_marks_in_parts; + + size_t sum_marks = 0; + size_t total_rows = 0; + size_t adaptive_parts = 0; + size_t index_granularity_bytes = 0; + size_t max_marks_to_use_cache = 0; + size_t min_marks_for_concurrent_read = 0; + + bool use_uncompressed_cache = false; + + PartRangesReadInfo( + const RangesInDataParts & parts, + const Settings & settings, + const MergeTreeSettings & data_settings) + { + /// Count marks for each part. + sum_marks_in_parts.resize(parts.size()); + for (size_t i = 0; i < parts.size(); ++i) + { + total_rows += parts[i].getRowsCount(); + sum_marks_in_parts[i] = parts[i].getMarksCount(); + sum_marks += sum_marks_in_parts[i]; + + if (parts[i].data_part->index_granularity_info.is_adaptive) + ++adaptive_parts; + } + + if (adaptive_parts > parts.size() / 2) + index_granularity_bytes = data_settings.index_granularity_bytes; + + max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( + settings.merge_tree_max_rows_to_use_cache, + settings.merge_tree_max_bytes_to_use_cache, + data_settings.index_granularity, + index_granularity_bytes); + + min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( + settings.merge_tree_min_rows_for_concurrent_read, + settings.merge_tree_min_bytes_for_concurrent_read, + data_settings.index_granularity, + index_granularity_bytes, + sum_marks); + + use_uncompressed_cache = settings.use_uncompressed_cache; + if (sum_marks > max_marks_to_use_cache) + use_uncompressed_cache = false; + } +}; + +} + Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( RangesInDataParts && parts_with_ranges, const Names & column_names) { const auto & settings = context->getSettingsRef(); - - /// Count marks for each part. - std::vector sum_marks_in_parts(parts_with_ranges.size()); - size_t sum_marks = 0; - size_t total_rows = 0; - const auto data_settings = data.getSettings(); - size_t adaptive_parts = 0; - for (size_t i = 0; i < parts_with_ranges.size(); ++i) - { - total_rows += parts_with_ranges[i].getRowsCount(); - sum_marks_in_parts[i] = parts_with_ranges[i].getMarksCount(); - sum_marks += sum_marks_in_parts[i]; - if (parts_with_ranges[i].data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } + PartRangesReadInfo info(parts_with_ranges, settings, *data_settings); - size_t index_granularity_bytes = 0; - if (adaptive_parts > parts_with_ranges.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( - settings.merge_tree_min_rows_for_concurrent_read, - settings.merge_tree_min_bytes_for_concurrent_read, - data_settings->index_granularity, - index_granularity_bytes, - sum_marks); - - bool use_uncompressed_cache = settings.use_uncompressed_cache; - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; - - if (0 == sum_marks) + if (0 == info.sum_marks) return {}; size_t num_streams = requested_num_streams; if (num_streams > 1) { /// Reduce the number of num_streams if the data is small. - if (sum_marks < num_streams * min_marks_for_concurrent_read && parts_with_ranges.size() < num_streams) - num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts_with_ranges.size()); + if (info.sum_marks < num_streams * info.min_marks_for_concurrent_read && parts_with_ranges.size() < num_streams) + num_streams = std::max((info.sum_marks + info.min_marks_for_concurrent_read - 1) / info.min_marks_for_concurrent_read, parts_with_ranges.size()); } return read(std::move(parts_with_ranges), column_names, ReadType::Default, - num_streams, min_marks_for_concurrent_read, use_uncompressed_cache); + num_streams, info.min_marks_for_concurrent_read, info.use_uncompressed_cache); } static ActionsDAGPtr createProjection(const Block & header) @@ -305,44 +327,13 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( const InputOrderInfoPtr & input_order_info) { const auto & settings = context->getSettingsRef(); - size_t sum_marks = 0; - size_t adaptive_parts = 0; - std::vector sum_marks_in_parts(parts_with_ranges.size()); const auto data_settings = data.getSettings(); - for (size_t i = 0; i < parts_with_ranges.size(); ++i) - { - sum_marks_in_parts[i] = parts_with_ranges[i].getMarksCount(); - sum_marks += sum_marks_in_parts[i]; - - if (parts_with_ranges[i].data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } - - size_t index_granularity_bytes = 0; - if (adaptive_parts > parts_with_ranges.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( - settings.merge_tree_min_rows_for_concurrent_read, - settings.merge_tree_min_bytes_for_concurrent_read, - data_settings->index_granularity, - index_granularity_bytes, - sum_marks); - - bool use_uncompressed_cache = settings.use_uncompressed_cache; - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; + PartRangesReadInfo info(parts_with_ranges, settings, *data_settings); Pipes res; - if (sum_marks == 0) + if (info.sum_marks == 0) return {}; /// Let's split ranges to avoid reading much data. @@ -391,7 +382,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( return new_ranges; }; - const size_t min_marks_per_stream = (sum_marks - 1) / requested_num_streams + 1; + const size_t min_marks_per_stream = (info.sum_marks - 1) / requested_num_streams + 1; bool need_preliminary_merge = (parts_with_ranges.size() > settings.read_in_order_two_level_merge_threshold); Pipes pipes; @@ -409,16 +400,16 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( RangesInDataPart part = parts_with_ranges.back(); parts_with_ranges.pop_back(); - size_t & marks_in_part = sum_marks_in_parts.back(); + size_t & marks_in_part = info.sum_marks_in_parts.back(); /// We will not take too few rows from a part. - if (marks_in_part >= min_marks_for_concurrent_read && - need_marks < min_marks_for_concurrent_read) - need_marks = min_marks_for_concurrent_read; + if (marks_in_part >= info.min_marks_for_concurrent_read && + need_marks < info.min_marks_for_concurrent_read) + need_marks = info.min_marks_for_concurrent_read; /// Do not leave too few rows in the part. if (marks_in_part > need_marks && - marks_in_part - need_marks < min_marks_for_concurrent_read) + marks_in_part - need_marks < info.min_marks_for_concurrent_read) need_marks = marks_in_part; MarkRanges ranges_to_get_from_part; @@ -429,7 +420,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( ranges_to_get_from_part = part.ranges; need_marks -= marks_in_part; - sum_marks_in_parts.pop_back(); + info.sum_marks_in_parts.pop_back(); } else { @@ -463,7 +454,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( : ReadFromMergeTree::ReadType::InReverseOrder; pipes.emplace_back(read(std::move(new_parts), column_names, read_type, - requested_num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); + requested_num_streams, info.min_marks_for_concurrent_read, info.use_uncompressed_cache)); } if (need_preliminary_merge) @@ -609,36 +600,14 @@ static void addMergingFinal( Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts_with_range, + RangesInDataParts && parts_with_ranges, const Names & column_names, ActionsDAGPtr & out_projection) { const auto & settings = context->getSettingsRef(); const auto data_settings = data.getSettings(); - size_t sum_marks = 0; - size_t adaptive_parts = 0; - for (const auto & part : parts_with_range) - { - for (const auto & range : part.ranges) - sum_marks += range.end - range.begin; - if (part.data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } - - size_t index_granularity_bytes = 0; - if (adaptive_parts >= parts_with_range.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - bool use_uncompressed_cache = settings.use_uncompressed_cache; - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; + PartRangesReadInfo info(parts_with_ranges, settings, *data_settings); size_t num_streams = requested_num_streams; if (num_streams > settings.max_final_threads) @@ -650,15 +619,15 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// then we will create a pipe for each partition that will run selecting processor and merging processor /// for the parts with this partition. In the end we will unite all the pipes. std::vector parts_to_merge_ranges; - auto it = parts_with_range.begin(); + auto it = parts_with_ranges.begin(); parts_to_merge_ranges.push_back(it); if (settings.do_not_merge_across_partitions_select_final) { - while (it != parts_with_range.end()) + while (it != parts_with_ranges.end()) { it = std::find_if( - it, parts_with_range.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); + it, parts_with_ranges.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); parts_to_merge_ranges.push_back(it); } /// We divide threads for each partition equally. But we will create at least the number of partitions threads. @@ -668,7 +637,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( else { /// If do_not_merge_across_partitions_select_final is false we just merge all the parts. - parts_to_merge_ranges.push_back(parts_with_range.end()); + parts_to_merge_ranges.push_back(parts_with_ranges.end()); } Pipes partition_pipes; @@ -711,7 +680,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( continue; pipe = read(std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder, - num_streams, 0, use_uncompressed_cache); + num_streams, 0, info.use_uncompressed_cache); /// Drop temporary columns, added by 'sorting_key_expr' if (!out_projection) @@ -761,11 +730,12 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size(); + const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( settings.merge_tree_min_rows_for_concurrent_read, settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, - index_granularity_bytes, + info.index_granularity_bytes, sum_marks_in_lonely_parts); /// Reduce the number of num_streams_for_lonely_parts if the data is small. @@ -773,7 +743,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); auto pipe = read(std::move(lonely_parts), column_names, ReadFromMergeTree::ReadType::Default, - num_streams_for_lonely_parts, min_marks_for_concurrent_read, use_uncompressed_cache); + num_streams_for_lonely_parts, min_marks_for_concurrent_read, info.use_uncompressed_cache); /// Drop temporary columns, added by 'sorting_key_expr' if (!out_projection) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a89563682bc..ca9f03d26f1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3855,8 +3855,6 @@ static void selectBestProjection( if (projection_parts.empty()) return; - //candidate.merge_tree_data_select_base_cache = std::make_unique(); - //candidate.merge_tree_data_select_projection_cache = std::make_unique(); auto sum_marks = reader.estimateNumMarksToRead( projection_parts, candidate.required_columns, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index caf69b8ecfb..2d3722ce7cc 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -166,11 +166,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( ProjectionDescription::typeToString(query_info.projection->desc->type), query_info.projection->desc->name); - // if (query_info.projection->merge_tree_data_select_base_cache->sum_marks - // + query_info.projection->merge_tree_data_select_projection_cache->sum_marks - // == 0) - // return std::make_unique(); - MergeTreeData::DataPartsVector projection_parts; MergeTreeData::DataPartsVector normal_parts; for (const auto & part : parts) From 4187aa444bcf090837c0a3978bc2cb4e97d3cc37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 May 2021 22:21:28 +0300 Subject: [PATCH 132/154] Fix typo. Remove "allow_experimental_bigint_types" setting. --- src/Core/Settings.h | 2 +- src/DataTypes/IDataType.h | 2 -- src/Interpreters/InterpreterCreateQuery.cpp | 16 ---------------- 3 files changed, 1 insertion(+), 19 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 321957e2c90..9a584662734 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -415,7 +415,6 @@ class IColumn; M(Bool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \ M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ - M(Bool, allow_experimental_bigint_types, false, "Allow Int128, Int256, UInt256 and Decimal256 types", 0) \ M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ @@ -478,6 +477,7 @@ class IColumn; M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \ M(UInt64, offset, 0, "Offset on read rows from the most 'end' result for select query", 0) \ M(Bool, allow_experimental_funnel_functions, true, "Enable experimental functions for funnel analysis.", 0) \ + M(Bool, allow_experimental_bigint_types, true, "Obsolete setting, does nothing. Allow Int128, UInt128, Int256, UInt256 and Decimal256 types", 0) \ // End of COMMON_SETTINGS diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 85526cd98de..e0f5d3302df 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -341,8 +341,6 @@ struct WhichDataType constexpr bool isNullable() const { return idx == TypeIndex::Nullable; } constexpr bool isFunction() const { return idx == TypeIndex::Function; } constexpr bool isAggregateFunction() const { return idx == TypeIndex::AggregateFunction; } - - constexpr bool IsBigIntOrDeimal() const { return isInt128() || isUInt128() || isInt256() || isUInt256() || isDecimal256(); } }; /// IDataType helpers (alternative for IDataType virtual methods with single point of truth) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 229ffa2af03..dc3fc0458b9 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -645,22 +645,6 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat } } - if (!create.attach && !settings.allow_experimental_bigint_types) - { - for (const auto & name_and_type_pair : properties.columns.getAllPhysical()) - { - WhichDataType which(*name_and_type_pair.type); - if (which.IsBigIntOrDeimal()) - { - const auto & type_name = name_and_type_pair.type->getName(); - String message = "Cannot create table with column '" + name_and_type_pair.name + "' which type is '" - + type_name + "' because experimental bigint types are not allowed. " - + "Set 'allow_experimental_bigint_types' setting to enable."; - throw Exception(message, ErrorCodes::ILLEGAL_COLUMN); - } - } - } - if (!create.attach && !settings.allow_experimental_map_type) { for (const auto & name_and_type_pair : properties.columns.getAllPhysical()) From 3791c1c99d9f64afded5a1eec5226c72265b20c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 May 2021 22:26:34 +0300 Subject: [PATCH 133/154] Someone added useful settings to the section with obsolete settings --- src/Core/Settings.h | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9a584662734..bc308b56599 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -457,13 +457,6 @@ class IColumn; M(UInt64, distributed_ddl_entry_format_version, 1, "Version of DDL entry to write into ZooKeeper", 0) \ M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializeMySQL. If equal to 0, this setting is disabled", 0) \ M(UInt64, external_storage_max_read_bytes, 0, "Limit maximum number of bytes when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializeMySQL. If equal to 0, this setting is disabled", 0) \ - \ - /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ - \ - M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ - M(UInt64, multiple_joins_rewriter_version, 0, "Obsolete setting, does nothing. Will be removed after 2021-03-31", 0) \ - M(Bool, enable_debug_queries, false, "Enabled debug queries, but now is obsolete", 0) \ - M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0) \ M(UnionMode, union_default_mode, UnionMode::Unspecified, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \ M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \ M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \ @@ -472,12 +465,20 @@ class IColumn; M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \ M(Bool, query_plan_filter_push_down, true, "Allow to push down filter by predicate query plan step", 0) \ \ - M(Bool, database_replicated_ddl_output, true, "Obsolete setting, does nothing. Will be removed after 2021-09-08", 0) \ - M(HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Passible values: default, stream.", 0) \ M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \ M(UInt64, offset, 0, "Offset on read rows from the most 'end' result for select query", 0) \ - M(Bool, allow_experimental_funnel_functions, true, "Enable experimental functions for funnel analysis.", 0) \ - M(Bool, allow_experimental_bigint_types, true, "Obsolete setting, does nothing. Allow Int128, UInt128, Int256, UInt256 and Decimal256 types", 0) \ + \ + \ + /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ + M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete setting, does nothing.", 0) \ + M(UInt64, multiple_joins_rewriter_version, 0, "Obsolete setting, does nothing.", 0) \ + 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_funnel_functions, true, "Obsolete setting, does nothing.", 0) \ + M(Bool, allow_experimental_bigint_types, 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. */ // End of COMMON_SETTINGS From 0093ee17b0687345793b06eae8ab87b8150d634d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 31 May 2021 23:17:40 +0300 Subject: [PATCH 134/154] ExpressionJIT remove unnecessary logging --- src/Interpreters/ExpressionJIT.cpp | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 21326947213..b9fb1ae89d8 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -286,15 +286,13 @@ static FunctionBasePtr compile( return nullptr; } - LOG_TRACE(getLogger(), "Try to compile expression {}", dag.dump()); - auto llvm_function = std::make_shared(dag); if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) { - - auto [compiled_function_cache_entry, was_inserted] = compilation_cache->getOrSet(hash_key, [&] () + auto [compiled_function_cache_entry, _] = compilation_cache->getOrSet(hash_key, [&] () { + LOG_TRACE(getLogger(), "Compile expression {}", llvm_function->getName()); CHJIT::CompiledModuleInfo compiled_module_info = compileFunction(getJITInstance(), *llvm_function); auto * compiled_jit_function = getJITInstance().findCompiledFunction(compiled_module_info, llvm_function->getName()); auto compiled_function = std::make_shared(compiled_jit_function, compiled_module_info); @@ -302,27 +300,17 @@ static FunctionBasePtr compile( return std::make_shared(std::move(compiled_function), compiled_module_info.size); }); - if (was_inserted) - LOG_TRACE(getLogger(), - "Put compiled expression {} in cache used cache size {} total cache size {}", - llvm_function->getName(), - compilation_cache->weight(), - compilation_cache->maxSize()); - else - LOG_TRACE(getLogger(), "Get compiled expression {} from cache", llvm_function->getName()); - llvm_function->setCompiledFunction(compiled_function_cache_entry->getCompiledFunction()); } else { + LOG_TRACE(getLogger(), "Compile expression {}", llvm_function->getName()); CHJIT::CompiledModuleInfo compiled_module_info = compileFunction(getJITInstance(), *llvm_function); auto * compiled_jit_function = getJITInstance().findCompiledFunction(compiled_module_info, llvm_function->getName()); auto compiled_function = std::make_shared(compiled_jit_function, compiled_module_info); llvm_function->setCompiledFunction(compiled_function); } - LOG_TRACE(getLogger(), "Use compiled expression {}", llvm_function->getName()); - return llvm_function; } From fd6a6fb1c9e05db6fa34a20e751674d0fe370d19 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 1 Jun 2021 01:46:46 +0300 Subject: [PATCH 135/154] Updated libunwind --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index a491c27b331..53734f420f1 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit a491c27b33109a842d577c0f7ac5f5f218859181 +Subproject commit 53734f420f166e1ca2732dec8998469bfbb7731d From f9cf7c46e105c1d236c08d5762441c6fc675caf9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Jun 2021 02:22:05 +0300 Subject: [PATCH 136/154] better lock in StorageJoin --- src/Storages/StorageJoin.cpp | 67 +++++++++++-------- src/Storages/StorageJoin.h | 6 +- .../01821_join_table_mutation.reference | 2 + .../0_stateless/01821_join_table_mutation.sql | 18 ++++- .../01821_join_table_race_long.reference | 1 + .../0_stateless/01821_join_table_race_long.sh | 25 +++++++ 6 files changed, 85 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/01821_join_table_race_long.reference create mode 100755 tests/queries/0_stateless/01821_join_table_race_long.sh diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 6f7df6f4252..e271a50e505 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -68,11 +68,18 @@ StorageJoin::StorageJoin( restore(); } +BlockOutputStreamPtr StorageJoin::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +{ + std::lock_guard mutate_lock(mutate_mutex); + return StorageSetOrJoinBase::write(query, metadata_snapshot, context); +} void StorageJoin::truncate( const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder&) { - std::lock_guard lock(mutex); + std::lock_guard mutate_lock(mutate_mutex); + std::unique_lock lock(rwlock); + disk->removeRecursive(path); disk->createDirectories(path); disk->createDirectories(path + "tmp/"); @@ -83,45 +90,47 @@ void StorageJoin::truncate( void StorageJoin::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const { - for (const auto & command: commands) - { - switch (command.type) - { - case MutationCommand::Type::DELETE: - break; - case MutationCommand::Type::UPDATE: - throw Exception("Table engine Join doesn't support update mutation, please use insert instead", ErrorCodes::NOT_IMPLEMENTED); - default: - throw Exception("Table engine Join doesn't support this mutation", ErrorCodes::NOT_IMPLEMENTED); - } - } + for (const auto & command : commands) + if (command.type != MutationCommand::DELETE) + throw Exception("Table engine Join supports only DELETE mutations", ErrorCodes::NOT_IMPLEMENTED); } void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) { - // Only delete is supported - std::lock_guard lock(mutex); + /// Firstly accuire lock for mutation, that locks changes of data. + /// We cannot accuire rwlock here, because read lock is needed + /// for execution of mutation interpreter. + std::lock_guard mutate_lock(mutate_mutex); + + constexpr auto tmp_backup_file_name = "tmp/mut.bin"; auto metadata_snapshot = getInMemoryMetadataPtr(); - auto storage = getStorageID(); - auto storage_ptr = DatabaseCatalog::instance().getTable(storage, context); - auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true); - auto in = interpreter->execute(); - in->readPrefix(); - auto new_data = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); - - const String backup_file_name = "1.bin"; // id starts from 1 - auto backup_buf = disk->writeFile(path + "tmp/" + backup_file_name); + auto backup_buf = disk->writeFile(path + tmp_backup_file_name); auto compressed_backup_buf = CompressedWriteBuffer(*backup_buf); auto backup_stream = NativeBlockOutputStream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()); - while (const Block & block = in->read()) + auto new_data = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); + + // New scope controls lifetime of InputStream. { - new_data->addJoinedBlock(block, true); - if (persistent) - backup_stream.write(block); + auto storage_ptr = DatabaseCatalog::instance().getTable(getStorageID(), context); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true); + auto in = interpreter->execute(); + in->readPrefix(); + + while (const Block & block = in->read()) + { + new_data->addJoinedBlock(block, true); + if (persistent) + backup_stream.write(block); + } + + in->readSuffix(); } + /// Now accuire exclusive lock and modify storage. + std::unique_lock lock(rwlock); + join = std::move(new_data); increment = 1; @@ -140,7 +149,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) disk->removeFileIfExists(path + file_name); } - disk->replaceFile(path + "tmp/" + backup_file_name, path + backup_file_name); + disk->replaceFile(path + tmp_backup_file_name, path + std::to_string(increment) + ".bin"); } } diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 7315115520e..78d8b9768e9 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -29,6 +29,7 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; + /// Only delete is supported. void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; void mutate(const MutationCommands & commands, ContextPtr context) override; @@ -44,6 +45,8 @@ public: /// (but not during processing whole query, it's safe for joinGet that doesn't involve `used_flags` from HashJoin) ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, @@ -71,8 +74,7 @@ private: /// Protect state for concurrent use in insertFromBlock and joinBlock. /// Lock is stored in HashJoin instance during query and blocks concurrent insertions. mutable std::shared_mutex rwlock; - - mutable std::mutex mutex; + mutable std::mutex mutate_mutex; void insertBlock(const Block & block) override; void finishInsert() override {} diff --git a/tests/queries/0_stateless/01821_join_table_mutation.reference b/tests/queries/0_stateless/01821_join_table_mutation.reference index e79d145b39b..8c446c806b5 100644 --- a/tests/queries/0_stateless/01821_join_table_mutation.reference +++ b/tests/queries/0_stateless/01821_join_table_mutation.reference @@ -3,3 +3,5 @@ 99 m10 50 +48 +0 diff --git a/tests/queries/0_stateless/01821_join_table_mutation.sql b/tests/queries/0_stateless/01821_join_table_mutation.sql index 9662a197b88..78903ebd6ec 100644 --- a/tests/queries/0_stateless/01821_join_table_mutation.sql +++ b/tests/queries/0_stateless/01821_join_table_mutation.sql @@ -4,13 +4,13 @@ CREATE TABLE join_table_mutation(id Int32, name String) ENGINE = Join(ANY, LEFT, INSERT INTO join_table_mutation select number, toString(number) from numbers(100); -SELECT count(1) FROM join_table_mutation; +SELECT count() FROM join_table_mutation; SELECT name FROM join_table_mutation WHERE id = 10; ALTER TABLE join_table_mutation DELETE WHERE id = 10; -SELECT count(1) FROM join_table_mutation; +SELECT count() FROM join_table_mutation; SELECT name FROM join_table_mutation WHERE id = 10; @@ -20,4 +20,16 @@ SELECT name FROM join_table_mutation WHERE id = 10; ALTER TABLE join_table_mutation DELETE WHERE id % 2 = 0; -SELECT count(1) FROM join_table_mutation; \ No newline at end of file +ALTER TABLE join_table_mutation UPDATE name = 'some' WHERE 1; -- {serverError 48} + +SELECT count() FROM join_table_mutation; + +ALTER TABLE join_table_mutation DELETE WHERE name IN ('1', '2', '3', '4'); + +SELECT count() FROM join_table_mutation; + +ALTER TABLE join_table_mutation DELETE WHERE 1; + +SELECT count() FROM join_table_mutation; + +DROP TABLE join_table_mutation; diff --git a/tests/queries/0_stateless/01821_join_table_race_long.reference b/tests/queries/0_stateless/01821_join_table_race_long.reference new file mode 100644 index 00000000000..f2018833bc6 --- /dev/null +++ b/tests/queries/0_stateless/01821_join_table_race_long.reference @@ -0,0 +1 @@ +1 foo diff --git a/tests/queries/0_stateless/01821_join_table_race_long.sh b/tests/queries/0_stateless/01821_join_table_race_long.sh new file mode 100755 index 00000000000..9602da1e12a --- /dev/null +++ b/tests/queries/0_stateless/01821_join_table_race_long.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS join_table_race" +$CLICKHOUSE_CLIENT -q "CREATE TABLE join_table_race(id Int32, name String) ENGINE = Join(ANY, LEFT, id)" + +for i in {0..100}; do $CLICKHOUSE_CLIENT -q "INSERT INTO join_table_race VALUES ($RANDOM, '$RANDOM')" > /dev/null 2> /dev/null; done & + +for i in {0..200}; do $CLICKHOUSE_CLIENT -q "SELECT count() FROM join_table_race FORMAT Null" > /dev/null 2> /dev/null; done & + +for i in {0..100}; do $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE join_table_race" > /dev/null 2> /dev/null; done & + +for i in {0..100}; do $CLICKHOUSE_CLIENT -q "ALTER TABLE join_table_race DELETE WHERE id % 2 = 0" > /dev/null 2> /dev/null; done & + +wait + +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE join_table_race" +$CLICKHOUSE_CLIENT -q "INSERT INTO join_table_race VALUES (1, 'foo')" +$CLICKHOUSE_CLIENT -q "SELECT id, name FROM join_table_race" + +$CLICKHOUSE_CLIENT -q "DROP TABLE join_table_race" From 4e038b2a61e24c7486b9b46e6d5a6d0755b60b6e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Jun 2021 10:30:06 +0300 Subject: [PATCH 137/154] Remove copy-paste and rename (review fixes) --- src/Common/ZooKeeper/ZooKeeper.cpp | 40 +++++++++---------- src/Common/ZooKeeper/ZooKeeper.h | 14 +++---- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- 4 files changed, 30 insertions(+), 30 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 6d64aff99bf..7f848cfca42 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -238,7 +238,7 @@ Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings Coordination::Stat * stat, Coordination::WatchCallback watch_callback) { - auto future_result = tryAsyncGetChildrenNoThrow(path, watch_callback); + auto future_result = asyncTryGetChildrenNoThrow(path, watch_callback); if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { @@ -299,7 +299,7 @@ Coordination::Error ZooKeeper::tryGetChildrenWatch(const std::string & path, Str Coordination::Error ZooKeeper::createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created) { - auto future_result = tryAsyncCreateNoThrow(path, data, mode); + auto future_result = asyncTryCreateNoThrow(path, data, mode); if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { @@ -368,7 +368,7 @@ void ZooKeeper::createAncestors(const std::string & path) Coordination::Error ZooKeeper::removeImpl(const std::string & path, int32_t version) { - auto future_result = tryAsyncRemoveNoThrow(path, version); + auto future_result = asyncTryRemoveNoThrow(path, version); if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) @@ -401,7 +401,7 @@ Coordination::Error ZooKeeper::tryRemove(const std::string & path, int32_t versi Coordination::Error ZooKeeper::existsImpl(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback) { - auto future_result = tryAsyncExistsNoThrow(path, watch_callback); + auto future_result = asyncTryExistsNoThrow(path, watch_callback); if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { @@ -435,7 +435,7 @@ bool ZooKeeper::existsWatch(const std::string & path, Coordination::Stat * stat, Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback) { - auto future_result = tryAsyncGetNoThrow(path, watch_callback); + auto future_result = asyncTryGetNoThrow(path, watch_callback); if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { @@ -508,7 +508,7 @@ bool ZooKeeper::tryGetWatch( Coordination::Error ZooKeeper::setImpl(const std::string & path, const std::string & data, int32_t version, Coordination::Stat * stat) { - auto future_result = tryAsyncSetNoThrow(path, data, version); + auto future_result = asyncTrySetNoThrow(path, data, version); if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { @@ -560,7 +560,7 @@ Coordination::Error ZooKeeper::multiImpl(const Coordination::Requests & requests if (requests.empty()) return Coordination::Error::ZOK; - auto future_result = tryAsyncMultiNoThrow(requests); + auto future_result = asyncTryMultiNoThrow(requests); if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { @@ -758,12 +758,12 @@ std::future ZooKeeper::asyncCreate(const std::stri return future; } -std::future ZooKeeper::tryAsyncCreateNoThrow(const std::string & path, const std::string & data, int32_t mode) +std::future ZooKeeper::asyncTryCreateNoThrow(const std::string & path, const std::string & data, int32_t mode) { auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise, path](const Coordination::CreateResponse & response) mutable + auto callback = [promise](const Coordination::CreateResponse & response) mutable { promise->set_value(response); }; @@ -789,12 +789,12 @@ std::future ZooKeeper::asyncGet(const std::string & p return future; } -std::future ZooKeeper::tryAsyncGetNoThrow(const std::string & path, Coordination::WatchCallback watch_callback) +std::future ZooKeeper::asyncTryGetNoThrow(const std::string & path, Coordination::WatchCallback watch_callback) { auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise, path](const Coordination::GetResponse & response) mutable + auto callback = [promise](const Coordination::GetResponse & response) mutable { promise->set_value(response); }; @@ -838,12 +838,12 @@ std::future ZooKeeper::asyncExists(const std::stri return future; } -std::future ZooKeeper::tryAsyncExistsNoThrow(const std::string & path, Coordination::WatchCallback watch_callback) +std::future ZooKeeper::asyncTryExistsNoThrow(const std::string & path, Coordination::WatchCallback watch_callback) { auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise, path](const Coordination::ExistsResponse & response) mutable + auto callback = [promise](const Coordination::ExistsResponse & response) mutable { promise->set_value(response); }; @@ -870,12 +870,12 @@ std::future ZooKeeper::asyncSet(const std::string & p } -std::future ZooKeeper::tryAsyncSetNoThrow(const std::string & path, const std::string & data, int32_t version) +std::future ZooKeeper::asyncTrySetNoThrow(const std::string & path, const std::string & data, int32_t version) { auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise, path](const Coordination::SetResponse & response) mutable + auto callback = [promise](const Coordination::SetResponse & response) mutable { promise->set_value(response); }; @@ -901,12 +901,12 @@ std::future ZooKeeper::asyncGetChildren(const std::s return future; } -std::future ZooKeeper::tryAsyncGetChildrenNoThrow(const std::string & path, Coordination::WatchCallback watch_callback) +std::future ZooKeeper::asyncTryGetChildrenNoThrow(const std::string & path, Coordination::WatchCallback watch_callback) { auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise, path](const Coordination::ListResponse & response) mutable + auto callback = [promise](const Coordination::ListResponse & response) mutable { promise->set_value(response); }; @@ -954,12 +954,12 @@ std::future ZooKeeper::asyncTryRemove(const std::s return future; } -std::future ZooKeeper::tryAsyncRemoveNoThrow(const std::string & path, int32_t version) +std::future ZooKeeper::asyncTryRemoveNoThrow(const std::string & path, int32_t version) { auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise, path](const Coordination::RemoveResponse & response) mutable + auto callback = [promise](const Coordination::RemoveResponse & response) mutable { promise->set_value(response); }; @@ -968,7 +968,7 @@ std::future ZooKeeper::tryAsyncRemoveNoThrow(const return future; } -std::future ZooKeeper::tryAsyncMultiNoThrow(const Coordination::Requests & ops) +std::future ZooKeeper::asyncTryMultiNoThrow(const Coordination::Requests & ops) { auto promise = std::make_shared>(); auto future = promise->get_future(); diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index e855a450b96..7aafee52bf0 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -222,37 +222,37 @@ public: using FutureCreate = std::future; FutureCreate asyncCreate(const std::string & path, const std::string & data, int32_t mode); /// Like the previous one but don't throw any exceptions on future.get() - FutureCreate tryAsyncCreateNoThrow(const std::string & path, const std::string & data, int32_t mode); + FutureCreate asyncTryCreateNoThrow(const std::string & path, const std::string & data, int32_t mode); using FutureGet = std::future; FutureGet asyncGet(const std::string & path, Coordination::WatchCallback watch_callback = {}); /// Like the previous one but don't throw any exceptions on future.get() - FutureGet tryAsyncGetNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); + FutureGet asyncTryGetNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); using FutureExists = std::future; FutureExists asyncExists(const std::string & path, Coordination::WatchCallback watch_callback = {}); /// Like the previous one but don't throw any exceptions on future.get() - FutureExists tryAsyncExistsNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); + FutureExists asyncTryExistsNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); using FutureGetChildren = std::future; FutureGetChildren asyncGetChildren(const std::string & path, Coordination::WatchCallback watch_callback = {}); /// Like the previous one but don't throw any exceptions on future.get() - FutureGetChildren tryAsyncGetChildrenNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); + FutureGetChildren asyncTryGetChildrenNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); using FutureSet = std::future; FutureSet asyncSet(const std::string & path, const std::string & data, int32_t version = -1); /// Like the previous one but don't throw any exceptions on future.get() - FutureSet tryAsyncSetNoThrow(const std::string & path, const std::string & data, int32_t version = -1); + FutureSet asyncTrySetNoThrow(const std::string & path, const std::string & data, int32_t version = -1); using FutureRemove = std::future; FutureRemove asyncRemove(const std::string & path, int32_t version = -1); /// Like the previous one but don't throw any exceptions on future.get() - FutureRemove tryAsyncRemoveNoThrow(const std::string & path, int32_t version = -1); + FutureRemove asyncTryRemoveNoThrow(const std::string & path, int32_t version = -1); using FutureMulti = std::future; FutureMulti asyncMulti(const Coordination::Requests & ops); /// Like the previous one but don't throw any exceptions on future.get() - FutureMulti tryAsyncMultiNoThrow(const Coordination::Requests & ops); + FutureMulti asyncTryMultiNoThrow(const Coordination::Requests & ops); /// Very specific methods introduced without following general style. Implements /// some custom throw/no throw logic on future.get(). diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 9e994b3f7f4..10e2d77eb27 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -291,7 +291,7 @@ void ReplicatedMergeTreeCleanupThread::markLostReplicas(const std::unordered_map std::vector futures; for (size_t i = 0; i < candidate_lost_replicas.size(); ++i) - futures.emplace_back(zookeeper->tryAsyncMultiNoThrow(requests[i])); + futures.emplace_back(zookeeper->asyncTryMultiNoThrow(requests[i])); for (size_t i = 0; i < candidate_lost_replicas.size(); ++i) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 91246cf5ac9..43c005eceb4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6144,7 +6144,7 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const St { Coordination::Requests ops; removePartFromZooKeeper(part_names[i], ops, exists_resp.stat.numChildren > 0); - remove_futures.emplace_back(zookeeper->tryAsyncMultiNoThrow(ops)); + remove_futures.emplace_back(zookeeper->asyncTryMultiNoThrow(ops)); } } @@ -6205,7 +6205,7 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper( { Coordination::Requests ops; removePartFromZooKeeper(part_names[i], ops, exists_resp.stat.numChildren > 0); - remove_futures.emplace_back(zookeeper->tryAsyncMultiNoThrow(ops)); + remove_futures.emplace_back(zookeeper->asyncTryMultiNoThrow(ops)); } else { From a563990812d96c50f5c84bc7b161b43e4b169398 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 1 Jun 2021 10:50:02 +0300 Subject: [PATCH 138/154] Fixed clang tidy --- src/Storages/StorageDictionary.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 1ce74b16f46..5aeaff590e1 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -185,7 +185,7 @@ void StorageDictionary::startup() bool lazy_load = global_context->getConfigRef().getBool("dictionaries_lazy_load", true); if (!lazy_load) { - auto & external_dictionaries_loader = global_context->getExternalDictionariesLoader(); + const auto & external_dictionaries_loader = global_context->getExternalDictionariesLoader(); /// reloadConfig() is called here to force loading the dictionary. external_dictionaries_loader.reloadConfig(getStorageID().getInternalDictionaryName()); @@ -220,7 +220,7 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id) configuration->setString("dictionary.database", new_table_id.database_name); configuration->setString("dictionary.name", new_table_id.table_name); - auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); + const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); external_dictionaries_loader.reloadConfig(getStorageID().getInternalDictionaryName()); auto result = external_dictionaries_loader.getLoadResult(getStorageID().getInternalDictionaryName()); From aa366829dec063b3a86b2434eedd02b618a2d3c8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Jun 2021 12:00:19 +0300 Subject: [PATCH 139/154] Fix special build check. --- src/Storages/StorageDictionary.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 1ce74b16f46..5aeaff590e1 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -185,7 +185,7 @@ void StorageDictionary::startup() bool lazy_load = global_context->getConfigRef().getBool("dictionaries_lazy_load", true); if (!lazy_load) { - auto & external_dictionaries_loader = global_context->getExternalDictionariesLoader(); + const auto & external_dictionaries_loader = global_context->getExternalDictionariesLoader(); /// reloadConfig() is called here to force loading the dictionary. external_dictionaries_loader.reloadConfig(getStorageID().getInternalDictionaryName()); @@ -220,7 +220,7 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id) configuration->setString("dictionary.database", new_table_id.database_name); configuration->setString("dictionary.name", new_table_id.table_name); - auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); + const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); external_dictionaries_loader.reloadConfig(getStorageID().getInternalDictionaryName()); auto result = external_dictionaries_loader.getLoadResult(getStorageID().getInternalDictionaryName()); From c673eb20408e01e546719f11690a69932c3fdced Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Jun 2021 12:27:05 +0300 Subject: [PATCH 140/154] Use mutable context for TCPHandler::customizeContext --- src/Server/TCPHandler.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 287f849ce4c..086c1f7d5e5 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -119,7 +119,7 @@ public: void run() override; /// This method is called right before the query execution. - virtual void customizeContext(ContextPtr /*context*/) {} + virtual void customizeContext(ContextMutablePtr /*context*/) {} private: IServer & server; From 82c90ebc056c563076f12f4b033efc8bbb18f09e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Jun 2021 12:45:01 +0300 Subject: [PATCH 141/154] Some improvements in stress test --- docker/test/stress/stress | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 4fbedceb0b8..e59c87a8b68 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -27,7 +27,7 @@ def get_options(i): options += " --db-engine=Ordinary" if i % 3 == 2: - options += ''' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i) + options += ''' --client-option='allow_experimental_database_replicated=1' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i) # If database name is not specified, new database is created for each functional test. # Run some threads with one database for all tests. @@ -58,6 +58,10 @@ def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_t time.sleep(0.5) return pipes +def compress_stress_logs(output_path, files_prefix): + cmd = f"cd {output_path} && tar -zcf stress_run_logs.tar.gz {files_prefix}* && rm {files_prefix}*" + check_output(cmd, shell=True) + def prepare_for_hung_check(): # FIXME this function should not exist, but... @@ -116,7 +120,7 @@ if __name__ == "__main__": parser.add_argument("--server-log-folder", default='/var/log/clickhouse-server') parser.add_argument("--output-folder") parser.add_argument("--global-time-limit", type=int, default=3600) - parser.add_argument("--num-parallel", default=cpu_count()) + parser.add_argument("--num-parallel", type=int, default=cpu_count()) parser.add_argument('--hung-check', action='store_true', default=False) args = parser.parse_args() @@ -135,6 +139,11 @@ if __name__ == "__main__": time.sleep(5) logging.info("All processes finished") + + logging.info("Compressing stress logs") + compress_stress_logs(args.output_folder, "stress_test_run_") + logging.info("Logs compressed") + if args.hung_check: have_long_running_queries = prepare_for_hung_check() logging.info("Checking if some queries hung") From 94d2aed336ea551c2a8bd1752615bd75dee01529 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Jun 2021 13:24:06 +0300 Subject: [PATCH 142/154] fix tests --- src/Databases/DatabaseReplicated.cpp | 24 +++++++++++++++---- ...per_test_alter_compression_codecs_long.sql | 4 ++-- .../01710_projection_fetch.reference | 8 +++---- .../0_stateless/01710_projection_fetch.sql | 4 ++-- tests/queries/skip_list.json | 6 +++-- 5 files changed, 32 insertions(+), 14 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 3482fad80b9..811114704c3 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include @@ -317,10 +318,25 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ if (!replicated_table || !create->storage->engine->arguments) return; - ASTs & args = create->storage->engine->arguments->children; + ASTs & args_ref = create->storage->engine->arguments->children; + ASTs args = args_ref; if (args.size() < 2) return; + /// It can be a constant expression. Try to evaluate it, ignore exception if we cannot. + bool has_expression_argument = args_ref[0]->as() || args_ref[0]->as(); + if (has_expression_argument) + { + try + { + args[0] = evaluateConstantExpressionAsLiteral(args_ref[0]->clone(), query_context); + args[1] = evaluateConstantExpressionAsLiteral(args_ref[1]->clone(), query_context); + } + catch (...) + { + } + } + ASTLiteral * arg1 = args[0]->as(); ASTLiteral * arg2 = args[1]->as(); if (!arg1 || !arg2 || arg1->value.getType() != Field::Types::String || arg2->value.getType() != Field::Types::String) @@ -348,12 +364,12 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ if (maybe_shard_macros && maybe_replica_macros) return; - if (enable_functional_tests_helper) + if (enable_functional_tests_helper && !has_expression_argument) { if (maybe_path.empty() || maybe_path.back() != '/') maybe_path += '/'; - arg1->value = maybe_path + "auto_{shard}"; - arg2->value = maybe_replica + "auto_{replica}"; + args_ref[0]->as()->value = maybe_path + "auto_{shard}"; + args_ref[1]->as()->value = maybe_replica + "auto_{replica}"; return; } 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 085a79485fb..548f26eadd0 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', '1') PARTITION BY somedate ORDER BY id; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/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', '2') PARTITION BY somedate ORDER BY id; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/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/01710_projection_fetch.reference b/tests/queries/0_stateless/01710_projection_fetch.reference index 54e5bff80a9..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/01710_projection_fetch_default\', \'2\')\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/01710_projection_fetch_default\', \'2\')\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/01710_projection_fetch_default\', \'2\')\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/01710_projection_fetch_default\', \'2\')\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 06790317808..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/01710_projection_fetch_' || currentDatabase(), '1') 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/01710_projection_fetch_' || currentDatabase(), '2') 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); diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 6a278316387..a9a41b1ac1f 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -109,6 +109,7 @@ "01153_attach_mv_uuid" ], "database-replicated": [ + /// Unclassified "memory_tracking", "memory_usage", "live_view", @@ -167,8 +168,9 @@ /// Does not support renaming of multiple tables in single query "00634_rename_view", "00140_rename", - "01783_http_chunk_size", - "01710_projection_fetch" + /// Requires investigation + "00953_zookeeper_suetin_deduplication_bug", + "01783_http_chunk_size" ], "polymorphic-parts": [ "01508_partition_pruning_long", /// bug, shoud be fixed From 24d1d7b6c14f4a8ea18791154cbd6e3fd8a05e71 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Tue, 1 Jun 2021 13:49:24 +0300 Subject: [PATCH 143/154] Update docs/en/sql-reference/statements/optimize.md --- docs/en/sql-reference/statements/optimize.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 8ba6db4cbf6..8dd431cd694 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -60,7 +60,7 @@ CREATE TABLE example ( materialized_value UInt32 MATERIALIZED 12345, aliased_value UInt32 ALIAS 2, PRIMARY KEY primary_key -) ENGINE=MergeTree  +) ENGINE=MergeTree PARTITION BY partition_key ORDER BY (primary_key, secondary_key); ``` @@ -189,4 +189,4 @@ Result: ┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐ │ 1 │ 1 │ 2 │ 3 │ └─────────────┴───────────────┴───────┴───────────────┘ -``` \ No newline at end of file +``` From 356734bd471cca28110f94e03fa5d348762741c6 Mon Sep 17 00:00:00 2001 From: Max Bruce Date: Tue, 1 Jun 2021 04:21:33 -0700 Subject: [PATCH 144/154] Add Rust language third-party SDK, Klickhouse --- docs/en/interfaces/third-party/client-libraries.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index f5c85289171..835e3c6e98a 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -42,6 +42,8 @@ toc_title: Client Libraries - Ruby - [ClickHouse (Ruby)](https://github.com/shlima/click_house) - [clickhouse-activerecord](https://github.com/PNixx/clickhouse-activerecord) +- Rust + - [Klickhouse](https://github.com/Protryon/klickhouse) - R - [clickhouse-r](https://github.com/hannesmuehleisen/clickhouse-r) - [RClickHouse](https://github.com/IMSMWU/RClickHouse) From e2941a8006bbcb0e34b9967aaae7e3da8ea821d3 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 1 Jun 2021 15:15:13 +0300 Subject: [PATCH 145/154] Update DatabaseReplicated.cpp --- 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 811114704c3..b3e5fc67151 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -324,7 +324,7 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ return; /// It can be a constant expression. Try to evaluate it, ignore exception if we cannot. - bool has_expression_argument = args_ref[0]->as() || args_ref[0]->as(); + bool has_expression_argument = args_ref[0]->as() || args_ref[1]->as(); if (has_expression_argument) { try From 84af235a6037f7d87b0139a94cf1be91ec1ff407 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Jun 2021 15:21:41 +0300 Subject: [PATCH 146/154] fix test --- tests/queries/0_stateless/01821_join_table_race_long.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01821_join_table_race_long.sh b/tests/queries/0_stateless/01821_join_table_race_long.sh index 9602da1e12a..7c56bf77bfd 100755 --- a/tests/queries/0_stateless/01821_join_table_race_long.sh +++ b/tests/queries/0_stateless/01821_join_table_race_long.sh @@ -8,13 +8,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS join_table_race" $CLICKHOUSE_CLIENT -q "CREATE TABLE join_table_race(id Int32, name String) ENGINE = Join(ANY, LEFT, id)" -for i in {0..100}; do $CLICKHOUSE_CLIENT -q "INSERT INTO join_table_race VALUES ($RANDOM, '$RANDOM')" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "INSERT INTO join_table_race VALUES ($RANDOM, '$RANDOM')" > /dev/null 2> /dev/null; done & -for i in {0..200}; do $CLICKHOUSE_CLIENT -q "SELECT count() FROM join_table_race FORMAT Null" > /dev/null 2> /dev/null; done & +for _ in {0..200}; do $CLICKHOUSE_CLIENT -q "SELECT count() FROM join_table_race FORMAT Null" > /dev/null 2> /dev/null; done & -for i in {0..100}; do $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE join_table_race" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE join_table_race" > /dev/null 2> /dev/null; done & -for i in {0..100}; do $CLICKHOUSE_CLIENT -q "ALTER TABLE join_table_race DELETE WHERE id % 2 = 0" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "ALTER TABLE join_table_race DELETE WHERE id % 2 = 0" > /dev/null 2> /dev/null; done & wait From 375dd1b6c4ade021bf1948584734053497aecc93 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 1 Jun 2021 19:26:54 +0300 Subject: [PATCH 147/154] Update 01154_move_partition_long.sh --- tests/queries/0_stateless/01154_move_partition_long.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01154_move_partition_long.sh b/tests/queries/0_stateless/01154_move_partition_long.sh index f666cc929cc..66ebbacee42 100755 --- a/tests/queries/0_stateless/01154_move_partition_long.sh +++ b/tests/queries/0_stateless/01154_move_partition_long.sh @@ -6,8 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) declare -A engines engines[0]="MergeTree" -engines[1]="ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/src', toString(randConstant()))" -engines[2]="ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/src_' || toString(randConstant()), 'single_replica')" +engines[1]="ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}/src', '{replica}_' || toString(randConstant()))" +engines[2]="ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}/src_' || toString(randConstant()), '{replica}')" for ((i=0; i<16; i++)) do $CLICKHOUSE_CLIENT -q "CREATE TABLE dst_$i (p UInt64, k UInt64, v UInt64) From 8893f07b2989ae2daa60acd03a2e3c08e48046a8 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Tue, 1 Jun 2021 20:58:33 +0300 Subject: [PATCH 148/154] Touch to rebuild --- docker/test/sqlancer/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/sqlancer/Dockerfile b/docker/test/sqlancer/Dockerfile index 253ca1b729a..b08b1b0bf0d 100644 --- a/docker/test/sqlancer/Dockerfile +++ b/docker/test/sqlancer/Dockerfile @@ -3,6 +3,7 @@ FROM ubuntu:20.04 RUN apt-get update --yes && env DEBIAN_FRONTEND=noninteractive apt-get install wget unzip git openjdk-14-jdk maven python3 --yes --no-install-recommends RUN wget https://github.com/sqlancer/sqlancer/archive/master.zip -O /sqlancer.zip + RUN mkdir /sqlancer && \ cd /sqlancer && \ unzip /sqlancer.zip From b4570bc06b9d9d5581e6f5cd1996140f00c4b6c3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 1 Jun 2021 21:09:41 +0300 Subject: [PATCH 149/154] Reverted libunwind from upstream --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index 53734f420f1..a491c27b331 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 53734f420f166e1ca2732dec8998469bfbb7731d +Subproject commit a491c27b33109a842d577c0f7ac5f5f218859181 From 94b18bdfb8301439593b69e28c46d3c6147102b0 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 2 Jun 2021 03:11:32 +0300 Subject: [PATCH 150/154] Update requirements.txt --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 85f9dc2a9dd..facfc2d1ba1 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -34,5 +34,5 @@ soupsieve==2.0.1 termcolor==1.1.0 tornado==6.1 Unidecode==1.1.1 -urllib3==1.25.10 +urllib3>=1.26.5 Pygments>=2.7.4 From 931cfb548dd49bbe1c065b7269be92e7f80ab0b7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 2 Jun 2021 09:10:03 +0300 Subject: [PATCH 151/154] docs: update requests (to fix conflicts with urllib3) Fixes: 94b18bdfb8301439593b69e28c46d3c6147102b0 --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index facfc2d1ba1..9bb4f57e9e2 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -27,7 +27,7 @@ pymdown-extensions==8.0 python-slugify==4.0.1 PyYAML==5.4.1 repackage==0.7.3 -requests==2.24.0 +requests==2.25.1 singledispatch==3.4.0.3 six==1.15.0 soupsieve==2.0.1 From 2248c776438b1b2e70b40b282fa5d735721d39ad Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 2 Jun 2021 17:18:54 +0300 Subject: [PATCH 152/154] done --- programs/client/Suggest.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index 8d4c0fdbd5a..029388a0bda 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace DB { @@ -90,7 +92,7 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo /// NOTE: Once you will update the completion list, /// do not forget to update 01676_clickhouse_client_autocomplete.sh - std::stringstream query; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + WriteBufferFromOwnString query; query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" "SELECT name FROM system.functions" " UNION ALL " From 9bbc9f97b4b0694983f2b0d7c8536d69e730056b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 2 Jun 2021 23:09:55 +0800 Subject: [PATCH 153/154] Implement antlr parser for projections (#24245) * implement projection grammer for antlr parser * Add comment --- src/Parsers/ASTProjectionDeclaration.cpp | 9 +- src/Parsers/ASTProjectionDeclaration.h | 2 +- src/Parsers/ASTProjectionSelectQuery.cpp | 18 +- src/Parsers/ASTProjectionSelectQuery.h | 7 - src/Parsers/New/AST/AlterTableQuery.cpp | 144 +- src/Parsers/New/AST/AlterTableQuery.h | 26 +- src/Parsers/New/AST/CreateTableQuery.cpp | 5 + src/Parsers/New/AST/SelectUnionQuery.cpp | 72 + src/Parsers/New/AST/SelectUnionQuery.h | 23 + src/Parsers/New/AST/TableElementExpr.cpp | 30 + src/Parsers/New/AST/TableElementExpr.h | 7 + src/Parsers/New/AST/fwd_decl.h | 2 + src/Parsers/New/ClickHouseLexer.cpp | 2700 +++---- src/Parsers/New/ClickHouseLexer.g4 | 2 + src/Parsers/New/ClickHouseLexer.h | 84 +- src/Parsers/New/ClickHouseParser.cpp | 7260 ++++++++++--------- src/Parsers/New/ClickHouseParser.g4 | 66 +- src/Parsers/New/ClickHouseParser.h | 466 +- src/Parsers/New/ClickHouseParserVisitor.h | 22 +- src/Parsers/New/ParseTreeVisitor.h | 12 +- src/Parsers/ParserCreateQuery.cpp | 3 +- src/Parsers/ParserProjectionSelectQuery.cpp | 18 +- tests/queries/skip_list.json | 3 - 23 files changed, 6102 insertions(+), 4879 deletions(-) diff --git a/src/Parsers/ASTProjectionDeclaration.cpp b/src/Parsers/ASTProjectionDeclaration.cpp index e607605ff36..3692b1a2013 100644 --- a/src/Parsers/ASTProjectionDeclaration.cpp +++ b/src/Parsers/ASTProjectionDeclaration.cpp @@ -4,11 +4,14 @@ namespace DB { + ASTPtr ASTProjectionDeclaration::clone() const { - auto clone = std::make_shared(*this); - clone->cloneChildren(); - return clone; + auto res = std::make_shared(); + res->name = name; + if (query) + res->set(res->query, query->clone()); + return res; } diff --git a/src/Parsers/ASTProjectionDeclaration.h b/src/Parsers/ASTProjectionDeclaration.h index f96307ac51a..9d6c953a52b 100644 --- a/src/Parsers/ASTProjectionDeclaration.h +++ b/src/Parsers/ASTProjectionDeclaration.h @@ -12,7 +12,7 @@ class ASTProjectionDeclaration : public IAST { public: String name; - ASTPtr query; + IAST * query; /** Get the text that identifies this element. */ String getID(char) const override { return "Projection"; } diff --git a/src/Parsers/ASTProjectionSelectQuery.cpp b/src/Parsers/ASTProjectionSelectQuery.cpp index cc0d387b933..31a306bc011 100644 --- a/src/Parsers/ASTProjectionSelectQuery.cpp +++ b/src/Parsers/ASTProjectionSelectQuery.cpp @@ -37,7 +37,6 @@ ASTPtr ASTProjectionSelectQuery::clone() const */ CLONE(Expression::WITH); CLONE(Expression::SELECT); - CLONE(Expression::WHERE); CLONE(Expression::GROUP_BY); CLONE(Expression::ORDER_BY); @@ -47,13 +46,6 @@ ASTPtr ASTProjectionSelectQuery::clone() const } -void ASTProjectionSelectQuery::updateTreeHashImpl(SipHash & hash_state) const -{ - hash_state.update(distinct); - IAST::updateTreeHashImpl(hash_state); -} - - void ASTProjectionSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const { frame.current_select = this; @@ -67,16 +59,10 @@ void ASTProjectionSelectQuery::formatImpl(const FormatSettings & s, FormatState s.ostr << s.nl_or_ws; } - s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "SELECT " << (distinct ? "DISTINCT " : "") << (s.hilite ? hilite_none : ""); + s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "SELECT " << (s.hilite ? hilite_none : ""); s.one_line ? select()->formatImpl(s, state, frame) : select()->as().formatImplMultiline(s, state, frame); - if (where()) - { - s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "WHERE " << (s.hilite ? hilite_none : ""); - where()->formatImpl(s, state, frame); - } - if (groupBy()) { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY " << (s.hilite ? hilite_none : ""); @@ -129,8 +115,6 @@ ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const select_query->setExpression(ASTSelectQuery::Expression::WITH, with()->clone()); if (select()) select_query->setExpression(ASTSelectQuery::Expression::SELECT, select()->clone()); - if (where()) - select_query->setExpression(ASTSelectQuery::Expression::WHERE, where()->clone()); if (groupBy()) select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, groupBy()->clone()); // Get rid of orderBy. It's used for projection definition only diff --git a/src/Parsers/ASTProjectionSelectQuery.h b/src/Parsers/ASTProjectionSelectQuery.h index 85baa3289e2..71334c50868 100644 --- a/src/Parsers/ASTProjectionSelectQuery.h +++ b/src/Parsers/ASTProjectionSelectQuery.h @@ -15,7 +15,6 @@ public: { WITH, SELECT, - WHERE, GROUP_BY, ORDER_BY, }; @@ -25,14 +24,10 @@ public: ASTPtr clone() const override; - bool distinct = false; - ASTPtr & refSelect() { return getExpression(Expression::SELECT); } - ASTPtr & refWhere() { return getExpression(Expression::WHERE); } const ASTPtr with() const { return getExpression(Expression::WITH); } const ASTPtr select() const { return getExpression(Expression::SELECT); } - const ASTPtr where() const { return getExpression(Expression::WHERE); } const ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); } const ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); } @@ -47,8 +42,6 @@ public: return {}; } - void updateTreeHashImpl(SipHash & hash_state) const override; - ASTPtr cloneToASTSelect() const; protected: diff --git a/src/Parsers/New/AST/AlterTableQuery.cpp b/src/Parsers/New/AST/AlterTableQuery.cpp index 4b75e8fe1b6..3f901f95639 100644 --- a/src/Parsers/New/AST/AlterTableQuery.cpp +++ b/src/Parsers/New/AST/AlterTableQuery.cpp @@ -91,6 +91,15 @@ PtrTo AlterTableClause::createAddIndex(bool if_not_exists, Ptr return query; } +// static +PtrTo AlterTableClause::createAddProjection(bool if_not_exists, PtrTo element, PtrTo after) +{ + assert(element->getType() == TableElementExpr::ExprType::PROJECTION); + PtrTo query(new AlterTableClause(ClauseType::ADD_PROJECTION, {element, after})); + query->if_not_exists = if_not_exists; + return query; +} + // static PtrTo AlterTableClause::createAttach(PtrTo clause, PtrTo from) { @@ -98,9 +107,23 @@ PtrTo AlterTableClause::createAttach(PtrTo cl } // static -PtrTo AlterTableClause::createClear(bool if_exists, PtrTo identifier, PtrTo in) +PtrTo AlterTableClause::createClearColumn(bool if_exists, PtrTo identifier, PtrTo in) { - PtrTo query(new AlterTableClause(ClauseType::CLEAR, {identifier, in})); + PtrTo query(new AlterTableClause(ClauseType::CLEAR_COLUMN, {identifier, in})); + query->if_exists = if_exists; + return query; +} + +PtrTo AlterTableClause::createClearIndex(bool if_exists, PtrTo identifier, PtrTo in) +{ + PtrTo query(new AlterTableClause(ClauseType::CLEAR_INDEX, {identifier, in})); + query->if_exists = if_exists; + return query; +} + +PtrTo AlterTableClause::createClearProjection(bool if_exists, PtrTo identifier, PtrTo in) +{ + PtrTo query(new AlterTableClause(ClauseType::CLEAR_PROJECTION, {identifier, in})); query->if_exists = if_exists; return query; } @@ -149,6 +172,14 @@ PtrTo AlterTableClause::createDropIndex(bool if_exists, PtrTo< return query; } +// static +PtrTo AlterTableClause::createDropProjection(bool if_exists, PtrTo identifier) +{ + PtrTo query(new AlterTableClause(ClauseType::DROP_PROJECTION, {identifier})); + query->if_exists = if_exists; + return query; +} + // static PtrTo AlterTableClause::createDropPartition(PtrTo clause) { @@ -161,6 +192,22 @@ PtrTo AlterTableClause::createFreezePartition(PtrTo(new AlterTableClause(ClauseType::FREEZE_PARTITION, {clause})); } +// static +PtrTo AlterTableClause::createMaterializeIndex(bool if_exists, PtrTo identifier, PtrTo in) +{ + PtrTo query(new AlterTableClause(ClauseType::MATERIALIZE_INDEX, {identifier, in})); + query->if_exists = if_exists; + return query; +} + +// static +PtrTo AlterTableClause::createMaterializeProjection(bool if_exists, PtrTo identifier, PtrTo in) +{ + PtrTo query(new AlterTableClause(ClauseType::MATERIALIZE_PROJECTION, {identifier, in})); + query->if_exists = if_exists; + return query; +} + // static PtrTo AlterTableClause::createModify(bool if_exists, PtrTo element) { @@ -256,6 +303,13 @@ ASTPtr AlterTableClause::convertToOld() const if (has(AFTER)) command->index = get(AFTER)->convertToOld(); break; + case ClauseType::ADD_PROJECTION: + command->type = ASTAlterCommand::ADD_PROJECTION; + command->if_not_exists = if_not_exists; + command->projection_decl = get(ELEMENT)->convertToOld(); + if (has(AFTER)) command->projection = get(AFTER)->convertToOld(); + break; + case ClauseType::ATTACH: command->type = ASTAlterCommand::ATTACH_PARTITION; command->partition = get(PARTITION)->convertToOld(); @@ -271,12 +325,30 @@ ASTPtr AlterTableClause::convertToOld() const } break; - case ClauseType::CLEAR: + case ClauseType::CLEAR_COLUMN: command->type = ASTAlterCommand::DROP_COLUMN; command->if_exists = if_exists; command->clear_column = true; command->detach = false; - command->column = get(COLUMN)->convertToOld(); + command->column = get(ELEMENT)->convertToOld(); + if (has(IN)) command->partition = get(IN)->convertToOld(); + break; + + case ClauseType::CLEAR_INDEX: + command->type = ASTAlterCommand::DROP_INDEX; + command->if_exists = if_exists; + command->clear_index = true; + command->detach = false; + command->index = get(ELEMENT)->convertToOld(); + if (has(IN)) command->partition = get(IN)->convertToOld(); + break; + + case ClauseType::CLEAR_PROJECTION: + command->type = ASTAlterCommand::DROP_PROJECTION; + command->if_exists = if_exists; + command->clear_projection = true; + command->detach = false; + command->projection = get(ELEMENT)->convertToOld(); if (has(IN)) command->partition = get(IN)->convertToOld(); break; @@ -315,14 +387,21 @@ ASTPtr AlterTableClause::convertToOld() const command->type = ASTAlterCommand::DROP_COLUMN; command->if_exists = if_exists; command->detach = false; - command->column = get(COLUMN)->convertToOld(); + command->column = get(ELEMENT)->convertToOld(); break; case ClauseType::DROP_INDEX: command->type = ASTAlterCommand::DROP_INDEX; command->if_exists = if_exists; command->detach = false; - command->index = get(COLUMN)->convertToOld(); + command->index = get(ELEMENT)->convertToOld(); + break; + + case ClauseType::DROP_PROJECTION: + command->type = ASTAlterCommand::DROP_PROJECTION; + command->if_exists = if_exists; + command->detach = false; + command->projection = get(ELEMENT)->convertToOld(); break; case ClauseType::DROP_PARTITION: @@ -340,6 +419,20 @@ ASTPtr AlterTableClause::convertToOld() const command->type = ASTAlterCommand::FREEZE_ALL; break; + case ClauseType::MATERIALIZE_INDEX: + command->type = ASTAlterCommand::MATERIALIZE_INDEX; + command->if_exists = if_exists; + command->index = get(ELEMENT)->convertToOld(); + if (has(IN)) command->partition = get(IN)->convertToOld(); + break; + + case ClauseType::MATERIALIZE_PROJECTION: + command->type = ASTAlterCommand::MATERIALIZE_PROJECTION; + command->if_exists = if_exists; + command->projection = get(ELEMENT)->convertToOld(); + if (has(IN)) command->partition = get(IN)->convertToOld(); + break; + case ClauseType::MODIFY: command->type = ASTAlterCommand::MODIFY_COLUMN; command->if_exists = if_exists; @@ -511,16 +604,34 @@ antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseAddIndex(ClickHouseParser:: return AlterTableClause::createAddIndex(!!ctx->IF(), visit(ctx->tableIndexDfnt()), after); } +antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseAddProjection(ClickHouseParser::AlterTableClauseAddProjectionContext * ctx) +{ + auto after = ctx->AFTER() ? visit(ctx->nestedIdentifier()).as>() : nullptr; + return AlterTableClause::createAddProjection(!!ctx->IF(), visit(ctx->tableProjectionDfnt()), after); +} + antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseAttach(ClickHouseParser::AlterTableClauseAttachContext *ctx) { auto from = ctx->tableIdentifier() ? visit(ctx->tableIdentifier()).as>() : nullptr; return AlterTableClause::createAttach(visit(ctx->partitionClause()), from); } -antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseClear(ClickHouseParser::AlterTableClauseClearContext * ctx) +antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseClearColumn(ClickHouseParser::AlterTableClauseClearColumnContext * ctx) { auto partition = ctx->partitionClause() ? visit(ctx->partitionClause()).as>() : nullptr; - return AlterTableClause::createClear(!!ctx->IF(), visit(ctx->nestedIdentifier()), partition); + return AlterTableClause::createClearColumn(!!ctx->IF(), visit(ctx->nestedIdentifier()), partition); +} + +antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseClearIndex(ClickHouseParser::AlterTableClauseClearIndexContext * ctx) +{ + auto partition = ctx->partitionClause() ? visit(ctx->partitionClause()).as>() : nullptr; + return AlterTableClause::createClearIndex(!!ctx->IF(), visit(ctx->nestedIdentifier()), partition); +} + +antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseClearProjection(ClickHouseParser::AlterTableClauseClearProjectionContext * ctx) +{ + auto partition = ctx->partitionClause() ? visit(ctx->partitionClause()).as>() : nullptr; + return AlterTableClause::createClearProjection(!!ctx->IF(), visit(ctx->nestedIdentifier()), partition); } antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseComment(ClickHouseParser::AlterTableClauseCommentContext * ctx) @@ -548,6 +659,11 @@ antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseDropIndex(ClickHouseParser: return AlterTableClause::createDropIndex(!!ctx->IF(), visit(ctx->nestedIdentifier())); } +antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseDropProjection(ClickHouseParser::AlterTableClauseDropProjectionContext * ctx) +{ + return AlterTableClause::createDropProjection(!!ctx->IF(), visit(ctx->nestedIdentifier())); +} + antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseDropPartition(ClickHouseParser::AlterTableClauseDropPartitionContext *ctx) { return AlterTableClause::createDropPartition(visit(ctx->partitionClause())); @@ -559,6 +675,18 @@ antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseFreezePartition(ClickHouseP return AlterTableClause::createFreezePartition(clause); } +antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseMaterializeIndex(ClickHouseParser::AlterTableClauseMaterializeIndexContext * ctx) +{ + auto partition = ctx->partitionClause() ? visit(ctx->partitionClause()).as>() : nullptr; + return AlterTableClause::createMaterializeIndex(!!ctx->IF(), visit(ctx->nestedIdentifier()), partition); +} + +antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseMaterializeProjection(ClickHouseParser::AlterTableClauseMaterializeProjectionContext * ctx) +{ + auto partition = ctx->partitionClause() ? visit(ctx->partitionClause()).as>() : nullptr; + return AlterTableClause::createMaterializeProjection(!!ctx->IF(), visit(ctx->nestedIdentifier()), partition); +} + antlrcpp::Any ParseTreeVisitor::visitAlterTableClauseModify(ClickHouseParser::AlterTableClauseModifyContext * ctx) { return AlterTableClause::createModify(!!ctx->IF(), visit(ctx->tableColumnDfnt())); diff --git a/src/Parsers/New/AST/AlterTableQuery.h b/src/Parsers/New/AST/AlterTableQuery.h index a1e5becedaf..7e7783c49ad 100644 --- a/src/Parsers/New/AST/AlterTableQuery.h +++ b/src/Parsers/New/AST/AlterTableQuery.h @@ -61,16 +61,22 @@ class AlterTableClause : public INode public: static PtrTo createAddColumn(bool if_not_exists, PtrTo element, PtrTo after); static PtrTo createAddIndex(bool if_not_exists, PtrTo element, PtrTo after); + static PtrTo createAddProjection(bool if_not_exists, PtrTo element, PtrTo after); static PtrTo createAttach(PtrTo clause, PtrTo from); - static PtrTo createClear(bool if_exists, PtrTo identifier, PtrTo in); + static PtrTo createClearColumn(bool if_exists, PtrTo identifier, PtrTo in); + static PtrTo createClearIndex(bool if_exists, PtrTo identifier, PtrTo in); + static PtrTo createClearProjection(bool if_exists, PtrTo identifier, PtrTo in); static PtrTo createCodec(bool if_exists, PtrTo identifier, PtrTo codec); static PtrTo createComment(bool if_exists, PtrTo identifier, PtrTo comment); static PtrTo createDelete(PtrTo expr); static PtrTo createDetach(PtrTo clause); static PtrTo createDropColumn(bool if_exists, PtrTo identifier); static PtrTo createDropIndex(bool if_exists, PtrTo identifier); + static PtrTo createDropProjection(bool if_exists, PtrTo identifier); static PtrTo createDropPartition(PtrTo clause); static PtrTo createFreezePartition(PtrTo clause); + static PtrTo createMaterializeIndex(bool if_exists, PtrTo identifier, PtrTo in); + static PtrTo createMaterializeProjection(bool if_exists, PtrTo identifier, PtrTo in); static PtrTo createModify(bool if_exists, PtrTo element); static PtrTo createMovePartitionToDisk(PtrTo clause, PtrTo literal); static PtrTo createMovePartitionToTable(PtrTo clause, PtrTo identifier); @@ -88,19 +94,19 @@ class AlterTableClause : public INode private: enum ChildIndex : UInt8 { - // ADD COLUMN or INDEX - ELEMENT = 0, // TableElementExpr + // ADD COLUMN, INDEX or PROJECTION + ELEMENT = 0, // TableElementExpr (COLUMN, CONSTRAINT, INDEX, PROJECTION) AFTER = 1, // Identifier (optional) // ATTACH/REPLACE PARTITION = 0, // PartitionClause FROM = 1, // TableIdentifier (optional) - // CLEAR - COLUMN = 0, // Identifier + // CLEAR COLUMN, INDEX or PROJECTION IN = 1, // PartitionClause - // CODEC + // CODEC, COMMENT and RENAME + COLUMN = 0, // Identifier CODEC = 1, // CodecExpr // COMMENT @@ -127,16 +133,22 @@ class AlterTableClause : public INode { ADD_COLUMN, ADD_INDEX, + ADD_PROJECTION, ATTACH, - CLEAR, + CLEAR_COLUMN, + CLEAR_INDEX, + CLEAR_PROJECTION, CODEC, COMMENT, DELETE, DETACH, DROP_COLUMN, DROP_INDEX, + DROP_PROJECTION, DROP_PARTITION, FREEZE_PARTITION, + MATERIALIZE_INDEX, + MATERIALIZE_PROJECTION, MODIFY, MOVE_PARTITION_TO_DISK, MOVE_PARTITION_TO_TABLE, diff --git a/src/Parsers/New/AST/CreateTableQuery.cpp b/src/Parsers/New/AST/CreateTableQuery.cpp index 77b43f525ec..bb2ec505138 100644 --- a/src/Parsers/New/AST/CreateTableQuery.cpp +++ b/src/Parsers/New/AST/CreateTableQuery.cpp @@ -48,6 +48,7 @@ ASTPtr TableSchemaClause::convertToOld() const auto column_list = std::make_shared(); auto constraint_list = std::make_shared(); auto index_list = std::make_shared(); + auto projection_list = std::make_shared(); for (const auto & element : get(ELEMENTS)->as()) { @@ -62,12 +63,16 @@ ASTPtr TableSchemaClause::convertToOld() const case TableElementExpr::ExprType::INDEX: index_list->children.push_back(element->convertToOld()); break; + case TableElementExpr::ExprType::PROJECTION: + projection_list->children.push_back(element->convertToOld()); + break; } } if (!column_list->children.empty()) columns->set(columns->columns, column_list); if (!constraint_list->children.empty()) columns->set(columns->constraints, constraint_list); if (!index_list->children.empty()) columns->set(columns->indices, index_list); + if (!projection_list->children.empty()) columns->set(columns->projections, projection_list); return columns; } diff --git a/src/Parsers/New/AST/SelectUnionQuery.cpp b/src/Parsers/New/AST/SelectUnionQuery.cpp index b3c6e2a923b..35eda09b473 100644 --- a/src/Parsers/New/AST/SelectUnionQuery.cpp +++ b/src/Parsers/New/AST/SelectUnionQuery.cpp @@ -1,7 +1,9 @@ #include #include +#include #include +#include #include #include #include @@ -102,6 +104,59 @@ ASTPtr SettingsClause::convertToOld() const return expr; } +// PROJECTION SELECT Caluse + +ProjectionSelectStmt::ProjectionSelectStmt(PtrTo expr_list) + : INode(MAX_INDEX) +{ + set(COLUMNS, expr_list); +} + +void ProjectionSelectStmt::setWithClause(PtrTo clause) +{ + set(WITH, clause); +} + +void ProjectionSelectStmt::setGroupByClause(PtrTo clause) +{ + set(GROUP_BY, clause); +} + +void ProjectionSelectStmt::setOrderByClause(PtrTo clause) +{ + set(ORDER_BY, clause); +} + +ASTPtr ProjectionSelectStmt::convertToOld() const +{ + auto old_select = std::make_shared(); + + old_select->setExpression(ASTProjectionSelectQuery::Expression::SELECT, get(COLUMNS)->convertToOld()); + + if (has(WITH)) old_select->setExpression(ASTProjectionSelectQuery::Expression::WITH, get(WITH)->convertToOld()); + if (has(GROUP_BY)) old_select->setExpression(ASTProjectionSelectQuery::Expression::GROUP_BY, get(GROUP_BY)->convertToOld()); + if (has(ORDER_BY)) + { + ASTPtr order_expression; + auto expr_list = get(ORDER_BY)->convertToOld(); + if (expr_list->children.size() == 1) + { + order_expression = expr_list->children.front(); + } + else + { + auto function_node = std::make_shared(); + function_node->name = "tuple"; + function_node->arguments = expr_list; + function_node->children.push_back(expr_list); + order_expression = function_node; + } + old_select->setExpression(ASTProjectionSelectQuery::Expression::ORDER_BY, std::move(order_expression)); + } + + return old_select; +} + // SELECT Statement SelectStmt::SelectStmt(bool distinct_, ModifierType type, bool totals, PtrTo expr_list) @@ -302,6 +357,11 @@ antlrcpp::Any ParseTreeVisitor::visitOrderByClause(ClickHouseParser::OrderByClau return std::make_shared(visit(ctx->orderExprList()).as>()); } +antlrcpp::Any ParseTreeVisitor::visitProjectionOrderByClause(ClickHouseParser::ProjectionOrderByClauseContext *ctx) +{ + return std::make_shared(visit(ctx->columnExprList()).as>()); +} + antlrcpp::Any ParseTreeVisitor::visitLimitByClause(ClickHouseParser::LimitByClauseContext *ctx) { return std::make_shared(visit(ctx->limitExpr()), visit(ctx->columnExprList())); @@ -317,6 +377,18 @@ antlrcpp::Any ParseTreeVisitor::visitSettingsClause(ClickHouseParser::SettingsCl return std::make_shared(visit(ctx->settingExprList()).as>()); } +antlrcpp::Any ParseTreeVisitor::visitProjectionSelectStmt(ClickHouseParser::ProjectionSelectStmtContext *ctx) +{ + PtrTo column_list = visit(ctx->columnExprList()); + auto select_stmt = std::make_shared(column_list); + + if (ctx->withClause()) select_stmt->setWithClause(visit(ctx->withClause())); + if (ctx->groupByClause()) select_stmt->setGroupByClause(visit(ctx->groupByClause())); + if (ctx->projectionOrderByClause()) select_stmt->setOrderByClause(visit(ctx->projectionOrderByClause())); + + return select_stmt; +} + antlrcpp::Any ParseTreeVisitor::visitSelectStmt(ClickHouseParser::SelectStmtContext *ctx) { SelectStmt::ModifierType type = SelectStmt::ModifierType::NONE; diff --git a/src/Parsers/New/AST/SelectUnionQuery.h b/src/Parsers/New/AST/SelectUnionQuery.h index 0d9b74dd192..587da271a78 100644 --- a/src/Parsers/New/AST/SelectUnionQuery.h +++ b/src/Parsers/New/AST/SelectUnionQuery.h @@ -97,6 +97,29 @@ class SettingsClause : public INode // Statement +class ProjectionSelectStmt : public INode +{ + public: + ProjectionSelectStmt(PtrTo expr_list); + + void setWithClause(PtrTo clause); + void setGroupByClause(PtrTo clause); + void setOrderByClause(PtrTo clause); + + ASTPtr convertToOld() const override; + + private: + enum ChildIndex : UInt8 + { + COLUMNS = 0, // ColumnExprList + WITH, // WithClause (optional) + GROUP_BY, // GroupByClause (optional) + ORDER_BY, // OrderByClause (optional) + + MAX_INDEX, + }; +}; + class SelectStmt : public INode { public: diff --git a/src/Parsers/New/AST/TableElementExpr.cpp b/src/Parsers/New/AST/TableElementExpr.cpp index d994ea7eef6..70855fee697 100644 --- a/src/Parsers/New/AST/TableElementExpr.cpp +++ b/src/Parsers/New/AST/TableElementExpr.cpp @@ -4,10 +4,12 @@ #include #include #include +#include #include #include #include #include +#include #include @@ -81,6 +83,13 @@ TableElementExpr::createIndex(PtrTo name, PtrTo expr, Pt return PtrTo(new TableElementExpr(ExprType::INDEX, {name, expr, type, granularity})); } +// static +PtrTo +TableElementExpr::createProjection(PtrTo name, PtrTo query) +{ + return PtrTo(new TableElementExpr(ExprType::PROJECTION, {name, query})); +} + TableElementExpr::TableElementExpr(ExprType type, PtrList exprs) : INode(exprs), expr_type(type) { } @@ -152,6 +161,15 @@ ASTPtr TableElementExpr::convertToOld() const expr->set(expr->type, get(INDEX_TYPE)->convertToOld()); expr->granularity = get(GRANULARITY)->as().value_or(0); // FIXME: throw exception instead of default. + return expr; + } + case ExprType::PROJECTION: + { + auto expr = std::make_shared(); + + expr->name = get(NAME)->getName(); + expr->set(expr->query, get(QUERY)->convertToOld()); + return expr; } } @@ -222,6 +240,11 @@ antlrcpp::Any ParseTreeVisitor::visitTableElementExprIndex(ClickHouseParser::Tab return visit(ctx->tableIndexDfnt()); } +antlrcpp::Any ParseTreeVisitor::visitTableElementExprProjection(ClickHouseParser::TableElementExprProjectionContext *ctx) +{ + return visit(ctx->tableProjectionDfnt()); +} + antlrcpp::Any ParseTreeVisitor::visitTableIndexDfnt(ClickHouseParser::TableIndexDfntContext *ctx) { return TableElementExpr::createIndex( @@ -231,4 +254,11 @@ antlrcpp::Any ParseTreeVisitor::visitTableIndexDfnt(ClickHouseParser::TableIndex Literal::createNumber(ctx->DECIMAL_LITERAL())); } +antlrcpp::Any ParseTreeVisitor::visitTableProjectionDfnt(ClickHouseParser::TableProjectionDfntContext *ctx) +{ + return TableElementExpr::createProjection( + visit(ctx->nestedIdentifier()), + visit(ctx->projectionSelectStmt())); +} + } diff --git a/src/Parsers/New/AST/TableElementExpr.h b/src/Parsers/New/AST/TableElementExpr.h index ca0a4f23bdd..18d1aa9c456 100644 --- a/src/Parsers/New/AST/TableElementExpr.h +++ b/src/Parsers/New/AST/TableElementExpr.h @@ -68,6 +68,7 @@ class TableElementExpr : public INode COLUMN, CONSTRAINT, INDEX, + PROJECTION, }; static PtrTo createColumn( @@ -83,6 +84,9 @@ class TableElementExpr : public INode static PtrTo createIndex(PtrTo name, PtrTo expr, PtrTo type, PtrTo granularity); + static PtrTo + createProjection(PtrTo name, PtrTo query); + auto getType() const { return expr_type; } ASTPtr convertToOld() const override; @@ -106,6 +110,9 @@ class TableElementExpr : public INode EXPR = 1, // ColumnExpr INDEX_TYPE = 2, // ColumnTypeExpr GRANULARITY = 3, // NumberLiteral + + // PROJECTION + QUERY = 1, // ColumnExpr }; const ExprType expr_type; diff --git a/src/Parsers/New/AST/fwd_decl.h b/src/Parsers/New/AST/fwd_decl.h index 555e9cb5727..4f9bde4bbbb 100644 --- a/src/Parsers/New/AST/fwd_decl.h +++ b/src/Parsers/New/AST/fwd_decl.h @@ -47,6 +47,7 @@ class PartitionClause; class Query; class RatioExpr; class TableSchemaClause; +class ProjectionSelectStmt; class SelectStmt; class SelectUnionQuery; class SettingExpr; @@ -81,6 +82,7 @@ using TTLExprList = List; using ClusterClause = SimpleClause; using DestinationClause = SimpleClause; using OrderByClause = SimpleClause; +using ProjectionOrderByClause = SimpleClause; using PrimaryKeyClause = SimpleClause; using TTLClause = SimpleClause; using UUIDClause = SimpleClause; diff --git a/src/Parsers/New/ClickHouseLexer.cpp b/src/Parsers/New/ClickHouseLexer.cpp index 832e5969426..f5db3a71dee 100644 --- a/src/Parsers/New/ClickHouseLexer.cpp +++ b/src/Parsers/New/ClickHouseLexer.cpp @@ -76,30 +76,30 @@ std::vector ClickHouseLexer::_ruleNames = { u8"INDEX", u8"INF", u8"INJECTIVE", u8"INNER", u8"INSERT", u8"INTERVAL", u8"INTO", u8"IS", u8"IS_OBJECT_ID", u8"JOIN", u8"KEY", u8"KILL", u8"LAST", u8"LAYOUT", u8"LEADING", u8"LEFT", u8"LIFETIME", u8"LIKE", u8"LIMIT", - u8"LIVE", u8"LOCAL", u8"LOGS", u8"MATERIALIZED", u8"MAX", u8"MERGES", - u8"MIN", u8"MINUTE", u8"MODIFY", u8"MONTH", u8"MOVE", u8"MUTATION", u8"NAN_SQL", - u8"NO", u8"NOT", u8"NULL_SQL", u8"NULLS", u8"OFFSET", u8"ON", u8"OPTIMIZE", - u8"OR", u8"ORDER", u8"OUTER", u8"OUTFILE", u8"PARTITION", u8"POPULATE", - u8"PREWHERE", u8"PRIMARY", u8"QUARTER", u8"RANGE", u8"RELOAD", u8"REMOVE", - u8"RENAME", u8"REPLACE", u8"REPLICA", u8"REPLICATED", u8"RIGHT", u8"ROLLUP", - u8"SAMPLE", u8"SECOND", u8"SELECT", u8"SEMI", u8"SENDS", u8"SET", u8"SETTINGS", - u8"SHOW", u8"SOURCE", u8"START", u8"STOP", u8"SUBSTRING", u8"SYNC", u8"SYNTAX", - u8"SYSTEM", u8"TABLE", u8"TABLES", u8"TEMPORARY", u8"TEST", u8"THEN", - u8"TIES", u8"TIMEOUT", u8"TIMESTAMP", u8"TO", u8"TOP", u8"TOTALS", u8"TRAILING", - u8"TRIM", u8"TRUNCATE", u8"TTL", u8"TYPE", u8"UNION", u8"UPDATE", u8"USE", - u8"USING", u8"UUID", u8"VALUES", u8"VIEW", u8"VOLUME", u8"WATCH", u8"WEEK", - u8"WHEN", u8"WHERE", u8"WITH", u8"YEAR", u8"JSON_FALSE", u8"JSON_TRUE", - u8"IDENTIFIER", u8"FLOATING_LITERAL", u8"OCTAL_LITERAL", u8"DECIMAL_LITERAL", - u8"HEXADECIMAL_LITERAL", u8"STRING_LITERAL", u8"A", u8"B", u8"C", u8"D", - u8"E", u8"F", u8"G", u8"H", u8"I", u8"J", u8"K", u8"L", u8"M", u8"N", - u8"O", u8"P", u8"Q", u8"R", u8"S", u8"T", u8"U", u8"V", u8"W", u8"X", - u8"Y", u8"Z", u8"LETTER", u8"OCT_DIGIT", u8"DEC_DIGIT", u8"HEX_DIGIT", - u8"ARROW", u8"ASTERISK", u8"BACKQUOTE", u8"BACKSLASH", u8"COLON", u8"COMMA", - u8"CONCAT", u8"DASH", u8"DOT", u8"EQ_DOUBLE", u8"EQ_SINGLE", u8"GE", u8"GT", - u8"LBRACE", u8"LBRACKET", u8"LE", u8"LPAREN", u8"LT", u8"NOT_EQ", u8"PERCENT", - u8"PLUS", u8"QUERY", u8"QUOTE_DOUBLE", u8"QUOTE_SINGLE", u8"RBRACE", u8"RBRACKET", - u8"RPAREN", u8"SEMICOLON", u8"SLASH", u8"UNDERSCORE", u8"MULTI_LINE_COMMENT", - u8"SINGLE_LINE_COMMENT", u8"WHITESPACE" + u8"LIVE", u8"LOCAL", u8"LOGS", u8"MATERIALIZED", u8"MATERIALIZE", u8"MAX", + u8"MERGES", u8"MIN", u8"MINUTE", u8"MODIFY", u8"MONTH", u8"MOVE", u8"MUTATION", + u8"NAN_SQL", u8"NO", u8"NOT", u8"NULL_SQL", u8"NULLS", u8"OFFSET", u8"ON", + u8"OPTIMIZE", u8"OR", u8"ORDER", u8"OUTER", u8"OUTFILE", u8"PARTITION", + u8"POPULATE", u8"PREWHERE", u8"PRIMARY", u8"PROJECTION", u8"QUARTER", + u8"RANGE", u8"RELOAD", u8"REMOVE", u8"RENAME", u8"REPLACE", u8"REPLICA", + u8"REPLICATED", u8"RIGHT", u8"ROLLUP", u8"SAMPLE", u8"SECOND", u8"SELECT", + u8"SEMI", u8"SENDS", u8"SET", u8"SETTINGS", u8"SHOW", u8"SOURCE", u8"START", + u8"STOP", u8"SUBSTRING", u8"SYNC", u8"SYNTAX", u8"SYSTEM", u8"TABLE", + u8"TABLES", u8"TEMPORARY", u8"TEST", u8"THEN", u8"TIES", u8"TIMEOUT", + u8"TIMESTAMP", u8"TO", u8"TOP", u8"TOTALS", u8"TRAILING", u8"TRIM", u8"TRUNCATE", + u8"TTL", u8"TYPE", u8"UNION", u8"UPDATE", u8"USE", u8"USING", u8"UUID", + u8"VALUES", u8"VIEW", u8"VOLUME", u8"WATCH", u8"WEEK", u8"WHEN", u8"WHERE", + u8"WITH", u8"YEAR", u8"JSON_FALSE", u8"JSON_TRUE", u8"IDENTIFIER", u8"FLOATING_LITERAL", + u8"OCTAL_LITERAL", u8"DECIMAL_LITERAL", u8"HEXADECIMAL_LITERAL", u8"STRING_LITERAL", + u8"A", u8"B", u8"C", u8"D", u8"E", u8"F", u8"G", u8"H", u8"I", u8"J", + u8"K", u8"L", u8"M", u8"N", u8"O", u8"P", u8"Q", u8"R", u8"S", u8"T", + u8"U", u8"V", u8"W", u8"X", u8"Y", u8"Z", u8"LETTER", u8"OCT_DIGIT", u8"DEC_DIGIT", + u8"HEX_DIGIT", u8"ARROW", u8"ASTERISK", u8"BACKQUOTE", u8"BACKSLASH", + u8"COLON", u8"COMMA", u8"CONCAT", u8"DASH", u8"DOT", u8"EQ_DOUBLE", u8"EQ_SINGLE", + u8"GE", u8"GT", u8"LBRACE", u8"LBRACKET", u8"LE", u8"LPAREN", u8"LT", + u8"NOT_EQ", u8"PERCENT", u8"PLUS", u8"QUERY", u8"QUOTE_DOUBLE", u8"QUOTE_SINGLE", + u8"RBRACE", u8"RBRACKET", u8"RPAREN", u8"SEMICOLON", u8"SLASH", u8"UNDERSCORE", + u8"MULTI_LINE_COMMENT", u8"SINGLE_LINE_COMMENT", u8"WHITESPACE" }; std::vector ClickHouseLexer::_channelNames = { @@ -120,12 +120,12 @@ std::vector ClickHouseLexer::_literalNames = { "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", - "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", u8"'false'", - u8"'true'", "", "", "", "", "", "", u8"'->'", u8"'*'", u8"'`'", u8"'\\'", - u8"':'", u8"','", u8"'||'", u8"'-'", u8"'.'", u8"'=='", u8"'='", u8"'>='", - u8"'>'", u8"'{'", u8"'['", u8"'<='", u8"'('", u8"'<'", "", u8"'%'", u8"'+'", - u8"'?'", u8"'\"'", u8"'''", u8"'}'", u8"']'", u8"')'", u8"';'", u8"'/'", - u8"'_'" + "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", + "", u8"'false'", u8"'true'", "", "", "", "", "", "", u8"'->'", u8"'*'", + u8"'`'", u8"'\\'", u8"':'", u8"','", u8"'||'", u8"'-'", u8"'.'", u8"'=='", + u8"'='", u8"'>='", u8"'>'", u8"'{'", u8"'['", u8"'<='", u8"'('", u8"'<'", + "", u8"'%'", u8"'+'", u8"'?'", u8"'\"'", u8"'''", u8"'}'", u8"']'", u8"')'", + u8"';'", u8"'/'", u8"'_'" }; std::vector ClickHouseLexer::_symbolicNames = { @@ -144,27 +144,27 @@ std::vector ClickHouseLexer::_symbolicNames = { u8"INDEX", u8"INF", u8"INJECTIVE", u8"INNER", u8"INSERT", u8"INTERVAL", u8"INTO", u8"IS", u8"IS_OBJECT_ID", u8"JOIN", u8"KEY", u8"KILL", u8"LAST", u8"LAYOUT", u8"LEADING", u8"LEFT", u8"LIFETIME", u8"LIKE", u8"LIMIT", - u8"LIVE", u8"LOCAL", u8"LOGS", u8"MATERIALIZED", u8"MAX", u8"MERGES", - u8"MIN", u8"MINUTE", u8"MODIFY", u8"MONTH", u8"MOVE", u8"MUTATION", u8"NAN_SQL", - u8"NO", u8"NOT", u8"NULL_SQL", u8"NULLS", u8"OFFSET", u8"ON", u8"OPTIMIZE", - u8"OR", u8"ORDER", u8"OUTER", u8"OUTFILE", u8"PARTITION", u8"POPULATE", - u8"PREWHERE", u8"PRIMARY", u8"QUARTER", u8"RANGE", u8"RELOAD", u8"REMOVE", - u8"RENAME", u8"REPLACE", u8"REPLICA", u8"REPLICATED", u8"RIGHT", u8"ROLLUP", - u8"SAMPLE", u8"SECOND", u8"SELECT", u8"SEMI", u8"SENDS", u8"SET", u8"SETTINGS", - u8"SHOW", u8"SOURCE", u8"START", u8"STOP", u8"SUBSTRING", u8"SYNC", u8"SYNTAX", - u8"SYSTEM", u8"TABLE", u8"TABLES", u8"TEMPORARY", u8"TEST", u8"THEN", - u8"TIES", u8"TIMEOUT", u8"TIMESTAMP", u8"TO", u8"TOP", u8"TOTALS", u8"TRAILING", - u8"TRIM", u8"TRUNCATE", u8"TTL", u8"TYPE", u8"UNION", u8"UPDATE", u8"USE", - u8"USING", u8"UUID", u8"VALUES", u8"VIEW", u8"VOLUME", u8"WATCH", u8"WEEK", - u8"WHEN", u8"WHERE", u8"WITH", u8"YEAR", u8"JSON_FALSE", u8"JSON_TRUE", - u8"IDENTIFIER", u8"FLOATING_LITERAL", u8"OCTAL_LITERAL", u8"DECIMAL_LITERAL", - u8"HEXADECIMAL_LITERAL", u8"STRING_LITERAL", u8"ARROW", u8"ASTERISK", - u8"BACKQUOTE", u8"BACKSLASH", u8"COLON", u8"COMMA", u8"CONCAT", u8"DASH", - u8"DOT", u8"EQ_DOUBLE", u8"EQ_SINGLE", u8"GE", u8"GT", u8"LBRACE", u8"LBRACKET", - u8"LE", u8"LPAREN", u8"LT", u8"NOT_EQ", u8"PERCENT", u8"PLUS", u8"QUERY", - u8"QUOTE_DOUBLE", u8"QUOTE_SINGLE", u8"RBRACE", u8"RBRACKET", u8"RPAREN", - u8"SEMICOLON", u8"SLASH", u8"UNDERSCORE", u8"MULTI_LINE_COMMENT", u8"SINGLE_LINE_COMMENT", - u8"WHITESPACE" + u8"LIVE", u8"LOCAL", u8"LOGS", u8"MATERIALIZED", u8"MATERIALIZE", u8"MAX", + u8"MERGES", u8"MIN", u8"MINUTE", u8"MODIFY", u8"MONTH", u8"MOVE", u8"MUTATION", + u8"NAN_SQL", u8"NO", u8"NOT", u8"NULL_SQL", u8"NULLS", u8"OFFSET", u8"ON", + u8"OPTIMIZE", u8"OR", u8"ORDER", u8"OUTER", u8"OUTFILE", u8"PARTITION", + u8"POPULATE", u8"PREWHERE", u8"PRIMARY", u8"PROJECTION", u8"QUARTER", + u8"RANGE", u8"RELOAD", u8"REMOVE", u8"RENAME", u8"REPLACE", u8"REPLICA", + u8"REPLICATED", u8"RIGHT", u8"ROLLUP", u8"SAMPLE", u8"SECOND", u8"SELECT", + u8"SEMI", u8"SENDS", u8"SET", u8"SETTINGS", u8"SHOW", u8"SOURCE", u8"START", + u8"STOP", u8"SUBSTRING", u8"SYNC", u8"SYNTAX", u8"SYSTEM", u8"TABLE", + u8"TABLES", u8"TEMPORARY", u8"TEST", u8"THEN", u8"TIES", u8"TIMEOUT", + u8"TIMESTAMP", u8"TO", u8"TOP", u8"TOTALS", u8"TRAILING", u8"TRIM", u8"TRUNCATE", + u8"TTL", u8"TYPE", u8"UNION", u8"UPDATE", u8"USE", u8"USING", u8"UUID", + u8"VALUES", u8"VIEW", u8"VOLUME", u8"WATCH", u8"WEEK", u8"WHEN", u8"WHERE", + u8"WITH", u8"YEAR", u8"JSON_FALSE", u8"JSON_TRUE", u8"IDENTIFIER", u8"FLOATING_LITERAL", + u8"OCTAL_LITERAL", u8"DECIMAL_LITERAL", u8"HEXADECIMAL_LITERAL", u8"STRING_LITERAL", + u8"ARROW", u8"ASTERISK", u8"BACKQUOTE", u8"BACKSLASH", u8"COLON", u8"COMMA", + u8"CONCAT", u8"DASH", u8"DOT", u8"EQ_DOUBLE", u8"EQ_SINGLE", u8"GE", u8"GT", + u8"LBRACE", u8"LBRACKET", u8"LE", u8"LPAREN", u8"LT", u8"NOT_EQ", u8"PERCENT", + u8"PLUS", u8"QUERY", u8"QUOTE_DOUBLE", u8"QUOTE_SINGLE", u8"RBRACE", u8"RBRACKET", + u8"RPAREN", u8"SEMICOLON", u8"SLASH", u8"UNDERSCORE", u8"MULTI_LINE_COMMENT", + u8"SINGLE_LINE_COMMENT", u8"WHITESPACE" }; dfa::Vocabulary ClickHouseLexer::_vocabulary(_literalNames, _symbolicNames); @@ -188,7 +188,7 @@ ClickHouseLexer::Initializer::Initializer() { _serializedATN = { 0x3, 0x608b, 0xa72a, 0x8133, 0xb9ed, 0x417c, 0x3be7, 0x7786, 0x5964, - 0x2, 0xdd, 0x7e2, 0x8, 0x1, 0x4, 0x2, 0x9, 0x2, 0x4, 0x3, 0x9, 0x3, + 0x2, 0xdf, 0x7fd, 0x8, 0x1, 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, @@ -264,1308 +264,1326 @@ ClickHouseLexer::Initializer::Initializer() { 0xee, 0x4, 0xef, 0x9, 0xef, 0x4, 0xf0, 0x9, 0xf0, 0x4, 0xf1, 0x9, 0xf1, 0x4, 0xf2, 0x9, 0xf2, 0x4, 0xf3, 0x9, 0xf3, 0x4, 0xf4, 0x9, 0xf4, 0x4, 0xf5, 0x9, 0xf5, 0x4, 0xf6, 0x9, 0xf6, 0x4, 0xf7, 0x9, 0xf7, 0x4, 0xf8, - 0x9, 0xf8, 0x4, 0xf9, 0x9, 0xf9, 0x4, 0xfa, 0x9, 0xfa, 0x3, 0x2, 0x3, - 0x2, 0x3, 0x2, 0x3, 0x2, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, - 0x3, 0x3, 0x3, 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x3, - 0x4, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x6, 0x3, 0x6, 0x3, - 0x6, 0x3, 0x6, 0x3, 0x6, 0x3, 0x6, 0x3, 0x7, 0x3, 0x7, 0x3, 0x7, 0x3, - 0x7, 0x3, 0x8, 0x3, 0x8, 0x3, 0x8, 0x3, 0x8, 0x3, 0x8, 0x3, 0x9, 0x3, - 0x9, 0x3, 0x9, 0x3, 0x9, 0x3, 0xa, 0x3, 0xa, 0x3, 0xa, 0x3, 0xa, 0x3, - 0xa, 0x3, 0xa, 0x3, 0xb, 0x3, 0xb, 0x3, 0xb, 0x3, 0xc, 0x3, 0xc, 0x3, - 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, - 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x234, 0xa, 0xc, - 0x3, 0xd, 0x3, 0xd, 0x3, 0xd, 0x3, 0xd, 0x3, 0xd, 0x3, 0xe, 0x3, 0xe, - 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, - 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, - 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, 0x3, 0x11, 0x3, - 0x11, 0x3, 0x11, 0x3, 0x11, 0x3, 0x11, 0x3, 0x12, 0x3, 0x12, 0x3, 0x12, - 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x14, 0x3, - 0x14, 0x3, 0x14, 0x3, 0x14, 0x3, 0x14, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, - 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x16, 0x3, 0x16, 0x3, 0x16, 0x3, - 0x16, 0x3, 0x16, 0x3, 0x16, 0x3, 0x17, 0x3, 0x17, 0x3, 0x17, 0x3, 0x17, - 0x3, 0x17, 0x3, 0x17, 0x3, 0x17, 0x3, 0x17, 0x3, 0x18, 0x3, 0x18, 0x3, - 0x18, 0x3, 0x18, 0x3, 0x18, 0x3, 0x18, 0x3, 0x19, 0x3, 0x19, 0x3, 0x19, - 0x3, 0x19, 0x3, 0x19, 0x3, 0x19, 0x3, 0x19, 0x3, 0x19, 0x3, 0x1a, 0x3, - 0x1a, 0x3, 0x1a, 0x3, 0x1a, 0x3, 0x1a, 0x3, 0x1a, 0x3, 0x1a, 0x3, 0x1b, - 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, - 0x1b, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, - 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1d, 0x3, - 0x1d, 0x3, 0x1d, 0x3, 0x1d, 0x3, 0x1d, 0x3, 0x1d, 0x3, 0x1d, 0x3, 0x1e, - 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1f, 0x3, - 0x1f, 0x3, 0x1f, 0x3, 0x1f, 0x3, 0x1f, 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, - 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, 0x3, + 0x9, 0xf8, 0x4, 0xf9, 0x9, 0xf9, 0x4, 0xfa, 0x9, 0xfa, 0x4, 0xfb, 0x9, + 0xfb, 0x4, 0xfc, 0x9, 0xfc, 0x3, 0x2, 0x3, 0x2, 0x3, 0x2, 0x3, 0x2, + 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x4, + 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x3, 0x5, 0x3, 0x5, + 0x3, 0x5, 0x3, 0x5, 0x3, 0x6, 0x3, 0x6, 0x3, 0x6, 0x3, 0x6, 0x3, 0x6, + 0x3, 0x6, 0x3, 0x7, 0x3, 0x7, 0x3, 0x7, 0x3, 0x7, 0x3, 0x8, 0x3, 0x8, + 0x3, 0x8, 0x3, 0x8, 0x3, 0x8, 0x3, 0x9, 0x3, 0x9, 0x3, 0x9, 0x3, 0x9, + 0x3, 0xa, 0x3, 0xa, 0x3, 0xa, 0x3, 0xa, 0x3, 0xa, 0x3, 0xa, 0x3, 0xb, + 0x3, 0xb, 0x3, 0xb, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, + 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, + 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x238, 0xa, 0xc, 0x3, 0xd, 0x3, 0xd, 0x3, + 0xd, 0x3, 0xd, 0x3, 0xd, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, + 0xe, 0x3, 0xe, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, 0xf, 0x3, + 0xf, 0x3, 0xf, 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, + 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, 0x3, 0x11, 0x3, 0x11, 0x3, 0x11, 0x3, + 0x11, 0x3, 0x11, 0x3, 0x12, 0x3, 0x12, 0x3, 0x12, 0x3, 0x13, 0x3, 0x13, + 0x3, 0x13, 0x3, 0x13, 0x3, 0x13, 0x3, 0x14, 0x3, 0x14, 0x3, 0x14, 0x3, + 0x14, 0x3, 0x14, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, 0x3, 0x15, + 0x3, 0x15, 0x3, 0x16, 0x3, 0x16, 0x3, 0x16, 0x3, 0x16, 0x3, 0x16, 0x3, + 0x16, 0x3, 0x17, 0x3, 0x17, 0x3, 0x17, 0x3, 0x17, 0x3, 0x17, 0x3, 0x17, + 0x3, 0x17, 0x3, 0x17, 0x3, 0x18, 0x3, 0x18, 0x3, 0x18, 0x3, 0x18, 0x3, + 0x18, 0x3, 0x18, 0x3, 0x19, 0x3, 0x19, 0x3, 0x19, 0x3, 0x19, 0x3, 0x19, + 0x3, 0x19, 0x3, 0x19, 0x3, 0x19, 0x3, 0x1a, 0x3, 0x1a, 0x3, 0x1a, 0x3, + 0x1a, 0x3, 0x1a, 0x3, 0x1a, 0x3, 0x1a, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, + 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1c, 0x3, + 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, + 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1c, 0x3, 0x1d, 0x3, 0x1d, 0x3, 0x1d, 0x3, + 0x1d, 0x3, 0x1d, 0x3, 0x1d, 0x3, 0x1d, 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1e, + 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1e, 0x3, 0x1f, 0x3, 0x1f, 0x3, 0x1f, 0x3, + 0x1f, 0x3, 0x1f, 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, + 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, 0x3, 0x20, 0x3, 0x21, 0x3, 0x21, 0x3, 0x21, 0x3, 0x21, 0x3, 0x21, 0x3, 0x21, 0x3, 0x21, 0x3, 0x21, 0x3, 0x21, - 0x3, 0x21, 0x3, 0x21, 0x3, 0x21, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, - 0x22, 0x3, 0x22, 0x3, 0x23, 0x3, 0x23, 0x3, 0x23, 0x3, 0x23, 0x3, 0x24, + 0x3, 0x21, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, + 0x23, 0x3, 0x23, 0x3, 0x23, 0x3, 0x23, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, - 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x25, 0x3, 0x25, - 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, - 0x26, 0x3, 0x26, 0x3, 0x26, 0x3, 0x26, 0x3, 0x26, 0x3, 0x26, 0x3, 0x27, - 0x3, 0x27, 0x3, 0x27, 0x3, 0x27, 0x3, 0x27, 0x3, 0x27, 0x3, 0x27, 0x3, - 0x28, 0x3, 0x28, 0x3, 0x28, 0x3, 0x28, 0x3, 0x28, 0x3, 0x29, 0x3, 0x29, + 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, + 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x26, 0x3, 0x26, 0x3, + 0x26, 0x3, 0x26, 0x3, 0x26, 0x3, 0x26, 0x3, 0x27, 0x3, 0x27, 0x3, 0x27, + 0x3, 0x27, 0x3, 0x27, 0x3, 0x27, 0x3, 0x27, 0x3, 0x28, 0x3, 0x28, 0x3, + 0x28, 0x3, 0x28, 0x3, 0x28, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, - 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, - 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2b, 0x3, - 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2c, + 0x29, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, + 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x3, + 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, - 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2d, + 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, - 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2e, - 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, - 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x30, 0x3, 0x30, + 0x2d, 0x3, 0x2d, 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2e, + 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, + 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, - 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, 0x31, 0x3, 0x31, 0x3, 0x31, - 0x3, 0x31, 0x3, 0x31, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, - 0x32, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x34, 0x3, 0x34, - 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, 0x3, 0x35, 0x3, - 0x35, 0x3, 0x35, 0x3, 0x35, 0x3, 0x35, 0x3, 0x35, 0x3, 0x35, 0x3, 0x36, - 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, - 0x37, 0x3, 0x37, 0x3, 0x37, 0x3, 0x37, 0x3, 0x37, 0x3, 0x37, 0x3, 0x37, - 0x3, 0x37, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, - 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x39, - 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, 0x3, - 0x39, 0x3, 0x3a, 0x3, 0x3a, 0x3, 0x3a, 0x3, 0x3a, 0x3, 0x3a, 0x3, 0x3a, - 0x3, 0x3a, 0x3, 0x3a, 0x3, 0x3b, 0x3, 0x3b, 0x3, 0x3b, 0x3, 0x3b, 0x3, - 0x3b, 0x3, 0x3b, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3c, - 0x3, 0x3c, 0x3, 0x3d, 0x3, 0x3d, 0x3, 0x3d, 0x3, 0x3d, 0x3, 0x3d, 0x3, - 0x3d, 0x3, 0x3e, 0x3, 0x3e, 0x3, 0x3e, 0x3, 0x3e, 0x3, 0x3f, 0x3, 0x3f, - 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x40, 0x3, - 0x40, 0x3, 0x40, 0x3, 0x40, 0x3, 0x40, 0x3, 0x40, 0x3, 0x40, 0x3, 0x41, - 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, - 0x3, 0x43, 0x3, 0x43, 0x3, 0x43, 0x3, 0x43, 0x3, 0x43, 0x3, 0x44, 0x3, - 0x44, 0x3, 0x44, 0x3, 0x44, 0x3, 0x44, 0x3, 0x44, 0x3, 0x44, 0x3, 0x45, + 0x30, 0x3, 0x30, 0x3, 0x31, 0x3, 0x31, 0x3, 0x31, 0x3, 0x31, 0x3, 0x31, + 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, 0x33, 0x3, + 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, + 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, 0x3, 0x35, 0x3, 0x35, 0x3, 0x35, 0x3, + 0x35, 0x3, 0x35, 0x3, 0x35, 0x3, 0x35, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, + 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, 0x37, 0x3, 0x37, 0x3, + 0x37, 0x3, 0x37, 0x3, 0x37, 0x3, 0x37, 0x3, 0x37, 0x3, 0x37, 0x3, 0x38, + 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, + 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, + 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, 0x3, 0x3a, 0x3, + 0x3a, 0x3, 0x3a, 0x3, 0x3a, 0x3, 0x3a, 0x3, 0x3a, 0x3, 0x3a, 0x3, 0x3a, + 0x3, 0x3b, 0x3, 0x3b, 0x3, 0x3b, 0x3, 0x3b, 0x3, 0x3b, 0x3, 0x3b, 0x3, + 0x3c, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3d, + 0x3, 0x3d, 0x3, 0x3d, 0x3, 0x3d, 0x3, 0x3d, 0x3, 0x3d, 0x3, 0x3e, 0x3, + 0x3e, 0x3, 0x3e, 0x3, 0x3e, 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x3f, + 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x40, 0x3, 0x40, 0x3, 0x40, 0x3, + 0x40, 0x3, 0x40, 0x3, 0x40, 0x3, 0x40, 0x3, 0x41, 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, 0x3, 0x43, 0x3, 0x43, + 0x3, 0x43, 0x3, 0x43, 0x3, 0x43, 0x3, 0x44, 0x3, 0x44, 0x3, 0x44, 0x3, + 0x44, 0x3, 0x44, 0x3, 0x44, 0x3, 0x44, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, - 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x46, 0x3, 0x46, - 0x3, 0x46, 0x3, 0x46, 0x3, 0x46, 0x3, 0x46, 0x3, 0x47, 0x3, 0x47, 0x3, - 0x47, 0x3, 0x47, 0x3, 0x47, 0x3, 0x47, 0x3, 0x47, 0x3, 0x48, 0x3, 0x48, + 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x46, 0x3, 0x46, 0x3, 0x46, 0x3, 0x46, + 0x3, 0x46, 0x3, 0x46, 0x3, 0x47, 0x3, 0x47, 0x3, 0x47, 0x3, 0x47, 0x3, + 0x47, 0x3, 0x47, 0x3, 0x47, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, - 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x49, 0x3, 0x49, - 0x3, 0x49, 0x3, 0x49, 0x3, 0x49, 0x3, 0x4a, 0x3, 0x4a, 0x3, 0x4a, 0x3, - 0x4b, 0x3, 0x4b, 0x3, 0x4b, 0x3, 0x4c, 0x3, 0x4c, 0x3, 0x4c, 0x3, 0x4c, - 0x3, 0x4c, 0x3, 0x4c, 0x3, 0x4d, 0x3, 0x4d, 0x3, 0x4d, 0x3, 0x4e, 0x3, - 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4f, 0x3, 0x4f, + 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x49, 0x3, 0x49, 0x3, 0x49, 0x3, 0x49, + 0x3, 0x49, 0x3, 0x4a, 0x3, 0x4a, 0x3, 0x4a, 0x3, 0x4b, 0x3, 0x4b, 0x3, + 0x4b, 0x3, 0x4c, 0x3, 0x4c, 0x3, 0x4c, 0x3, 0x4c, 0x3, 0x4c, 0x3, 0x4c, + 0x3, 0x4d, 0x3, 0x4d, 0x3, 0x4d, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, + 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, - 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x5, 0x4f, 0x410, - 0xa, 0x4f, 0x3, 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, - 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, 0x51, 0x3, 0x51, - 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x52, 0x3, 0x52, 0x3, - 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x53, 0x3, 0x53, - 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, - 0x53, 0x3, 0x54, 0x3, 0x54, 0x3, 0x54, 0x3, 0x54, 0x3, 0x54, 0x3, 0x55, - 0x3, 0x55, 0x3, 0x55, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, + 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x5, 0x4f, 0x414, 0xa, 0x4f, 0x3, 0x50, + 0x3, 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, + 0x50, 0x3, 0x50, 0x3, 0x50, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, + 0x3, 0x51, 0x3, 0x51, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, + 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, + 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x54, 0x3, + 0x54, 0x3, 0x54, 0x3, 0x54, 0x3, 0x54, 0x3, 0x55, 0x3, 0x55, 0x3, 0x55, + 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x57, 0x3, 0x57, 0x3, 0x57, 0x3, 0x57, 0x3, - 0x57, 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, 0x3, 0x59, 0x3, 0x59, - 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x5a, 0x3, 0x5a, 0x3, 0x5a, 0x3, - 0x5a, 0x3, 0x5a, 0x3, 0x5b, 0x3, 0x5b, 0x3, 0x5b, 0x3, 0x5b, 0x3, 0x5b, - 0x3, 0x5b, 0x3, 0x5b, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, - 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5d, 0x3, 0x5d, 0x3, 0x5d, - 0x3, 0x5d, 0x3, 0x5d, 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x3, - 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5f, 0x3, 0x5f, - 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x60, 0x3, 0x60, 0x3, 0x60, 0x3, - 0x60, 0x3, 0x60, 0x3, 0x60, 0x3, 0x61, 0x3, 0x61, 0x3, 0x61, 0x3, 0x61, - 0x3, 0x61, 0x3, 0x62, 0x3, 0x62, 0x3, 0x62, 0x3, 0x62, 0x3, 0x62, 0x3, - 0x62, 0x3, 0x63, 0x3, 0x63, 0x3, 0x63, 0x3, 0x63, 0x3, 0x63, 0x3, 0x64, + 0x3, 0x57, 0x3, 0x57, 0x3, 0x57, 0x3, 0x57, 0x3, 0x57, 0x3, 0x58, 0x3, + 0x58, 0x3, 0x58, 0x3, 0x58, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, + 0x3, 0x59, 0x3, 0x5a, 0x3, 0x5a, 0x3, 0x5a, 0x3, 0x5a, 0x3, 0x5a, 0x3, + 0x5b, 0x3, 0x5b, 0x3, 0x5b, 0x3, 0x5b, 0x3, 0x5b, 0x3, 0x5b, 0x3, 0x5b, + 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, + 0x5c, 0x3, 0x5c, 0x3, 0x5d, 0x3, 0x5d, 0x3, 0x5d, 0x3, 0x5d, 0x3, 0x5d, + 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x3, + 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, + 0x3, 0x5f, 0x3, 0x60, 0x3, 0x60, 0x3, 0x60, 0x3, 0x60, 0x3, 0x60, 0x3, + 0x60, 0x3, 0x61, 0x3, 0x61, 0x3, 0x61, 0x3, 0x61, 0x3, 0x61, 0x3, 0x62, + 0x3, 0x62, 0x3, 0x62, 0x3, 0x62, 0x3, 0x62, 0x3, 0x62, 0x3, 0x63, 0x3, + 0x63, 0x3, 0x63, 0x3, 0x63, 0x3, 0x63, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, - 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x65, - 0x3, 0x65, 0x3, 0x65, 0x3, 0x65, 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, 0x3, - 0x66, 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, 0x3, 0x67, 0x3, 0x67, 0x3, 0x67, - 0x3, 0x67, 0x3, 0x68, 0x3, 0x68, 0x3, 0x68, 0x3, 0x68, 0x3, 0x68, 0x3, - 0x68, 0x3, 0x68, 0x3, 0x69, 0x3, 0x69, 0x3, 0x69, 0x3, 0x69, 0x3, 0x69, - 0x3, 0x69, 0x3, 0x69, 0x3, 0x6a, 0x3, 0x6a, 0x3, 0x6a, 0x3, 0x6a, 0x3, - 0x6a, 0x3, 0x6a, 0x3, 0x6b, 0x3, 0x6b, 0x3, 0x6b, 0x3, 0x6b, 0x3, 0x6b, - 0x3, 0x6c, 0x3, 0x6c, 0x3, 0x6c, 0x3, 0x6c, 0x3, 0x6c, 0x3, 0x6c, 0x3, - 0x6c, 0x3, 0x6c, 0x3, 0x6c, 0x3, 0x6d, 0x3, 0x6d, 0x3, 0x6d, 0x3, 0x6d, - 0x3, 0x6e, 0x3, 0x6e, 0x3, 0x6e, 0x3, 0x6f, 0x3, 0x6f, 0x3, 0x6f, 0x3, - 0x6f, 0x3, 0x70, 0x3, 0x70, 0x3, 0x70, 0x3, 0x70, 0x3, 0x70, 0x3, 0x71, - 0x3, 0x71, 0x3, 0x71, 0x3, 0x71, 0x3, 0x71, 0x3, 0x71, 0x3, 0x72, 0x3, - 0x72, 0x3, 0x72, 0x3, 0x72, 0x3, 0x72, 0x3, 0x72, 0x3, 0x72, 0x3, 0x73, - 0x3, 0x73, 0x3, 0x73, 0x3, 0x74, 0x3, 0x74, 0x3, 0x74, 0x3, 0x74, 0x3, - 0x74, 0x3, 0x74, 0x3, 0x74, 0x3, 0x74, 0x3, 0x74, 0x3, 0x75, 0x3, 0x75, - 0x3, 0x75, 0x3, 0x76, 0x3, 0x76, 0x3, 0x76, 0x3, 0x76, 0x3, 0x76, 0x3, - 0x76, 0x3, 0x77, 0x3, 0x77, 0x3, 0x77, 0x3, 0x77, 0x3, 0x77, 0x3, 0x77, - 0x3, 0x78, 0x3, 0x78, 0x3, 0x78, 0x3, 0x78, 0x3, 0x78, 0x3, 0x78, 0x3, - 0x78, 0x3, 0x78, 0x3, 0x79, 0x3, 0x79, 0x3, 0x79, 0x3, 0x79, 0x3, 0x79, + 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x3, 0x65, 0x3, 0x65, 0x3, 0x65, + 0x3, 0x65, 0x3, 0x65, 0x3, 0x65, 0x3, 0x65, 0x3, 0x65, 0x3, 0x65, 0x3, + 0x65, 0x3, 0x65, 0x3, 0x65, 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, + 0x3, 0x67, 0x3, 0x67, 0x3, 0x67, 0x3, 0x67, 0x3, 0x67, 0x3, 0x67, 0x3, + 0x67, 0x3, 0x68, 0x3, 0x68, 0x3, 0x68, 0x3, 0x68, 0x3, 0x69, 0x3, 0x69, + 0x3, 0x69, 0x3, 0x69, 0x3, 0x69, 0x3, 0x69, 0x3, 0x69, 0x3, 0x6a, 0x3, + 0x6a, 0x3, 0x6a, 0x3, 0x6a, 0x3, 0x6a, 0x3, 0x6a, 0x3, 0x6a, 0x3, 0x6b, + 0x3, 0x6b, 0x3, 0x6b, 0x3, 0x6b, 0x3, 0x6b, 0x3, 0x6b, 0x3, 0x6c, 0x3, + 0x6c, 0x3, 0x6c, 0x3, 0x6c, 0x3, 0x6c, 0x3, 0x6d, 0x3, 0x6d, 0x3, 0x6d, + 0x3, 0x6d, 0x3, 0x6d, 0x3, 0x6d, 0x3, 0x6d, 0x3, 0x6d, 0x3, 0x6d, 0x3, + 0x6e, 0x3, 0x6e, 0x3, 0x6e, 0x3, 0x6e, 0x3, 0x6f, 0x3, 0x6f, 0x3, 0x6f, + 0x3, 0x70, 0x3, 0x70, 0x3, 0x70, 0x3, 0x70, 0x3, 0x71, 0x3, 0x71, 0x3, + 0x71, 0x3, 0x71, 0x3, 0x71, 0x3, 0x72, 0x3, 0x72, 0x3, 0x72, 0x3, 0x72, + 0x3, 0x72, 0x3, 0x72, 0x3, 0x73, 0x3, 0x73, 0x3, 0x73, 0x3, 0x73, 0x3, + 0x73, 0x3, 0x73, 0x3, 0x73, 0x3, 0x74, 0x3, 0x74, 0x3, 0x74, 0x3, 0x75, + 0x3, 0x75, 0x3, 0x75, 0x3, 0x75, 0x3, 0x75, 0x3, 0x75, 0x3, 0x75, 0x3, + 0x75, 0x3, 0x75, 0x3, 0x76, 0x3, 0x76, 0x3, 0x76, 0x3, 0x77, 0x3, 0x77, + 0x3, 0x77, 0x3, 0x77, 0x3, 0x77, 0x3, 0x77, 0x3, 0x78, 0x3, 0x78, 0x3, + 0x78, 0x3, 0x78, 0x3, 0x78, 0x3, 0x78, 0x3, 0x79, 0x3, 0x79, 0x3, 0x79, 0x3, 0x79, 0x3, 0x79, 0x3, 0x79, 0x3, 0x79, 0x3, 0x79, 0x3, 0x7a, 0x3, 0x7a, 0x3, 0x7a, 0x3, 0x7a, 0x3, 0x7a, 0x3, 0x7a, 0x3, 0x7a, 0x3, 0x7a, - 0x3, 0x7a, 0x3, 0x7b, 0x3, 0x7b, 0x3, 0x7b, 0x3, 0x7b, 0x3, 0x7b, 0x3, - 0x7b, 0x3, 0x7b, 0x3, 0x7b, 0x3, 0x7b, 0x3, 0x7c, 0x3, 0x7c, 0x3, 0x7c, - 0x3, 0x7c, 0x3, 0x7c, 0x3, 0x7c, 0x3, 0x7c, 0x3, 0x7c, 0x3, 0x7d, 0x3, - 0x7d, 0x3, 0x7d, 0x3, 0x7d, 0x3, 0x7d, 0x3, 0x7d, 0x3, 0x7d, 0x3, 0x7d, - 0x3, 0x7e, 0x3, 0x7e, 0x3, 0x7e, 0x3, 0x7e, 0x3, 0x7e, 0x3, 0x7e, 0x3, - 0x7f, 0x3, 0x7f, 0x3, 0x7f, 0x3, 0x7f, 0x3, 0x7f, 0x3, 0x7f, 0x3, 0x7f, - 0x3, 0x80, 0x3, 0x80, 0x3, 0x80, 0x3, 0x80, 0x3, 0x80, 0x3, 0x80, 0x3, - 0x80, 0x3, 0x81, 0x3, 0x81, 0x3, 0x81, 0x3, 0x81, 0x3, 0x81, 0x3, 0x81, - 0x3, 0x81, 0x3, 0x82, 0x3, 0x82, 0x3, 0x82, 0x3, 0x82, 0x3, 0x82, 0x3, - 0x82, 0x3, 0x82, 0x3, 0x82, 0x3, 0x83, 0x3, 0x83, 0x3, 0x83, 0x3, 0x83, - 0x3, 0x83, 0x3, 0x83, 0x3, 0x83, 0x3, 0x83, 0x3, 0x84, 0x3, 0x84, 0x3, - 0x84, 0x3, 0x84, 0x3, 0x84, 0x3, 0x84, 0x3, 0x84, 0x3, 0x84, 0x3, 0x84, - 0x3, 0x84, 0x3, 0x84, 0x3, 0x85, 0x3, 0x85, 0x3, 0x85, 0x3, 0x85, 0x3, - 0x85, 0x3, 0x85, 0x3, 0x86, 0x3, 0x86, 0x3, 0x86, 0x3, 0x86, 0x3, 0x86, - 0x3, 0x86, 0x3, 0x86, 0x3, 0x87, 0x3, 0x87, 0x3, 0x87, 0x3, 0x87, 0x3, - 0x87, 0x3, 0x87, 0x3, 0x87, 0x3, 0x88, 0x3, 0x88, 0x3, 0x88, 0x3, 0x88, - 0x3, 0x88, 0x3, 0x88, 0x3, 0x88, 0x3, 0x89, 0x3, 0x89, 0x3, 0x89, 0x3, - 0x89, 0x3, 0x89, 0x3, 0x89, 0x3, 0x89, 0x3, 0x8a, 0x3, 0x8a, 0x3, 0x8a, - 0x3, 0x8a, 0x3, 0x8a, 0x3, 0x8b, 0x3, 0x8b, 0x3, 0x8b, 0x3, 0x8b, 0x3, - 0x8b, 0x3, 0x8b, 0x3, 0x8c, 0x3, 0x8c, 0x3, 0x8c, 0x3, 0x8c, 0x3, 0x8d, - 0x3, 0x8d, 0x3, 0x8d, 0x3, 0x8d, 0x3, 0x8d, 0x3, 0x8d, 0x3, 0x8d, 0x3, - 0x8d, 0x3, 0x8d, 0x3, 0x8e, 0x3, 0x8e, 0x3, 0x8e, 0x3, 0x8e, 0x3, 0x8e, - 0x3, 0x8f, 0x3, 0x8f, 0x3, 0x8f, 0x3, 0x8f, 0x3, 0x8f, 0x3, 0x8f, 0x3, - 0x8f, 0x3, 0x90, 0x3, 0x90, 0x3, 0x90, 0x3, 0x90, 0x3, 0x90, 0x3, 0x90, - 0x3, 0x91, 0x3, 0x91, 0x3, 0x91, 0x3, 0x91, 0x3, 0x91, 0x3, 0x92, 0x3, - 0x92, 0x3, 0x92, 0x3, 0x92, 0x3, 0x92, 0x3, 0x92, 0x3, 0x92, 0x3, 0x92, - 0x3, 0x92, 0x3, 0x92, 0x3, 0x93, 0x3, 0x93, 0x3, 0x93, 0x3, 0x93, 0x3, - 0x93, 0x3, 0x94, 0x3, 0x94, 0x3, 0x94, 0x3, 0x94, 0x3, 0x94, 0x3, 0x94, - 0x3, 0x94, 0x3, 0x95, 0x3, 0x95, 0x3, 0x95, 0x3, 0x95, 0x3, 0x95, 0x3, - 0x95, 0x3, 0x95, 0x3, 0x96, 0x3, 0x96, 0x3, 0x96, 0x3, 0x96, 0x3, 0x96, - 0x3, 0x96, 0x3, 0x97, 0x3, 0x97, 0x3, 0x97, 0x3, 0x97, 0x3, 0x97, 0x3, - 0x97, 0x3, 0x97, 0x3, 0x98, 0x3, 0x98, 0x3, 0x98, 0x3, 0x98, 0x3, 0x98, - 0x3, 0x98, 0x3, 0x98, 0x3, 0x98, 0x3, 0x98, 0x3, 0x98, 0x3, 0x99, 0x3, - 0x99, 0x3, 0x99, 0x3, 0x99, 0x3, 0x99, 0x3, 0x9a, 0x3, 0x9a, 0x3, 0x9a, - 0x3, 0x9a, 0x3, 0x9a, 0x3, 0x9b, 0x3, 0x9b, 0x3, 0x9b, 0x3, 0x9b, 0x3, - 0x9b, 0x3, 0x9c, 0x3, 0x9c, 0x3, 0x9c, 0x3, 0x9c, 0x3, 0x9c, 0x3, 0x9c, - 0x3, 0x9c, 0x3, 0x9c, 0x3, 0x9d, 0x3, 0x9d, 0x3, 0x9d, 0x3, 0x9d, 0x3, - 0x9d, 0x3, 0x9d, 0x3, 0x9d, 0x3, 0x9d, 0x3, 0x9d, 0x3, 0x9d, 0x3, 0x9e, - 0x3, 0x9e, 0x3, 0x9e, 0x3, 0x9f, 0x3, 0x9f, 0x3, 0x9f, 0x3, 0x9f, 0x3, - 0xa0, 0x3, 0xa0, 0x3, 0xa0, 0x3, 0xa0, 0x3, 0xa0, 0x3, 0xa0, 0x3, 0xa0, - 0x3, 0xa1, 0x3, 0xa1, 0x3, 0xa1, 0x3, 0xa1, 0x3, 0xa1, 0x3, 0xa1, 0x3, - 0xa1, 0x3, 0xa1, 0x3, 0xa1, 0x3, 0xa2, 0x3, 0xa2, 0x3, 0xa2, 0x3, 0xa2, - 0x3, 0xa2, 0x3, 0xa3, 0x3, 0xa3, 0x3, 0xa3, 0x3, 0xa3, 0x3, 0xa3, 0x3, - 0xa3, 0x3, 0xa3, 0x3, 0xa3, 0x3, 0xa3, 0x3, 0xa4, 0x3, 0xa4, 0x3, 0xa4, - 0x3, 0xa4, 0x3, 0xa5, 0x3, 0xa5, 0x3, 0xa5, 0x3, 0xa5, 0x3, 0xa5, 0x3, - 0xa6, 0x3, 0xa6, 0x3, 0xa6, 0x3, 0xa6, 0x3, 0xa6, 0x3, 0xa6, 0x3, 0xa7, - 0x3, 0xa7, 0x3, 0xa7, 0x3, 0xa7, 0x3, 0xa7, 0x3, 0xa7, 0x3, 0xa7, 0x3, - 0xa8, 0x3, 0xa8, 0x3, 0xa8, 0x3, 0xa8, 0x3, 0xa9, 0x3, 0xa9, 0x3, 0xa9, - 0x3, 0xa9, 0x3, 0xa9, 0x3, 0xa9, 0x3, 0xaa, 0x3, 0xaa, 0x3, 0xaa, 0x3, - 0xaa, 0x3, 0xaa, 0x3, 0xab, 0x3, 0xab, 0x3, 0xab, 0x3, 0xab, 0x3, 0xab, - 0x3, 0xab, 0x3, 0xab, 0x3, 0xac, 0x3, 0xac, 0x3, 0xac, 0x3, 0xac, 0x3, - 0xac, 0x3, 0xad, 0x3, 0xad, 0x3, 0xad, 0x3, 0xad, 0x3, 0xad, 0x3, 0xad, - 0x3, 0xad, 0x3, 0xae, 0x3, 0xae, 0x3, 0xae, 0x3, 0xae, 0x3, 0xae, 0x3, - 0xae, 0x3, 0xaf, 0x3, 0xaf, 0x3, 0xaf, 0x3, 0xaf, 0x3, 0xaf, 0x3, 0xb0, - 0x3, 0xb0, 0x3, 0xb0, 0x3, 0xb0, 0x3, 0xb0, 0x3, 0xb1, 0x3, 0xb1, 0x3, - 0xb1, 0x3, 0xb1, 0x3, 0xb1, 0x3, 0xb1, 0x3, 0xb2, 0x3, 0xb2, 0x3, 0xb2, - 0x3, 0xb2, 0x3, 0xb2, 0x3, 0xb3, 0x3, 0xb3, 0x3, 0xb3, 0x3, 0xb3, 0x3, - 0xb3, 0x3, 0xb3, 0x3, 0xb3, 0x3, 0xb3, 0x3, 0xb3, 0x3, 0xb3, 0x5, 0xb3, - 0x69a, 0xa, 0xb3, 0x3, 0xb4, 0x3, 0xb4, 0x3, 0xb4, 0x3, 0xb4, 0x3, 0xb4, - 0x3, 0xb4, 0x3, 0xb5, 0x3, 0xb5, 0x3, 0xb5, 0x3, 0xb5, 0x3, 0xb5, 0x3, - 0xb6, 0x3, 0xb6, 0x5, 0xb6, 0x6a9, 0xa, 0xb6, 0x3, 0xb6, 0x3, 0xb6, - 0x3, 0xb6, 0x7, 0xb6, 0x6ae, 0xa, 0xb6, 0xc, 0xb6, 0xe, 0xb6, 0x6b1, - 0xb, 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x3, - 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x7, 0xb6, 0x6bb, 0xa, 0xb6, 0xc, 0xb6, - 0xe, 0xb6, 0x6be, 0xb, 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb6, - 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x7, - 0xb6, 0x6ca, 0xa, 0xb6, 0xc, 0xb6, 0xe, 0xb6, 0x6cd, 0xb, 0xb6, 0x3, - 0xb6, 0x3, 0xb6, 0x5, 0xb6, 0x6d1, 0xa, 0xb6, 0x3, 0xb7, 0x3, 0xb7, - 0x3, 0xb7, 0x7, 0xb7, 0x6d6, 0xa, 0xb7, 0xc, 0xb7, 0xe, 0xb7, 0x6d9, - 0xb, 0xb7, 0x3, 0xb7, 0x3, 0xb7, 0x5, 0xb7, 0x6dd, 0xa, 0xb7, 0x3, 0xb7, - 0x3, 0xb7, 0x5, 0xb7, 0x6e1, 0xa, 0xb7, 0x3, 0xb7, 0x6, 0xb7, 0x6e4, - 0xa, 0xb7, 0xd, 0xb7, 0xe, 0xb7, 0x6e5, 0x3, 0xb7, 0x3, 0xb7, 0x3, 0xb7, - 0x5, 0xb7, 0x6eb, 0xa, 0xb7, 0x3, 0xb7, 0x3, 0xb7, 0x5, 0xb7, 0x6ef, - 0xa, 0xb7, 0x3, 0xb7, 0x6, 0xb7, 0x6f2, 0xa, 0xb7, 0xd, 0xb7, 0xe, 0xb7, - 0x6f3, 0x3, 0xb7, 0x3, 0xb7, 0x3, 0xb7, 0x7, 0xb7, 0x6f9, 0xa, 0xb7, - 0xc, 0xb7, 0xe, 0xb7, 0x6fc, 0xb, 0xb7, 0x3, 0xb7, 0x3, 0xb7, 0x3, 0xb7, - 0x5, 0xb7, 0x701, 0xa, 0xb7, 0x3, 0xb7, 0x6, 0xb7, 0x704, 0xa, 0xb7, - 0xd, 0xb7, 0xe, 0xb7, 0x705, 0x3, 0xb7, 0x3, 0xb7, 0x3, 0xb7, 0x3, 0xb7, - 0x3, 0xb7, 0x5, 0xb7, 0x70d, 0xa, 0xb7, 0x3, 0xb7, 0x6, 0xb7, 0x710, - 0xa, 0xb7, 0xd, 0xb7, 0xe, 0xb7, 0x711, 0x3, 0xb7, 0x3, 0xb7, 0x3, 0xb7, - 0x3, 0xb7, 0x5, 0xb7, 0x718, 0xa, 0xb7, 0x3, 0xb7, 0x6, 0xb7, 0x71b, - 0xa, 0xb7, 0xd, 0xb7, 0xe, 0xb7, 0x71c, 0x5, 0xb7, 0x71f, 0xa, 0xb7, - 0x3, 0xb8, 0x3, 0xb8, 0x6, 0xb8, 0x723, 0xa, 0xb8, 0xd, 0xb8, 0xe, 0xb8, - 0x724, 0x3, 0xb9, 0x6, 0xb9, 0x728, 0xa, 0xb9, 0xd, 0xb9, 0xe, 0xb9, - 0x729, 0x3, 0xba, 0x3, 0xba, 0x3, 0xba, 0x6, 0xba, 0x72f, 0xa, 0xba, - 0xd, 0xba, 0xe, 0xba, 0x730, 0x3, 0xbb, 0x3, 0xbb, 0x3, 0xbb, 0x3, 0xbb, - 0x3, 0xbb, 0x3, 0xbb, 0x3, 0xbb, 0x3, 0xbb, 0x7, 0xbb, 0x73b, 0xa, 0xbb, - 0xc, 0xbb, 0xe, 0xbb, 0x73e, 0xb, 0xbb, 0x3, 0xbb, 0x3, 0xbb, 0x3, 0xbc, - 0x3, 0xbc, 0x3, 0xbd, 0x3, 0xbd, 0x3, 0xbe, 0x3, 0xbe, 0x3, 0xbf, 0x3, - 0xbf, 0x3, 0xc0, 0x3, 0xc0, 0x3, 0xc1, 0x3, 0xc1, 0x3, 0xc2, 0x3, 0xc2, - 0x3, 0xc3, 0x3, 0xc3, 0x3, 0xc4, 0x3, 0xc4, 0x3, 0xc5, 0x3, 0xc5, 0x3, - 0xc6, 0x3, 0xc6, 0x3, 0xc7, 0x3, 0xc7, 0x3, 0xc8, 0x3, 0xc8, 0x3, 0xc9, - 0x3, 0xc9, 0x3, 0xca, 0x3, 0xca, 0x3, 0xcb, 0x3, 0xcb, 0x3, 0xcc, 0x3, - 0xcc, 0x3, 0xcd, 0x3, 0xcd, 0x3, 0xce, 0x3, 0xce, 0x3, 0xcf, 0x3, 0xcf, - 0x3, 0xd0, 0x3, 0xd0, 0x3, 0xd1, 0x3, 0xd1, 0x3, 0xd2, 0x3, 0xd2, 0x3, - 0xd3, 0x3, 0xd3, 0x3, 0xd4, 0x3, 0xd4, 0x3, 0xd5, 0x3, 0xd5, 0x3, 0xd6, - 0x3, 0xd6, 0x3, 0xd7, 0x3, 0xd7, 0x3, 0xd8, 0x3, 0xd8, 0x3, 0xd9, 0x3, - 0xd9, 0x3, 0xda, 0x3, 0xda, 0x3, 0xda, 0x3, 0xdb, 0x3, 0xdb, 0x3, 0xdc, - 0x3, 0xdc, 0x3, 0xdd, 0x3, 0xdd, 0x3, 0xde, 0x3, 0xde, 0x3, 0xdf, 0x3, - 0xdf, 0x3, 0xe0, 0x3, 0xe0, 0x3, 0xe0, 0x3, 0xe1, 0x3, 0xe1, 0x3, 0xe2, - 0x3, 0xe2, 0x3, 0xe3, 0x3, 0xe3, 0x3, 0xe3, 0x3, 0xe4, 0x3, 0xe4, 0x3, - 0xe5, 0x3, 0xe5, 0x3, 0xe5, 0x3, 0xe6, 0x3, 0xe6, 0x3, 0xe7, 0x3, 0xe7, - 0x3, 0xe8, 0x3, 0xe8, 0x3, 0xe9, 0x3, 0xe9, 0x3, 0xe9, 0x3, 0xea, 0x3, - 0xea, 0x3, 0xeb, 0x3, 0xeb, 0x3, 0xec, 0x3, 0xec, 0x3, 0xec, 0x3, 0xec, - 0x5, 0xec, 0x7ab, 0xa, 0xec, 0x3, 0xed, 0x3, 0xed, 0x3, 0xee, 0x3, 0xee, - 0x3, 0xef, 0x3, 0xef, 0x3, 0xf0, 0x3, 0xf0, 0x3, 0xf1, 0x3, 0xf1, 0x3, - 0xf2, 0x3, 0xf2, 0x3, 0xf3, 0x3, 0xf3, 0x3, 0xf4, 0x3, 0xf4, 0x3, 0xf5, - 0x3, 0xf5, 0x3, 0xf6, 0x3, 0xf6, 0x3, 0xf7, 0x3, 0xf7, 0x3, 0xf8, 0x3, - 0xf8, 0x3, 0xf8, 0x3, 0xf8, 0x7, 0xf8, 0x7c7, 0xa, 0xf8, 0xc, 0xf8, - 0xe, 0xf8, 0x7ca, 0xb, 0xf8, 0x3, 0xf8, 0x3, 0xf8, 0x3, 0xf8, 0x3, 0xf8, - 0x3, 0xf8, 0x3, 0xf9, 0x3, 0xf9, 0x3, 0xf9, 0x3, 0xf9, 0x7, 0xf9, 0x7d5, - 0xa, 0xf9, 0xc, 0xf9, 0xe, 0xf9, 0x7d8, 0xb, 0xf9, 0x3, 0xf9, 0x5, 0xf9, - 0x7db, 0xa, 0xf9, 0x3, 0xf9, 0x3, 0xf9, 0x3, 0xfa, 0x3, 0xfa, 0x3, 0xfa, - 0x3, 0xfa, 0x3, 0x7c8, 0x2, 0xfb, 0x3, 0x3, 0x5, 0x4, 0x7, 0x5, 0x9, - 0x6, 0xb, 0x7, 0xd, 0x8, 0xf, 0x9, 0x11, 0xa, 0x13, 0xb, 0x15, 0xc, - 0x17, 0xd, 0x19, 0xe, 0x1b, 0xf, 0x1d, 0x10, 0x1f, 0x11, 0x21, 0x12, - 0x23, 0x13, 0x25, 0x14, 0x27, 0x15, 0x29, 0x16, 0x2b, 0x17, 0x2d, 0x18, - 0x2f, 0x19, 0x31, 0x1a, 0x33, 0x1b, 0x35, 0x1c, 0x37, 0x1d, 0x39, 0x1e, - 0x3b, 0x1f, 0x3d, 0x20, 0x3f, 0x21, 0x41, 0x22, 0x43, 0x23, 0x45, 0x24, - 0x47, 0x25, 0x49, 0x26, 0x4b, 0x27, 0x4d, 0x28, 0x4f, 0x29, 0x51, 0x2a, - 0x53, 0x2b, 0x55, 0x2c, 0x57, 0x2d, 0x59, 0x2e, 0x5b, 0x2f, 0x5d, 0x30, - 0x5f, 0x31, 0x61, 0x32, 0x63, 0x33, 0x65, 0x34, 0x67, 0x35, 0x69, 0x36, - 0x6b, 0x37, 0x6d, 0x38, 0x6f, 0x39, 0x71, 0x3a, 0x73, 0x3b, 0x75, 0x3c, - 0x77, 0x3d, 0x79, 0x3e, 0x7b, 0x3f, 0x7d, 0x40, 0x7f, 0x41, 0x81, 0x42, - 0x83, 0x43, 0x85, 0x44, 0x87, 0x45, 0x89, 0x46, 0x8b, 0x47, 0x8d, 0x48, - 0x8f, 0x49, 0x91, 0x4a, 0x93, 0x4b, 0x95, 0x4c, 0x97, 0x4d, 0x99, 0x4e, - 0x9b, 0x4f, 0x9d, 0x50, 0x9f, 0x51, 0xa1, 0x52, 0xa3, 0x53, 0xa5, 0x54, - 0xa7, 0x55, 0xa9, 0x56, 0xab, 0x57, 0xad, 0x58, 0xaf, 0x59, 0xb1, 0x5a, - 0xb3, 0x5b, 0xb5, 0x5c, 0xb7, 0x5d, 0xb9, 0x5e, 0xbb, 0x5f, 0xbd, 0x60, - 0xbf, 0x61, 0xc1, 0x62, 0xc3, 0x63, 0xc5, 0x64, 0xc7, 0x65, 0xc9, 0x66, - 0xcb, 0x67, 0xcd, 0x68, 0xcf, 0x69, 0xd1, 0x6a, 0xd3, 0x6b, 0xd5, 0x6c, - 0xd7, 0x6d, 0xd9, 0x6e, 0xdb, 0x6f, 0xdd, 0x70, 0xdf, 0x71, 0xe1, 0x72, - 0xe3, 0x73, 0xe5, 0x74, 0xe7, 0x75, 0xe9, 0x76, 0xeb, 0x77, 0xed, 0x78, - 0xef, 0x79, 0xf1, 0x7a, 0xf3, 0x7b, 0xf5, 0x7c, 0xf7, 0x7d, 0xf9, 0x7e, - 0xfb, 0x7f, 0xfd, 0x80, 0xff, 0x81, 0x101, 0x82, 0x103, 0x83, 0x105, - 0x84, 0x107, 0x85, 0x109, 0x86, 0x10b, 0x87, 0x10d, 0x88, 0x10f, 0x89, - 0x111, 0x8a, 0x113, 0x8b, 0x115, 0x8c, 0x117, 0x8d, 0x119, 0x8e, 0x11b, - 0x8f, 0x11d, 0x90, 0x11f, 0x91, 0x121, 0x92, 0x123, 0x93, 0x125, 0x94, - 0x127, 0x95, 0x129, 0x96, 0x12b, 0x97, 0x12d, 0x98, 0x12f, 0x99, 0x131, - 0x9a, 0x133, 0x9b, 0x135, 0x9c, 0x137, 0x9d, 0x139, 0x9e, 0x13b, 0x9f, - 0x13d, 0xa0, 0x13f, 0xa1, 0x141, 0xa2, 0x143, 0xa3, 0x145, 0xa4, 0x147, - 0xa5, 0x149, 0xa6, 0x14b, 0xa7, 0x14d, 0xa8, 0x14f, 0xa9, 0x151, 0xaa, - 0x153, 0xab, 0x155, 0xac, 0x157, 0xad, 0x159, 0xae, 0x15b, 0xaf, 0x15d, - 0xb0, 0x15f, 0xb1, 0x161, 0xb2, 0x163, 0xb3, 0x165, 0xb4, 0x167, 0xb5, - 0x169, 0xb6, 0x16b, 0xb7, 0x16d, 0xb8, 0x16f, 0xb9, 0x171, 0xba, 0x173, - 0xbb, 0x175, 0xbc, 0x177, 0x2, 0x179, 0x2, 0x17b, 0x2, 0x17d, 0x2, 0x17f, - 0x2, 0x181, 0x2, 0x183, 0x2, 0x185, 0x2, 0x187, 0x2, 0x189, 0x2, 0x18b, - 0x2, 0x18d, 0x2, 0x18f, 0x2, 0x191, 0x2, 0x193, 0x2, 0x195, 0x2, 0x197, - 0x2, 0x199, 0x2, 0x19b, 0x2, 0x19d, 0x2, 0x19f, 0x2, 0x1a1, 0x2, 0x1a3, - 0x2, 0x1a5, 0x2, 0x1a7, 0x2, 0x1a9, 0x2, 0x1ab, 0x2, 0x1ad, 0x2, 0x1af, - 0x2, 0x1b1, 0x2, 0x1b3, 0xbd, 0x1b5, 0xbe, 0x1b7, 0xbf, 0x1b9, 0xc0, - 0x1bb, 0xc1, 0x1bd, 0xc2, 0x1bf, 0xc3, 0x1c1, 0xc4, 0x1c3, 0xc5, 0x1c5, - 0xc6, 0x1c7, 0xc7, 0x1c9, 0xc8, 0x1cb, 0xc9, 0x1cd, 0xca, 0x1cf, 0xcb, - 0x1d1, 0xcc, 0x1d3, 0xcd, 0x1d5, 0xce, 0x1d7, 0xcf, 0x1d9, 0xd0, 0x1db, - 0xd1, 0x1dd, 0xd2, 0x1df, 0xd3, 0x1e1, 0xd4, 0x1e3, 0xd5, 0x1e5, 0xd6, - 0x1e7, 0xd7, 0x1e9, 0xd8, 0x1eb, 0xd9, 0x1ed, 0xda, 0x1ef, 0xdb, 0x1f1, - 0xdc, 0x1f3, 0xdd, 0x3, 0x2, 0x26, 0x4, 0x2, 0x5e, 0x5e, 0x62, 0x62, - 0x4, 0x2, 0x24, 0x24, 0x5e, 0x5e, 0x4, 0x2, 0x29, 0x29, 0x5e, 0x5e, - 0x4, 0x2, 0x43, 0x43, 0x63, 0x63, 0x4, 0x2, 0x44, 0x44, 0x64, 0x64, - 0x4, 0x2, 0x45, 0x45, 0x65, 0x65, 0x4, 0x2, 0x46, 0x46, 0x66, 0x66, - 0x4, 0x2, 0x47, 0x47, 0x67, 0x67, 0x4, 0x2, 0x48, 0x48, 0x68, 0x68, - 0x4, 0x2, 0x49, 0x49, 0x69, 0x69, 0x4, 0x2, 0x4a, 0x4a, 0x6a, 0x6a, - 0x4, 0x2, 0x4b, 0x4b, 0x6b, 0x6b, 0x4, 0x2, 0x4c, 0x4c, 0x6c, 0x6c, - 0x4, 0x2, 0x4d, 0x4d, 0x6d, 0x6d, 0x4, 0x2, 0x4e, 0x4e, 0x6e, 0x6e, - 0x4, 0x2, 0x4f, 0x4f, 0x6f, 0x6f, 0x4, 0x2, 0x50, 0x50, 0x70, 0x70, - 0x4, 0x2, 0x51, 0x51, 0x71, 0x71, 0x4, 0x2, 0x52, 0x52, 0x72, 0x72, - 0x4, 0x2, 0x53, 0x53, 0x73, 0x73, 0x4, 0x2, 0x54, 0x54, 0x74, 0x74, - 0x4, 0x2, 0x55, 0x55, 0x75, 0x75, 0x4, 0x2, 0x56, 0x56, 0x76, 0x76, - 0x4, 0x2, 0x57, 0x57, 0x77, 0x77, 0x4, 0x2, 0x58, 0x58, 0x78, 0x78, - 0x4, 0x2, 0x59, 0x59, 0x79, 0x79, 0x4, 0x2, 0x5a, 0x5a, 0x7a, 0x7a, - 0x4, 0x2, 0x5b, 0x5b, 0x7b, 0x7b, 0x4, 0x2, 0x5c, 0x5c, 0x7c, 0x7c, - 0x4, 0x2, 0x43, 0x5c, 0x63, 0x7c, 0x3, 0x2, 0x32, 0x39, 0x3, 0x2, 0x32, - 0x3b, 0x5, 0x2, 0x32, 0x3b, 0x43, 0x48, 0x63, 0x68, 0x4, 0x2, 0xc, 0xc, - 0xf, 0xf, 0x4, 0x3, 0xc, 0xc, 0xf, 0xf, 0x4, 0x2, 0xb, 0xf, 0x22, 0x22, - 0x2, 0x7f2, 0x2, 0x3, 0x3, 0x2, 0x2, 0x2, 0x2, 0x5, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x7, 0x3, 0x2, 0x2, 0x2, 0x2, 0x9, 0x3, 0x2, 0x2, 0x2, 0x2, 0xb, - 0x3, 0x2, 0x2, 0x2, 0x2, 0xd, 0x3, 0x2, 0x2, 0x2, 0x2, 0xf, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x11, 0x3, 0x2, 0x2, 0x2, 0x2, 0x13, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x15, 0x3, 0x2, 0x2, 0x2, 0x2, 0x17, 0x3, 0x2, 0x2, 0x2, 0x2, 0x19, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x1b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1d, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x1f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x21, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x23, 0x3, 0x2, 0x2, 0x2, 0x2, 0x25, 0x3, 0x2, 0x2, 0x2, 0x2, 0x27, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x29, 0x3, 0x2, 0x2, 0x2, 0x2, 0x2b, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x2d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x2f, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x31, 0x3, 0x2, 0x2, 0x2, 0x2, 0x33, 0x3, 0x2, 0x2, 0x2, 0x2, 0x35, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x37, 0x3, 0x2, 0x2, 0x2, 0x2, 0x39, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x3b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x3d, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x3f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x41, 0x3, 0x2, 0x2, 0x2, 0x2, 0x43, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x45, 0x3, 0x2, 0x2, 0x2, 0x2, 0x47, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x49, 0x3, 0x2, 0x2, 0x2, 0x2, 0x4b, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x4d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x4f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x51, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x53, 0x3, 0x2, 0x2, 0x2, 0x2, 0x55, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x57, 0x3, 0x2, 0x2, 0x2, 0x2, 0x59, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x5b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x5d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x5f, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x61, 0x3, 0x2, 0x2, 0x2, 0x2, 0x63, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x65, 0x3, 0x2, 0x2, 0x2, 0x2, 0x67, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x69, 0x3, 0x2, 0x2, 0x2, 0x2, 0x6b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x6d, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x6f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x71, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x73, 0x3, 0x2, 0x2, 0x2, 0x2, 0x75, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x77, 0x3, 0x2, 0x2, 0x2, 0x2, 0x79, 0x3, 0x2, 0x2, 0x2, 0x2, 0x7b, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x7d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x7f, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x81, 0x3, 0x2, 0x2, 0x2, 0x2, 0x83, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x85, 0x3, 0x2, 0x2, 0x2, 0x2, 0x87, 0x3, 0x2, 0x2, 0x2, 0x2, 0x89, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x8b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x8d, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x8f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x91, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x93, 0x3, 0x2, 0x2, 0x2, 0x2, 0x95, 0x3, 0x2, 0x2, 0x2, 0x2, 0x97, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x99, 0x3, 0x2, 0x2, 0x2, 0x2, 0x9b, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x9d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x9f, 0x3, 0x2, 0x2, 0x2, - 0x2, 0xa1, 0x3, 0x2, 0x2, 0x2, 0x2, 0xa3, 0x3, 0x2, 0x2, 0x2, 0x2, 0xa5, - 0x3, 0x2, 0x2, 0x2, 0x2, 0xa7, 0x3, 0x2, 0x2, 0x2, 0x2, 0xa9, 0x3, 0x2, - 0x2, 0x2, 0x2, 0xab, 0x3, 0x2, 0x2, 0x2, 0x2, 0xad, 0x3, 0x2, 0x2, 0x2, - 0x2, 0xaf, 0x3, 0x2, 0x2, 0x2, 0x2, 0xb1, 0x3, 0x2, 0x2, 0x2, 0x2, 0xb3, - 0x3, 0x2, 0x2, 0x2, 0x2, 0xb5, 0x3, 0x2, 0x2, 0x2, 0x2, 0xb7, 0x3, 0x2, - 0x2, 0x2, 0x2, 0xb9, 0x3, 0x2, 0x2, 0x2, 0x2, 0xbb, 0x3, 0x2, 0x2, 0x2, - 0x2, 0xbd, 0x3, 0x2, 0x2, 0x2, 0x2, 0xbf, 0x3, 0x2, 0x2, 0x2, 0x2, 0xc1, - 0x3, 0x2, 0x2, 0x2, 0x2, 0xc3, 0x3, 0x2, 0x2, 0x2, 0x2, 0xc5, 0x3, 0x2, - 0x2, 0x2, 0x2, 0xc7, 0x3, 0x2, 0x2, 0x2, 0x2, 0xc9, 0x3, 0x2, 0x2, 0x2, - 0x2, 0xcb, 0x3, 0x2, 0x2, 0x2, 0x2, 0xcd, 0x3, 0x2, 0x2, 0x2, 0x2, 0xcf, - 0x3, 0x2, 0x2, 0x2, 0x2, 0xd1, 0x3, 0x2, 0x2, 0x2, 0x2, 0xd3, 0x3, 0x2, - 0x2, 0x2, 0x2, 0xd5, 0x3, 0x2, 0x2, 0x2, 0x2, 0xd7, 0x3, 0x2, 0x2, 0x2, - 0x2, 0xd9, 0x3, 0x2, 0x2, 0x2, 0x2, 0xdb, 0x3, 0x2, 0x2, 0x2, 0x2, 0xdd, - 0x3, 0x2, 0x2, 0x2, 0x2, 0xdf, 0x3, 0x2, 0x2, 0x2, 0x2, 0xe1, 0x3, 0x2, - 0x2, 0x2, 0x2, 0xe3, 0x3, 0x2, 0x2, 0x2, 0x2, 0xe5, 0x3, 0x2, 0x2, 0x2, - 0x2, 0xe7, 0x3, 0x2, 0x2, 0x2, 0x2, 0xe9, 0x3, 0x2, 0x2, 0x2, 0x2, 0xeb, - 0x3, 0x2, 0x2, 0x2, 0x2, 0xed, 0x3, 0x2, 0x2, 0x2, 0x2, 0xef, 0x3, 0x2, - 0x2, 0x2, 0x2, 0xf1, 0x3, 0x2, 0x2, 0x2, 0x2, 0xf3, 0x3, 0x2, 0x2, 0x2, - 0x2, 0xf5, 0x3, 0x2, 0x2, 0x2, 0x2, 0xf7, 0x3, 0x2, 0x2, 0x2, 0x2, 0xf9, - 0x3, 0x2, 0x2, 0x2, 0x2, 0xfb, 0x3, 0x2, 0x2, 0x2, 0x2, 0xfd, 0x3, 0x2, - 0x2, 0x2, 0x2, 0xff, 0x3, 0x2, 0x2, 0x2, 0x2, 0x101, 0x3, 0x2, 0x2, - 0x2, 0x2, 0x103, 0x3, 0x2, 0x2, 0x2, 0x2, 0x105, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x107, 0x3, 0x2, 0x2, 0x2, 0x2, 0x109, 0x3, 0x2, 0x2, 0x2, 0x2, - 0x10b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x10d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x10f, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x111, 0x3, 0x2, 0x2, 0x2, 0x2, 0x113, 0x3, - 0x2, 0x2, 0x2, 0x2, 0x115, 0x3, 0x2, 0x2, 0x2, 0x2, 0x117, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x119, 0x3, 0x2, 0x2, 0x2, 0x2, 0x11b, 0x3, 0x2, 0x2, - 0x2, 0x2, 0x11d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x11f, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x121, 0x3, 0x2, 0x2, 0x2, 0x2, 0x123, 0x3, 0x2, 0x2, 0x2, 0x2, - 0x125, 0x3, 0x2, 0x2, 0x2, 0x2, 0x127, 0x3, 0x2, 0x2, 0x2, 0x2, 0x129, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x12b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x12d, 0x3, - 0x2, 0x2, 0x2, 0x2, 0x12f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x131, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x133, 0x3, 0x2, 0x2, 0x2, 0x2, 0x135, 0x3, 0x2, 0x2, - 0x2, 0x2, 0x137, 0x3, 0x2, 0x2, 0x2, 0x2, 0x139, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x13b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x13d, 0x3, 0x2, 0x2, 0x2, 0x2, - 0x13f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x141, 0x3, 0x2, 0x2, 0x2, 0x2, 0x143, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x145, 0x3, 0x2, 0x2, 0x2, 0x2, 0x147, 0x3, - 0x2, 0x2, 0x2, 0x2, 0x149, 0x3, 0x2, 0x2, 0x2, 0x2, 0x14b, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x14d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x14f, 0x3, 0x2, 0x2, - 0x2, 0x2, 0x151, 0x3, 0x2, 0x2, 0x2, 0x2, 0x153, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x155, 0x3, 0x2, 0x2, 0x2, 0x2, 0x157, 0x3, 0x2, 0x2, 0x2, 0x2, - 0x159, 0x3, 0x2, 0x2, 0x2, 0x2, 0x15b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x15d, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x15f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x161, 0x3, - 0x2, 0x2, 0x2, 0x2, 0x163, 0x3, 0x2, 0x2, 0x2, 0x2, 0x165, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x167, 0x3, 0x2, 0x2, 0x2, 0x2, 0x169, 0x3, 0x2, 0x2, - 0x2, 0x2, 0x16b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x16d, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x16f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x171, 0x3, 0x2, 0x2, 0x2, 0x2, - 0x173, 0x3, 0x2, 0x2, 0x2, 0x2, 0x175, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1b3, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x1b5, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1b7, 0x3, - 0x2, 0x2, 0x2, 0x2, 0x1b9, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1bb, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x1bd, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1bf, 0x3, 0x2, 0x2, - 0x2, 0x2, 0x1c1, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1c3, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x1c5, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1c7, 0x3, 0x2, 0x2, 0x2, 0x2, - 0x1c9, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1cb, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1cd, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x1cf, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1d1, 0x3, - 0x2, 0x2, 0x2, 0x2, 0x1d3, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1d5, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x1d7, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1d9, 0x3, 0x2, 0x2, - 0x2, 0x2, 0x1db, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1dd, 0x3, 0x2, 0x2, 0x2, - 0x2, 0x1df, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1e1, 0x3, 0x2, 0x2, 0x2, 0x2, - 0x1e3, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1e5, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1e7, - 0x3, 0x2, 0x2, 0x2, 0x2, 0x1e9, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1eb, 0x3, - 0x2, 0x2, 0x2, 0x2, 0x1ed, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1ef, 0x3, 0x2, - 0x2, 0x2, 0x2, 0x1f1, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1f3, 0x3, 0x2, 0x2, - 0x2, 0x3, 0x1f5, 0x3, 0x2, 0x2, 0x2, 0x5, 0x1f9, 0x3, 0x2, 0x2, 0x2, - 0x7, 0x1ff, 0x3, 0x2, 0x2, 0x2, 0x9, 0x205, 0x3, 0x2, 0x2, 0x2, 0xb, - 0x209, 0x3, 0x2, 0x2, 0x2, 0xd, 0x20f, 0x3, 0x2, 0x2, 0x2, 0xf, 0x213, - 0x3, 0x2, 0x2, 0x2, 0x11, 0x218, 0x3, 0x2, 0x2, 0x2, 0x13, 0x21c, 0x3, - 0x2, 0x2, 0x2, 0x15, 0x222, 0x3, 0x2, 0x2, 0x2, 0x17, 0x233, 0x3, 0x2, - 0x2, 0x2, 0x19, 0x235, 0x3, 0x2, 0x2, 0x2, 0x1b, 0x23a, 0x3, 0x2, 0x2, - 0x2, 0x1d, 0x240, 0x3, 0x2, 0x2, 0x2, 0x1f, 0x247, 0x3, 0x2, 0x2, 0x2, - 0x21, 0x24f, 0x3, 0x2, 0x2, 0x2, 0x23, 0x254, 0x3, 0x2, 0x2, 0x2, 0x25, - 0x257, 0x3, 0x2, 0x2, 0x2, 0x27, 0x25c, 0x3, 0x2, 0x2, 0x2, 0x29, 0x261, - 0x3, 0x2, 0x2, 0x2, 0x2b, 0x267, 0x3, 0x2, 0x2, 0x2, 0x2d, 0x26d, 0x3, - 0x2, 0x2, 0x2, 0x2f, 0x275, 0x3, 0x2, 0x2, 0x2, 0x31, 0x27b, 0x3, 0x2, - 0x2, 0x2, 0x33, 0x283, 0x3, 0x2, 0x2, 0x2, 0x35, 0x28a, 0x3, 0x2, 0x2, - 0x2, 0x37, 0x292, 0x3, 0x2, 0x2, 0x2, 0x39, 0x29d, 0x3, 0x2, 0x2, 0x2, - 0x3b, 0x2a4, 0x3, 0x2, 0x2, 0x2, 0x3d, 0x2aa, 0x3, 0x2, 0x2, 0x2, 0x3f, - 0x2af, 0x3, 0x2, 0x2, 0x2, 0x41, 0x2b8, 0x3, 0x2, 0x2, 0x2, 0x43, 0x2c2, - 0x3, 0x2, 0x2, 0x2, 0x45, 0x2c7, 0x3, 0x2, 0x2, 0x2, 0x47, 0x2cb, 0x3, - 0x2, 0x2, 0x2, 0x49, 0x2d7, 0x3, 0x2, 0x2, 0x2, 0x4b, 0x2df, 0x3, 0x2, - 0x2, 0x2, 0x4d, 0x2e5, 0x3, 0x2, 0x2, 0x2, 0x4f, 0x2ec, 0x3, 0x2, 0x2, - 0x2, 0x51, 0x2f1, 0x3, 0x2, 0x2, 0x2, 0x53, 0x2fc, 0x3, 0x2, 0x2, 0x2, - 0x55, 0x305, 0x3, 0x2, 0x2, 0x2, 0x57, 0x30c, 0x3, 0x2, 0x2, 0x2, 0x59, - 0x319, 0x3, 0x2, 0x2, 0x2, 0x5b, 0x324, 0x3, 0x2, 0x2, 0x2, 0x5d, 0x329, - 0x3, 0x2, 0x2, 0x2, 0x5f, 0x332, 0x3, 0x2, 0x2, 0x2, 0x61, 0x33e, 0x3, - 0x2, 0x2, 0x2, 0x63, 0x343, 0x3, 0x2, 0x2, 0x2, 0x65, 0x348, 0x3, 0x2, - 0x2, 0x2, 0x67, 0x34c, 0x3, 0x2, 0x2, 0x2, 0x69, 0x353, 0x3, 0x2, 0x2, - 0x2, 0x6b, 0x35a, 0x3, 0x2, 0x2, 0x2, 0x6d, 0x361, 0x3, 0x2, 0x2, 0x2, - 0x6f, 0x369, 0x3, 0x2, 0x2, 0x2, 0x71, 0x374, 0x3, 0x2, 0x2, 0x2, 0x73, - 0x37c, 0x3, 0x2, 0x2, 0x2, 0x75, 0x384, 0x3, 0x2, 0x2, 0x2, 0x77, 0x38a, - 0x3, 0x2, 0x2, 0x2, 0x79, 0x390, 0x3, 0x2, 0x2, 0x2, 0x7b, 0x396, 0x3, - 0x2, 0x2, 0x2, 0x7d, 0x39a, 0x3, 0x2, 0x2, 0x2, 0x7f, 0x3a1, 0x3, 0x2, - 0x2, 0x2, 0x81, 0x3a8, 0x3, 0x2, 0x2, 0x2, 0x83, 0x3ad, 0x3, 0x2, 0x2, - 0x2, 0x85, 0x3b2, 0x3, 0x2, 0x2, 0x2, 0x87, 0x3bb, 0x3, 0x2, 0x2, 0x2, - 0x89, 0x3c2, 0x3, 0x2, 0x2, 0x2, 0x8b, 0x3ce, 0x3, 0x2, 0x2, 0x2, 0x8d, - 0x3d4, 0x3, 0x2, 0x2, 0x2, 0x8f, 0x3db, 0x3, 0x2, 0x2, 0x2, 0x91, 0x3e8, - 0x3, 0x2, 0x2, 0x2, 0x93, 0x3ed, 0x3, 0x2, 0x2, 0x2, 0x95, 0x3f0, 0x3, - 0x2, 0x2, 0x2, 0x97, 0x3f3, 0x3, 0x2, 0x2, 0x2, 0x99, 0x3f9, 0x3, 0x2, - 0x2, 0x2, 0x9b, 0x3fc, 0x3, 0x2, 0x2, 0x2, 0x9d, 0x40f, 0x3, 0x2, 0x2, - 0x2, 0x9f, 0x411, 0x3, 0x2, 0x2, 0x2, 0xa1, 0x41b, 0x3, 0x2, 0x2, 0x2, - 0xa3, 0x421, 0x3, 0x2, 0x2, 0x2, 0xa5, 0x428, 0x3, 0x2, 0x2, 0x2, 0xa7, - 0x431, 0x3, 0x2, 0x2, 0x2, 0xa9, 0x436, 0x3, 0x2, 0x2, 0x2, 0xab, 0x439, - 0x3, 0x2, 0x2, 0x2, 0xad, 0x446, 0x3, 0x2, 0x2, 0x2, 0xaf, 0x44b, 0x3, - 0x2, 0x2, 0x2, 0xb1, 0x44f, 0x3, 0x2, 0x2, 0x2, 0xb3, 0x454, 0x3, 0x2, - 0x2, 0x2, 0xb5, 0x459, 0x3, 0x2, 0x2, 0x2, 0xb7, 0x460, 0x3, 0x2, 0x2, - 0x2, 0xb9, 0x468, 0x3, 0x2, 0x2, 0x2, 0xbb, 0x46d, 0x3, 0x2, 0x2, 0x2, - 0xbd, 0x476, 0x3, 0x2, 0x2, 0x2, 0xbf, 0x47b, 0x3, 0x2, 0x2, 0x2, 0xc1, - 0x481, 0x3, 0x2, 0x2, 0x2, 0xc3, 0x486, 0x3, 0x2, 0x2, 0x2, 0xc5, 0x48c, - 0x3, 0x2, 0x2, 0x2, 0xc7, 0x491, 0x3, 0x2, 0x2, 0x2, 0xc9, 0x49e, 0x3, - 0x2, 0x2, 0x2, 0xcb, 0x4a2, 0x3, 0x2, 0x2, 0x2, 0xcd, 0x4a9, 0x3, 0x2, - 0x2, 0x2, 0xcf, 0x4ad, 0x3, 0x2, 0x2, 0x2, 0xd1, 0x4b4, 0x3, 0x2, 0x2, - 0x2, 0xd3, 0x4bb, 0x3, 0x2, 0x2, 0x2, 0xd5, 0x4c1, 0x3, 0x2, 0x2, 0x2, - 0xd7, 0x4c6, 0x3, 0x2, 0x2, 0x2, 0xd9, 0x4cf, 0x3, 0x2, 0x2, 0x2, 0xdb, - 0x4d3, 0x3, 0x2, 0x2, 0x2, 0xdd, 0x4d6, 0x3, 0x2, 0x2, 0x2, 0xdf, 0x4da, - 0x3, 0x2, 0x2, 0x2, 0xe1, 0x4df, 0x3, 0x2, 0x2, 0x2, 0xe3, 0x4e5, 0x3, - 0x2, 0x2, 0x2, 0xe5, 0x4ec, 0x3, 0x2, 0x2, 0x2, 0xe7, 0x4ef, 0x3, 0x2, - 0x2, 0x2, 0xe9, 0x4f8, 0x3, 0x2, 0x2, 0x2, 0xeb, 0x4fb, 0x3, 0x2, 0x2, - 0x2, 0xed, 0x501, 0x3, 0x2, 0x2, 0x2, 0xef, 0x507, 0x3, 0x2, 0x2, 0x2, - 0xf1, 0x50f, 0x3, 0x2, 0x2, 0x2, 0xf3, 0x519, 0x3, 0x2, 0x2, 0x2, 0xf5, - 0x522, 0x3, 0x2, 0x2, 0x2, 0xf7, 0x52b, 0x3, 0x2, 0x2, 0x2, 0xf9, 0x533, - 0x3, 0x2, 0x2, 0x2, 0xfb, 0x53b, 0x3, 0x2, 0x2, 0x2, 0xfd, 0x541, 0x3, - 0x2, 0x2, 0x2, 0xff, 0x548, 0x3, 0x2, 0x2, 0x2, 0x101, 0x54f, 0x3, 0x2, - 0x2, 0x2, 0x103, 0x556, 0x3, 0x2, 0x2, 0x2, 0x105, 0x55e, 0x3, 0x2, - 0x2, 0x2, 0x107, 0x566, 0x3, 0x2, 0x2, 0x2, 0x109, 0x571, 0x3, 0x2, - 0x2, 0x2, 0x10b, 0x577, 0x3, 0x2, 0x2, 0x2, 0x10d, 0x57e, 0x3, 0x2, - 0x2, 0x2, 0x10f, 0x585, 0x3, 0x2, 0x2, 0x2, 0x111, 0x58c, 0x3, 0x2, - 0x2, 0x2, 0x113, 0x593, 0x3, 0x2, 0x2, 0x2, 0x115, 0x598, 0x3, 0x2, - 0x2, 0x2, 0x117, 0x59e, 0x3, 0x2, 0x2, 0x2, 0x119, 0x5a2, 0x3, 0x2, - 0x2, 0x2, 0x11b, 0x5ab, 0x3, 0x2, 0x2, 0x2, 0x11d, 0x5b0, 0x3, 0x2, - 0x2, 0x2, 0x11f, 0x5b7, 0x3, 0x2, 0x2, 0x2, 0x121, 0x5bd, 0x3, 0x2, - 0x2, 0x2, 0x123, 0x5c2, 0x3, 0x2, 0x2, 0x2, 0x125, 0x5cc, 0x3, 0x2, - 0x2, 0x2, 0x127, 0x5d1, 0x3, 0x2, 0x2, 0x2, 0x129, 0x5d8, 0x3, 0x2, - 0x2, 0x2, 0x12b, 0x5df, 0x3, 0x2, 0x2, 0x2, 0x12d, 0x5e5, 0x3, 0x2, - 0x2, 0x2, 0x12f, 0x5ec, 0x3, 0x2, 0x2, 0x2, 0x131, 0x5f6, 0x3, 0x2, - 0x2, 0x2, 0x133, 0x5fb, 0x3, 0x2, 0x2, 0x2, 0x135, 0x600, 0x3, 0x2, - 0x2, 0x2, 0x137, 0x605, 0x3, 0x2, 0x2, 0x2, 0x139, 0x60d, 0x3, 0x2, - 0x2, 0x2, 0x13b, 0x617, 0x3, 0x2, 0x2, 0x2, 0x13d, 0x61a, 0x3, 0x2, - 0x2, 0x2, 0x13f, 0x61e, 0x3, 0x2, 0x2, 0x2, 0x141, 0x625, 0x3, 0x2, - 0x2, 0x2, 0x143, 0x62e, 0x3, 0x2, 0x2, 0x2, 0x145, 0x633, 0x3, 0x2, - 0x2, 0x2, 0x147, 0x63c, 0x3, 0x2, 0x2, 0x2, 0x149, 0x640, 0x3, 0x2, - 0x2, 0x2, 0x14b, 0x645, 0x3, 0x2, 0x2, 0x2, 0x14d, 0x64b, 0x3, 0x2, - 0x2, 0x2, 0x14f, 0x652, 0x3, 0x2, 0x2, 0x2, 0x151, 0x656, 0x3, 0x2, - 0x2, 0x2, 0x153, 0x65c, 0x3, 0x2, 0x2, 0x2, 0x155, 0x661, 0x3, 0x2, - 0x2, 0x2, 0x157, 0x668, 0x3, 0x2, 0x2, 0x2, 0x159, 0x66d, 0x3, 0x2, - 0x2, 0x2, 0x15b, 0x674, 0x3, 0x2, 0x2, 0x2, 0x15d, 0x67a, 0x3, 0x2, - 0x2, 0x2, 0x15f, 0x67f, 0x3, 0x2, 0x2, 0x2, 0x161, 0x684, 0x3, 0x2, - 0x2, 0x2, 0x163, 0x68a, 0x3, 0x2, 0x2, 0x2, 0x165, 0x699, 0x3, 0x2, - 0x2, 0x2, 0x167, 0x69b, 0x3, 0x2, 0x2, 0x2, 0x169, 0x6a1, 0x3, 0x2, - 0x2, 0x2, 0x16b, 0x6d0, 0x3, 0x2, 0x2, 0x2, 0x16d, 0x71e, 0x3, 0x2, - 0x2, 0x2, 0x16f, 0x720, 0x3, 0x2, 0x2, 0x2, 0x171, 0x727, 0x3, 0x2, - 0x2, 0x2, 0x173, 0x72b, 0x3, 0x2, 0x2, 0x2, 0x175, 0x732, 0x3, 0x2, - 0x2, 0x2, 0x177, 0x741, 0x3, 0x2, 0x2, 0x2, 0x179, 0x743, 0x3, 0x2, - 0x2, 0x2, 0x17b, 0x745, 0x3, 0x2, 0x2, 0x2, 0x17d, 0x747, 0x3, 0x2, - 0x2, 0x2, 0x17f, 0x749, 0x3, 0x2, 0x2, 0x2, 0x181, 0x74b, 0x3, 0x2, - 0x2, 0x2, 0x183, 0x74d, 0x3, 0x2, 0x2, 0x2, 0x185, 0x74f, 0x3, 0x2, - 0x2, 0x2, 0x187, 0x751, 0x3, 0x2, 0x2, 0x2, 0x189, 0x753, 0x3, 0x2, - 0x2, 0x2, 0x18b, 0x755, 0x3, 0x2, 0x2, 0x2, 0x18d, 0x757, 0x3, 0x2, - 0x2, 0x2, 0x18f, 0x759, 0x3, 0x2, 0x2, 0x2, 0x191, 0x75b, 0x3, 0x2, - 0x2, 0x2, 0x193, 0x75d, 0x3, 0x2, 0x2, 0x2, 0x195, 0x75f, 0x3, 0x2, - 0x2, 0x2, 0x197, 0x761, 0x3, 0x2, 0x2, 0x2, 0x199, 0x763, 0x3, 0x2, - 0x2, 0x2, 0x19b, 0x765, 0x3, 0x2, 0x2, 0x2, 0x19d, 0x767, 0x3, 0x2, - 0x2, 0x2, 0x19f, 0x769, 0x3, 0x2, 0x2, 0x2, 0x1a1, 0x76b, 0x3, 0x2, - 0x2, 0x2, 0x1a3, 0x76d, 0x3, 0x2, 0x2, 0x2, 0x1a5, 0x76f, 0x3, 0x2, - 0x2, 0x2, 0x1a7, 0x771, 0x3, 0x2, 0x2, 0x2, 0x1a9, 0x773, 0x3, 0x2, - 0x2, 0x2, 0x1ab, 0x775, 0x3, 0x2, 0x2, 0x2, 0x1ad, 0x777, 0x3, 0x2, - 0x2, 0x2, 0x1af, 0x779, 0x3, 0x2, 0x2, 0x2, 0x1b1, 0x77b, 0x3, 0x2, - 0x2, 0x2, 0x1b3, 0x77d, 0x3, 0x2, 0x2, 0x2, 0x1b5, 0x780, 0x3, 0x2, - 0x2, 0x2, 0x1b7, 0x782, 0x3, 0x2, 0x2, 0x2, 0x1b9, 0x784, 0x3, 0x2, - 0x2, 0x2, 0x1bb, 0x786, 0x3, 0x2, 0x2, 0x2, 0x1bd, 0x788, 0x3, 0x2, - 0x2, 0x2, 0x1bf, 0x78a, 0x3, 0x2, 0x2, 0x2, 0x1c1, 0x78d, 0x3, 0x2, - 0x2, 0x2, 0x1c3, 0x78f, 0x3, 0x2, 0x2, 0x2, 0x1c5, 0x791, 0x3, 0x2, - 0x2, 0x2, 0x1c7, 0x794, 0x3, 0x2, 0x2, 0x2, 0x1c9, 0x796, 0x3, 0x2, - 0x2, 0x2, 0x1cb, 0x799, 0x3, 0x2, 0x2, 0x2, 0x1cd, 0x79b, 0x3, 0x2, - 0x2, 0x2, 0x1cf, 0x79d, 0x3, 0x2, 0x2, 0x2, 0x1d1, 0x79f, 0x3, 0x2, - 0x2, 0x2, 0x1d3, 0x7a2, 0x3, 0x2, 0x2, 0x2, 0x1d5, 0x7a4, 0x3, 0x2, - 0x2, 0x2, 0x1d7, 0x7aa, 0x3, 0x2, 0x2, 0x2, 0x1d9, 0x7ac, 0x3, 0x2, - 0x2, 0x2, 0x1db, 0x7ae, 0x3, 0x2, 0x2, 0x2, 0x1dd, 0x7b0, 0x3, 0x2, - 0x2, 0x2, 0x1df, 0x7b2, 0x3, 0x2, 0x2, 0x2, 0x1e1, 0x7b4, 0x3, 0x2, - 0x2, 0x2, 0x1e3, 0x7b6, 0x3, 0x2, 0x2, 0x2, 0x1e5, 0x7b8, 0x3, 0x2, - 0x2, 0x2, 0x1e7, 0x7ba, 0x3, 0x2, 0x2, 0x2, 0x1e9, 0x7bc, 0x3, 0x2, - 0x2, 0x2, 0x1eb, 0x7be, 0x3, 0x2, 0x2, 0x2, 0x1ed, 0x7c0, 0x3, 0x2, - 0x2, 0x2, 0x1ef, 0x7c2, 0x3, 0x2, 0x2, 0x2, 0x1f1, 0x7d0, 0x3, 0x2, - 0x2, 0x2, 0x1f3, 0x7de, 0x3, 0x2, 0x2, 0x2, 0x1f5, 0x1f6, 0x5, 0x177, - 0xbc, 0x2, 0x1f6, 0x1f7, 0x5, 0x17d, 0xbf, 0x2, 0x1f7, 0x1f8, 0x5, 0x17d, - 0xbf, 0x2, 0x1f8, 0x4, 0x3, 0x2, 0x2, 0x2, 0x1f9, 0x1fa, 0x5, 0x177, - 0xbc, 0x2, 0x1fa, 0x1fb, 0x5, 0x181, 0xc1, 0x2, 0x1fb, 0x1fc, 0x5, 0x19d, - 0xcf, 0x2, 0x1fc, 0x1fd, 0x5, 0x17f, 0xc0, 0x2, 0x1fd, 0x1fe, 0x5, 0x199, - 0xcd, 0x2, 0x1fe, 0x6, 0x3, 0x2, 0x2, 0x2, 0x1ff, 0x200, 0x5, 0x177, - 0xbc, 0x2, 0x200, 0x201, 0x5, 0x18d, 0xc7, 0x2, 0x201, 0x202, 0x5, 0x187, - 0xc4, 0x2, 0x202, 0x203, 0x5, 0x177, 0xbc, 0x2, 0x203, 0x204, 0x5, 0x19b, - 0xce, 0x2, 0x204, 0x8, 0x3, 0x2, 0x2, 0x2, 0x205, 0x206, 0x5, 0x177, - 0xbc, 0x2, 0x206, 0x207, 0x5, 0x18d, 0xc7, 0x2, 0x207, 0x208, 0x5, 0x18d, - 0xc7, 0x2, 0x208, 0xa, 0x3, 0x2, 0x2, 0x2, 0x209, 0x20a, 0x5, 0x177, - 0xbc, 0x2, 0x20a, 0x20b, 0x5, 0x18d, 0xc7, 0x2, 0x20b, 0x20c, 0x5, 0x19d, - 0xcf, 0x2, 0x20c, 0x20d, 0x5, 0x17f, 0xc0, 0x2, 0x20d, 0x20e, 0x5, 0x199, - 0xcd, 0x2, 0x20e, 0xc, 0x3, 0x2, 0x2, 0x2, 0x20f, 0x210, 0x5, 0x177, - 0xbc, 0x2, 0x210, 0x211, 0x5, 0x191, 0xc9, 0x2, 0x211, 0x212, 0x5, 0x17d, - 0xbf, 0x2, 0x212, 0xe, 0x3, 0x2, 0x2, 0x2, 0x213, 0x214, 0x5, 0x177, - 0xbc, 0x2, 0x214, 0x215, 0x5, 0x191, 0xc9, 0x2, 0x215, 0x216, 0x5, 0x19d, - 0xcf, 0x2, 0x216, 0x217, 0x5, 0x187, 0xc4, 0x2, 0x217, 0x10, 0x3, 0x2, - 0x2, 0x2, 0x218, 0x219, 0x5, 0x177, 0xbc, 0x2, 0x219, 0x21a, 0x5, 0x191, - 0xc9, 0x2, 0x21a, 0x21b, 0x5, 0x1a7, 0xd4, 0x2, 0x21b, 0x12, 0x3, 0x2, - 0x2, 0x2, 0x21c, 0x21d, 0x5, 0x177, 0xbc, 0x2, 0x21d, 0x21e, 0x5, 0x199, - 0xcd, 0x2, 0x21e, 0x21f, 0x5, 0x199, 0xcd, 0x2, 0x21f, 0x220, 0x5, 0x177, - 0xbc, 0x2, 0x220, 0x221, 0x5, 0x1a7, 0xd4, 0x2, 0x221, 0x14, 0x3, 0x2, - 0x2, 0x2, 0x222, 0x223, 0x5, 0x177, 0xbc, 0x2, 0x223, 0x224, 0x5, 0x19b, - 0xce, 0x2, 0x224, 0x16, 0x3, 0x2, 0x2, 0x2, 0x225, 0x226, 0x5, 0x177, - 0xbc, 0x2, 0x226, 0x227, 0x5, 0x19b, 0xce, 0x2, 0x227, 0x228, 0x5, 0x17b, - 0xbe, 0x2, 0x228, 0x234, 0x3, 0x2, 0x2, 0x2, 0x229, 0x22a, 0x5, 0x177, - 0xbc, 0x2, 0x22a, 0x22b, 0x5, 0x19b, 0xce, 0x2, 0x22b, 0x22c, 0x5, 0x17b, - 0xbe, 0x2, 0x22c, 0x22d, 0x5, 0x17f, 0xc0, 0x2, 0x22d, 0x22e, 0x5, 0x191, - 0xc9, 0x2, 0x22e, 0x22f, 0x5, 0x17d, 0xbf, 0x2, 0x22f, 0x230, 0x5, 0x187, - 0xc4, 0x2, 0x230, 0x231, 0x5, 0x191, 0xc9, 0x2, 0x231, 0x232, 0x5, 0x183, - 0xc2, 0x2, 0x232, 0x234, 0x3, 0x2, 0x2, 0x2, 0x233, 0x225, 0x3, 0x2, - 0x2, 0x2, 0x233, 0x229, 0x3, 0x2, 0x2, 0x2, 0x234, 0x18, 0x3, 0x2, 0x2, - 0x2, 0x235, 0x236, 0x5, 0x177, 0xbc, 0x2, 0x236, 0x237, 0x5, 0x19b, - 0xce, 0x2, 0x237, 0x238, 0x5, 0x193, 0xca, 0x2, 0x238, 0x239, 0x5, 0x181, - 0xc1, 0x2, 0x239, 0x1a, 0x3, 0x2, 0x2, 0x2, 0x23a, 0x23b, 0x5, 0x177, - 0xbc, 0x2, 0x23b, 0x23c, 0x5, 0x19b, 0xce, 0x2, 0x23c, 0x23d, 0x5, 0x1a7, - 0xd4, 0x2, 0x23d, 0x23e, 0x5, 0x191, 0xc9, 0x2, 0x23e, 0x23f, 0x5, 0x17b, - 0xbe, 0x2, 0x23f, 0x1c, 0x3, 0x2, 0x2, 0x2, 0x240, 0x241, 0x5, 0x177, - 0xbc, 0x2, 0x241, 0x242, 0x5, 0x19d, 0xcf, 0x2, 0x242, 0x243, 0x5, 0x19d, - 0xcf, 0x2, 0x243, 0x244, 0x5, 0x177, 0xbc, 0x2, 0x244, 0x245, 0x5, 0x17b, - 0xbe, 0x2, 0x245, 0x246, 0x5, 0x185, 0xc3, 0x2, 0x246, 0x1e, 0x3, 0x2, - 0x2, 0x2, 0x247, 0x248, 0x5, 0x179, 0xbd, 0x2, 0x248, 0x249, 0x5, 0x17f, - 0xc0, 0x2, 0x249, 0x24a, 0x5, 0x19d, 0xcf, 0x2, 0x24a, 0x24b, 0x5, 0x1a3, - 0xd2, 0x2, 0x24b, 0x24c, 0x5, 0x17f, 0xc0, 0x2, 0x24c, 0x24d, 0x5, 0x17f, - 0xc0, 0x2, 0x24d, 0x24e, 0x5, 0x191, 0xc9, 0x2, 0x24e, 0x20, 0x3, 0x2, - 0x2, 0x2, 0x24f, 0x250, 0x5, 0x179, 0xbd, 0x2, 0x250, 0x251, 0x5, 0x193, - 0xca, 0x2, 0x251, 0x252, 0x5, 0x19d, 0xcf, 0x2, 0x252, 0x253, 0x5, 0x185, - 0xc3, 0x2, 0x253, 0x22, 0x3, 0x2, 0x2, 0x2, 0x254, 0x255, 0x5, 0x179, - 0xbd, 0x2, 0x255, 0x256, 0x5, 0x1a7, 0xd4, 0x2, 0x256, 0x24, 0x3, 0x2, - 0x2, 0x2, 0x257, 0x258, 0x5, 0x17b, 0xbe, 0x2, 0x258, 0x259, 0x5, 0x177, - 0xbc, 0x2, 0x259, 0x25a, 0x5, 0x19b, 0xce, 0x2, 0x25a, 0x25b, 0x5, 0x17f, - 0xc0, 0x2, 0x25b, 0x26, 0x3, 0x2, 0x2, 0x2, 0x25c, 0x25d, 0x5, 0x17b, - 0xbe, 0x2, 0x25d, 0x25e, 0x5, 0x177, 0xbc, 0x2, 0x25e, 0x25f, 0x5, 0x19b, - 0xce, 0x2, 0x25f, 0x260, 0x5, 0x19d, 0xcf, 0x2, 0x260, 0x28, 0x3, 0x2, - 0x2, 0x2, 0x261, 0x262, 0x5, 0x17b, 0xbe, 0x2, 0x262, 0x263, 0x5, 0x185, - 0xc3, 0x2, 0x263, 0x264, 0x5, 0x17f, 0xc0, 0x2, 0x264, 0x265, 0x5, 0x17b, - 0xbe, 0x2, 0x265, 0x266, 0x5, 0x18b, 0xc6, 0x2, 0x266, 0x2a, 0x3, 0x2, - 0x2, 0x2, 0x267, 0x268, 0x5, 0x17b, 0xbe, 0x2, 0x268, 0x269, 0x5, 0x18d, - 0xc7, 0x2, 0x269, 0x26a, 0x5, 0x17f, 0xc0, 0x2, 0x26a, 0x26b, 0x5, 0x177, - 0xbc, 0x2, 0x26b, 0x26c, 0x5, 0x199, 0xcd, 0x2, 0x26c, 0x2c, 0x3, 0x2, - 0x2, 0x2, 0x26d, 0x26e, 0x5, 0x17b, 0xbe, 0x2, 0x26e, 0x26f, 0x5, 0x18d, - 0xc7, 0x2, 0x26f, 0x270, 0x5, 0x19f, 0xd0, 0x2, 0x270, 0x271, 0x5, 0x19b, - 0xce, 0x2, 0x271, 0x272, 0x5, 0x19d, 0xcf, 0x2, 0x272, 0x273, 0x5, 0x17f, - 0xc0, 0x2, 0x273, 0x274, 0x5, 0x199, 0xcd, 0x2, 0x274, 0x2e, 0x3, 0x2, - 0x2, 0x2, 0x275, 0x276, 0x5, 0x17b, 0xbe, 0x2, 0x276, 0x277, 0x5, 0x193, - 0xca, 0x2, 0x277, 0x278, 0x5, 0x17d, 0xbf, 0x2, 0x278, 0x279, 0x5, 0x17f, - 0xc0, 0x2, 0x279, 0x27a, 0x5, 0x17b, 0xbe, 0x2, 0x27a, 0x30, 0x3, 0x2, - 0x2, 0x2, 0x27b, 0x27c, 0x5, 0x17b, 0xbe, 0x2, 0x27c, 0x27d, 0x5, 0x193, - 0xca, 0x2, 0x27d, 0x27e, 0x5, 0x18d, 0xc7, 0x2, 0x27e, 0x27f, 0x5, 0x18d, - 0xc7, 0x2, 0x27f, 0x280, 0x5, 0x177, 0xbc, 0x2, 0x280, 0x281, 0x5, 0x19d, - 0xcf, 0x2, 0x281, 0x282, 0x5, 0x17f, 0xc0, 0x2, 0x282, 0x32, 0x3, 0x2, - 0x2, 0x2, 0x283, 0x284, 0x5, 0x17b, 0xbe, 0x2, 0x284, 0x285, 0x5, 0x193, - 0xca, 0x2, 0x285, 0x286, 0x5, 0x18d, 0xc7, 0x2, 0x286, 0x287, 0x5, 0x19f, - 0xd0, 0x2, 0x287, 0x288, 0x5, 0x18f, 0xc8, 0x2, 0x288, 0x289, 0x5, 0x191, - 0xc9, 0x2, 0x289, 0x34, 0x3, 0x2, 0x2, 0x2, 0x28a, 0x28b, 0x5, 0x17b, - 0xbe, 0x2, 0x28b, 0x28c, 0x5, 0x193, 0xca, 0x2, 0x28c, 0x28d, 0x5, 0x18f, - 0xc8, 0x2, 0x28d, 0x28e, 0x5, 0x18f, 0xc8, 0x2, 0x28e, 0x28f, 0x5, 0x17f, - 0xc0, 0x2, 0x28f, 0x290, 0x5, 0x191, 0xc9, 0x2, 0x290, 0x291, 0x5, 0x19d, - 0xcf, 0x2, 0x291, 0x36, 0x3, 0x2, 0x2, 0x2, 0x292, 0x293, 0x5, 0x17b, - 0xbe, 0x2, 0x293, 0x294, 0x5, 0x193, 0xca, 0x2, 0x294, 0x295, 0x5, 0x191, - 0xc9, 0x2, 0x295, 0x296, 0x5, 0x19b, 0xce, 0x2, 0x296, 0x297, 0x5, 0x19d, - 0xcf, 0x2, 0x297, 0x298, 0x5, 0x199, 0xcd, 0x2, 0x298, 0x299, 0x5, 0x177, - 0xbc, 0x2, 0x299, 0x29a, 0x5, 0x187, 0xc4, 0x2, 0x29a, 0x29b, 0x5, 0x191, - 0xc9, 0x2, 0x29b, 0x29c, 0x5, 0x19d, 0xcf, 0x2, 0x29c, 0x38, 0x3, 0x2, - 0x2, 0x2, 0x29d, 0x29e, 0x5, 0x17b, 0xbe, 0x2, 0x29e, 0x29f, 0x5, 0x199, - 0xcd, 0x2, 0x29f, 0x2a0, 0x5, 0x17f, 0xc0, 0x2, 0x2a0, 0x2a1, 0x5, 0x177, - 0xbc, 0x2, 0x2a1, 0x2a2, 0x5, 0x19d, 0xcf, 0x2, 0x2a2, 0x2a3, 0x5, 0x17f, - 0xc0, 0x2, 0x2a3, 0x3a, 0x3, 0x2, 0x2, 0x2, 0x2a4, 0x2a5, 0x5, 0x17b, - 0xbe, 0x2, 0x2a5, 0x2a6, 0x5, 0x199, 0xcd, 0x2, 0x2a6, 0x2a7, 0x5, 0x193, - 0xca, 0x2, 0x2a7, 0x2a8, 0x5, 0x19b, 0xce, 0x2, 0x2a8, 0x2a9, 0x5, 0x19b, - 0xce, 0x2, 0x2a9, 0x3c, 0x3, 0x2, 0x2, 0x2, 0x2aa, 0x2ab, 0x5, 0x17b, - 0xbe, 0x2, 0x2ab, 0x2ac, 0x5, 0x19f, 0xd0, 0x2, 0x2ac, 0x2ad, 0x5, 0x179, - 0xbd, 0x2, 0x2ad, 0x2ae, 0x5, 0x17f, 0xc0, 0x2, 0x2ae, 0x3e, 0x3, 0x2, - 0x2, 0x2, 0x2af, 0x2b0, 0x5, 0x17d, 0xbf, 0x2, 0x2b0, 0x2b1, 0x5, 0x177, - 0xbc, 0x2, 0x2b1, 0x2b2, 0x5, 0x19d, 0xcf, 0x2, 0x2b2, 0x2b3, 0x5, 0x177, - 0xbc, 0x2, 0x2b3, 0x2b4, 0x5, 0x179, 0xbd, 0x2, 0x2b4, 0x2b5, 0x5, 0x177, - 0xbc, 0x2, 0x2b5, 0x2b6, 0x5, 0x19b, 0xce, 0x2, 0x2b6, 0x2b7, 0x5, 0x17f, - 0xc0, 0x2, 0x2b7, 0x40, 0x3, 0x2, 0x2, 0x2, 0x2b8, 0x2b9, 0x5, 0x17d, - 0xbf, 0x2, 0x2b9, 0x2ba, 0x5, 0x177, 0xbc, 0x2, 0x2ba, 0x2bb, 0x5, 0x19d, - 0xcf, 0x2, 0x2bb, 0x2bc, 0x5, 0x177, 0xbc, 0x2, 0x2bc, 0x2bd, 0x5, 0x179, - 0xbd, 0x2, 0x2bd, 0x2be, 0x5, 0x177, 0xbc, 0x2, 0x2be, 0x2bf, 0x5, 0x19b, - 0xce, 0x2, 0x2bf, 0x2c0, 0x5, 0x17f, 0xc0, 0x2, 0x2c0, 0x2c1, 0x5, 0x19b, - 0xce, 0x2, 0x2c1, 0x42, 0x3, 0x2, 0x2, 0x2, 0x2c2, 0x2c3, 0x5, 0x17d, - 0xbf, 0x2, 0x2c3, 0x2c4, 0x5, 0x177, 0xbc, 0x2, 0x2c4, 0x2c5, 0x5, 0x19d, - 0xcf, 0x2, 0x2c5, 0x2c6, 0x5, 0x17f, 0xc0, 0x2, 0x2c6, 0x44, 0x3, 0x2, - 0x2, 0x2, 0x2c7, 0x2c8, 0x5, 0x17d, 0xbf, 0x2, 0x2c8, 0x2c9, 0x5, 0x177, - 0xbc, 0x2, 0x2c9, 0x2ca, 0x5, 0x1a7, 0xd4, 0x2, 0x2ca, 0x46, 0x3, 0x2, - 0x2, 0x2, 0x2cb, 0x2cc, 0x5, 0x17d, 0xbf, 0x2, 0x2cc, 0x2cd, 0x5, 0x17f, - 0xc0, 0x2, 0x2cd, 0x2ce, 0x5, 0x17d, 0xbf, 0x2, 0x2ce, 0x2cf, 0x5, 0x19f, - 0xd0, 0x2, 0x2cf, 0x2d0, 0x5, 0x195, 0xcb, 0x2, 0x2d0, 0x2d1, 0x5, 0x18d, - 0xc7, 0x2, 0x2d1, 0x2d2, 0x5, 0x187, 0xc4, 0x2, 0x2d2, 0x2d3, 0x5, 0x17b, - 0xbe, 0x2, 0x2d3, 0x2d4, 0x5, 0x177, 0xbc, 0x2, 0x2d4, 0x2d5, 0x5, 0x19d, - 0xcf, 0x2, 0x2d5, 0x2d6, 0x5, 0x17f, 0xc0, 0x2, 0x2d6, 0x48, 0x3, 0x2, - 0x2, 0x2, 0x2d7, 0x2d8, 0x5, 0x17d, 0xbf, 0x2, 0x2d8, 0x2d9, 0x5, 0x17f, - 0xc0, 0x2, 0x2d9, 0x2da, 0x5, 0x181, 0xc1, 0x2, 0x2da, 0x2db, 0x5, 0x177, - 0xbc, 0x2, 0x2db, 0x2dc, 0x5, 0x19f, 0xd0, 0x2, 0x2dc, 0x2dd, 0x5, 0x18d, - 0xc7, 0x2, 0x2dd, 0x2de, 0x5, 0x19d, 0xcf, 0x2, 0x2de, 0x4a, 0x3, 0x2, - 0x2, 0x2, 0x2df, 0x2e0, 0x5, 0x17d, 0xbf, 0x2, 0x2e0, 0x2e1, 0x5, 0x17f, - 0xc0, 0x2, 0x2e1, 0x2e2, 0x5, 0x18d, 0xc7, 0x2, 0x2e2, 0x2e3, 0x5, 0x177, - 0xbc, 0x2, 0x2e3, 0x2e4, 0x5, 0x1a7, 0xd4, 0x2, 0x2e4, 0x4c, 0x3, 0x2, - 0x2, 0x2, 0x2e5, 0x2e6, 0x5, 0x17d, 0xbf, 0x2, 0x2e6, 0x2e7, 0x5, 0x17f, - 0xc0, 0x2, 0x2e7, 0x2e8, 0x5, 0x18d, 0xc7, 0x2, 0x2e8, 0x2e9, 0x5, 0x17f, - 0xc0, 0x2, 0x2e9, 0x2ea, 0x5, 0x19d, 0xcf, 0x2, 0x2ea, 0x2eb, 0x5, 0x17f, - 0xc0, 0x2, 0x2eb, 0x4e, 0x3, 0x2, 0x2, 0x2, 0x2ec, 0x2ed, 0x5, 0x17d, - 0xbf, 0x2, 0x2ed, 0x2ee, 0x5, 0x17f, 0xc0, 0x2, 0x2ee, 0x2ef, 0x5, 0x19b, - 0xce, 0x2, 0x2ef, 0x2f0, 0x5, 0x17b, 0xbe, 0x2, 0x2f0, 0x50, 0x3, 0x2, - 0x2, 0x2, 0x2f1, 0x2f2, 0x5, 0x17d, 0xbf, 0x2, 0x2f2, 0x2f3, 0x5, 0x17f, - 0xc0, 0x2, 0x2f3, 0x2f4, 0x5, 0x19b, 0xce, 0x2, 0x2f4, 0x2f5, 0x5, 0x17b, - 0xbe, 0x2, 0x2f5, 0x2f6, 0x5, 0x17f, 0xc0, 0x2, 0x2f6, 0x2f7, 0x5, 0x191, - 0xc9, 0x2, 0x2f7, 0x2f8, 0x5, 0x17d, 0xbf, 0x2, 0x2f8, 0x2f9, 0x5, 0x187, - 0xc4, 0x2, 0x2f9, 0x2fa, 0x5, 0x191, 0xc9, 0x2, 0x2fa, 0x2fb, 0x5, 0x183, - 0xc2, 0x2, 0x2fb, 0x52, 0x3, 0x2, 0x2, 0x2, 0x2fc, 0x2fd, 0x5, 0x17d, - 0xbf, 0x2, 0x2fd, 0x2fe, 0x5, 0x17f, 0xc0, 0x2, 0x2fe, 0x2ff, 0x5, 0x19b, - 0xce, 0x2, 0x2ff, 0x300, 0x5, 0x17b, 0xbe, 0x2, 0x300, 0x301, 0x5, 0x199, - 0xcd, 0x2, 0x301, 0x302, 0x5, 0x187, 0xc4, 0x2, 0x302, 0x303, 0x5, 0x179, - 0xbd, 0x2, 0x303, 0x304, 0x5, 0x17f, 0xc0, 0x2, 0x304, 0x54, 0x3, 0x2, - 0x2, 0x2, 0x305, 0x306, 0x5, 0x17d, 0xbf, 0x2, 0x306, 0x307, 0x5, 0x17f, - 0xc0, 0x2, 0x307, 0x308, 0x5, 0x19d, 0xcf, 0x2, 0x308, 0x309, 0x5, 0x177, - 0xbc, 0x2, 0x309, 0x30a, 0x5, 0x17b, 0xbe, 0x2, 0x30a, 0x30b, 0x5, 0x185, - 0xc3, 0x2, 0x30b, 0x56, 0x3, 0x2, 0x2, 0x2, 0x30c, 0x30d, 0x5, 0x17d, - 0xbf, 0x2, 0x30d, 0x30e, 0x5, 0x187, 0xc4, 0x2, 0x30e, 0x30f, 0x5, 0x17b, - 0xbe, 0x2, 0x30f, 0x310, 0x5, 0x19d, 0xcf, 0x2, 0x310, 0x311, 0x5, 0x187, - 0xc4, 0x2, 0x311, 0x312, 0x5, 0x193, 0xca, 0x2, 0x312, 0x313, 0x5, 0x191, - 0xc9, 0x2, 0x313, 0x314, 0x5, 0x177, 0xbc, 0x2, 0x314, 0x315, 0x5, 0x199, - 0xcd, 0x2, 0x315, 0x316, 0x5, 0x187, 0xc4, 0x2, 0x316, 0x317, 0x5, 0x17f, - 0xc0, 0x2, 0x317, 0x318, 0x5, 0x19b, 0xce, 0x2, 0x318, 0x58, 0x3, 0x2, - 0x2, 0x2, 0x319, 0x31a, 0x5, 0x17d, 0xbf, 0x2, 0x31a, 0x31b, 0x5, 0x187, - 0xc4, 0x2, 0x31b, 0x31c, 0x5, 0x17b, 0xbe, 0x2, 0x31c, 0x31d, 0x5, 0x19d, - 0xcf, 0x2, 0x31d, 0x31e, 0x5, 0x187, 0xc4, 0x2, 0x31e, 0x31f, 0x5, 0x193, - 0xca, 0x2, 0x31f, 0x320, 0x5, 0x191, 0xc9, 0x2, 0x320, 0x321, 0x5, 0x177, - 0xbc, 0x2, 0x321, 0x322, 0x5, 0x199, 0xcd, 0x2, 0x322, 0x323, 0x5, 0x1a7, - 0xd4, 0x2, 0x323, 0x5a, 0x3, 0x2, 0x2, 0x2, 0x324, 0x325, 0x5, 0x17d, - 0xbf, 0x2, 0x325, 0x326, 0x5, 0x187, 0xc4, 0x2, 0x326, 0x327, 0x5, 0x19b, - 0xce, 0x2, 0x327, 0x328, 0x5, 0x18b, 0xc6, 0x2, 0x328, 0x5c, 0x3, 0x2, - 0x2, 0x2, 0x329, 0x32a, 0x5, 0x17d, 0xbf, 0x2, 0x32a, 0x32b, 0x5, 0x187, - 0xc4, 0x2, 0x32b, 0x32c, 0x5, 0x19b, 0xce, 0x2, 0x32c, 0x32d, 0x5, 0x19d, - 0xcf, 0x2, 0x32d, 0x32e, 0x5, 0x187, 0xc4, 0x2, 0x32e, 0x32f, 0x5, 0x191, - 0xc9, 0x2, 0x32f, 0x330, 0x5, 0x17b, 0xbe, 0x2, 0x330, 0x331, 0x5, 0x19d, - 0xcf, 0x2, 0x331, 0x5e, 0x3, 0x2, 0x2, 0x2, 0x332, 0x333, 0x5, 0x17d, - 0xbf, 0x2, 0x333, 0x334, 0x5, 0x187, 0xc4, 0x2, 0x334, 0x335, 0x5, 0x19b, - 0xce, 0x2, 0x335, 0x336, 0x5, 0x19d, 0xcf, 0x2, 0x336, 0x337, 0x5, 0x199, - 0xcd, 0x2, 0x337, 0x338, 0x5, 0x187, 0xc4, 0x2, 0x338, 0x339, 0x5, 0x179, - 0xbd, 0x2, 0x339, 0x33a, 0x5, 0x19f, 0xd0, 0x2, 0x33a, 0x33b, 0x5, 0x19d, - 0xcf, 0x2, 0x33b, 0x33c, 0x5, 0x17f, 0xc0, 0x2, 0x33c, 0x33d, 0x5, 0x17d, - 0xbf, 0x2, 0x33d, 0x60, 0x3, 0x2, 0x2, 0x2, 0x33e, 0x33f, 0x5, 0x17d, - 0xbf, 0x2, 0x33f, 0x340, 0x5, 0x199, 0xcd, 0x2, 0x340, 0x341, 0x5, 0x193, - 0xca, 0x2, 0x341, 0x342, 0x5, 0x195, 0xcb, 0x2, 0x342, 0x62, 0x3, 0x2, - 0x2, 0x2, 0x343, 0x344, 0x5, 0x17f, 0xc0, 0x2, 0x344, 0x345, 0x5, 0x18d, - 0xc7, 0x2, 0x345, 0x346, 0x5, 0x19b, 0xce, 0x2, 0x346, 0x347, 0x5, 0x17f, - 0xc0, 0x2, 0x347, 0x64, 0x3, 0x2, 0x2, 0x2, 0x348, 0x349, 0x5, 0x17f, - 0xc0, 0x2, 0x349, 0x34a, 0x5, 0x191, 0xc9, 0x2, 0x34a, 0x34b, 0x5, 0x17d, - 0xbf, 0x2, 0x34b, 0x66, 0x3, 0x2, 0x2, 0x2, 0x34c, 0x34d, 0x5, 0x17f, - 0xc0, 0x2, 0x34d, 0x34e, 0x5, 0x191, 0xc9, 0x2, 0x34e, 0x34f, 0x5, 0x183, - 0xc2, 0x2, 0x34f, 0x350, 0x5, 0x187, 0xc4, 0x2, 0x350, 0x351, 0x5, 0x191, - 0xc9, 0x2, 0x351, 0x352, 0x5, 0x17f, 0xc0, 0x2, 0x352, 0x68, 0x3, 0x2, - 0x2, 0x2, 0x353, 0x354, 0x5, 0x17f, 0xc0, 0x2, 0x354, 0x355, 0x5, 0x1a1, - 0xd1, 0x2, 0x355, 0x356, 0x5, 0x17f, 0xc0, 0x2, 0x356, 0x357, 0x5, 0x191, - 0xc9, 0x2, 0x357, 0x358, 0x5, 0x19d, 0xcf, 0x2, 0x358, 0x359, 0x5, 0x19b, - 0xce, 0x2, 0x359, 0x6a, 0x3, 0x2, 0x2, 0x2, 0x35a, 0x35b, 0x5, 0x17f, - 0xc0, 0x2, 0x35b, 0x35c, 0x5, 0x1a5, 0xd3, 0x2, 0x35c, 0x35d, 0x5, 0x187, - 0xc4, 0x2, 0x35d, 0x35e, 0x5, 0x19b, 0xce, 0x2, 0x35e, 0x35f, 0x5, 0x19d, - 0xcf, 0x2, 0x35f, 0x360, 0x5, 0x19b, 0xce, 0x2, 0x360, 0x6c, 0x3, 0x2, - 0x2, 0x2, 0x361, 0x362, 0x5, 0x17f, 0xc0, 0x2, 0x362, 0x363, 0x5, 0x1a5, - 0xd3, 0x2, 0x363, 0x364, 0x5, 0x195, 0xcb, 0x2, 0x364, 0x365, 0x5, 0x18d, - 0xc7, 0x2, 0x365, 0x366, 0x5, 0x177, 0xbc, 0x2, 0x366, 0x367, 0x5, 0x187, - 0xc4, 0x2, 0x367, 0x368, 0x5, 0x191, 0xc9, 0x2, 0x368, 0x6e, 0x3, 0x2, - 0x2, 0x2, 0x369, 0x36a, 0x5, 0x17f, 0xc0, 0x2, 0x36a, 0x36b, 0x5, 0x1a5, - 0xd3, 0x2, 0x36b, 0x36c, 0x5, 0x195, 0xcb, 0x2, 0x36c, 0x36d, 0x5, 0x199, - 0xcd, 0x2, 0x36d, 0x36e, 0x5, 0x17f, 0xc0, 0x2, 0x36e, 0x36f, 0x5, 0x19b, - 0xce, 0x2, 0x36f, 0x370, 0x5, 0x19b, 0xce, 0x2, 0x370, 0x371, 0x5, 0x187, - 0xc4, 0x2, 0x371, 0x372, 0x5, 0x193, 0xca, 0x2, 0x372, 0x373, 0x5, 0x191, - 0xc9, 0x2, 0x373, 0x70, 0x3, 0x2, 0x2, 0x2, 0x374, 0x375, 0x5, 0x17f, - 0xc0, 0x2, 0x375, 0x376, 0x5, 0x1a5, 0xd3, 0x2, 0x376, 0x377, 0x5, 0x19d, - 0xcf, 0x2, 0x377, 0x378, 0x5, 0x199, 0xcd, 0x2, 0x378, 0x379, 0x5, 0x177, - 0xbc, 0x2, 0x379, 0x37a, 0x5, 0x17b, 0xbe, 0x2, 0x37a, 0x37b, 0x5, 0x19d, - 0xcf, 0x2, 0x37b, 0x72, 0x3, 0x2, 0x2, 0x2, 0x37c, 0x37d, 0x5, 0x181, - 0xc1, 0x2, 0x37d, 0x37e, 0x5, 0x17f, 0xc0, 0x2, 0x37e, 0x37f, 0x5, 0x19d, - 0xcf, 0x2, 0x37f, 0x380, 0x5, 0x17b, 0xbe, 0x2, 0x380, 0x381, 0x5, 0x185, - 0xc3, 0x2, 0x381, 0x382, 0x5, 0x17f, 0xc0, 0x2, 0x382, 0x383, 0x5, 0x19b, - 0xce, 0x2, 0x383, 0x74, 0x3, 0x2, 0x2, 0x2, 0x384, 0x385, 0x5, 0x181, - 0xc1, 0x2, 0x385, 0x386, 0x5, 0x187, 0xc4, 0x2, 0x386, 0x387, 0x5, 0x191, - 0xc9, 0x2, 0x387, 0x388, 0x5, 0x177, 0xbc, 0x2, 0x388, 0x389, 0x5, 0x18d, - 0xc7, 0x2, 0x389, 0x76, 0x3, 0x2, 0x2, 0x2, 0x38a, 0x38b, 0x5, 0x181, - 0xc1, 0x2, 0x38b, 0x38c, 0x5, 0x187, 0xc4, 0x2, 0x38c, 0x38d, 0x5, 0x199, - 0xcd, 0x2, 0x38d, 0x38e, 0x5, 0x19b, 0xce, 0x2, 0x38e, 0x38f, 0x5, 0x19d, - 0xcf, 0x2, 0x38f, 0x78, 0x3, 0x2, 0x2, 0x2, 0x390, 0x391, 0x5, 0x181, - 0xc1, 0x2, 0x391, 0x392, 0x5, 0x18d, 0xc7, 0x2, 0x392, 0x393, 0x5, 0x19f, - 0xd0, 0x2, 0x393, 0x394, 0x5, 0x19b, 0xce, 0x2, 0x394, 0x395, 0x5, 0x185, - 0xc3, 0x2, 0x395, 0x7a, 0x3, 0x2, 0x2, 0x2, 0x396, 0x397, 0x5, 0x181, - 0xc1, 0x2, 0x397, 0x398, 0x5, 0x193, 0xca, 0x2, 0x398, 0x399, 0x5, 0x199, - 0xcd, 0x2, 0x399, 0x7c, 0x3, 0x2, 0x2, 0x2, 0x39a, 0x39b, 0x5, 0x181, - 0xc1, 0x2, 0x39b, 0x39c, 0x5, 0x193, 0xca, 0x2, 0x39c, 0x39d, 0x5, 0x199, - 0xcd, 0x2, 0x39d, 0x39e, 0x5, 0x18f, 0xc8, 0x2, 0x39e, 0x39f, 0x5, 0x177, - 0xbc, 0x2, 0x39f, 0x3a0, 0x5, 0x19d, 0xcf, 0x2, 0x3a0, 0x7e, 0x3, 0x2, - 0x2, 0x2, 0x3a1, 0x3a2, 0x5, 0x181, 0xc1, 0x2, 0x3a2, 0x3a3, 0x5, 0x199, - 0xcd, 0x2, 0x3a3, 0x3a4, 0x5, 0x17f, 0xc0, 0x2, 0x3a4, 0x3a5, 0x5, 0x17f, - 0xc0, 0x2, 0x3a5, 0x3a6, 0x5, 0x1a9, 0xd5, 0x2, 0x3a6, 0x3a7, 0x5, 0x17f, - 0xc0, 0x2, 0x3a7, 0x80, 0x3, 0x2, 0x2, 0x2, 0x3a8, 0x3a9, 0x5, 0x181, - 0xc1, 0x2, 0x3a9, 0x3aa, 0x5, 0x199, 0xcd, 0x2, 0x3aa, 0x3ab, 0x5, 0x193, - 0xca, 0x2, 0x3ab, 0x3ac, 0x5, 0x18f, 0xc8, 0x2, 0x3ac, 0x82, 0x3, 0x2, - 0x2, 0x2, 0x3ad, 0x3ae, 0x5, 0x181, 0xc1, 0x2, 0x3ae, 0x3af, 0x5, 0x19f, - 0xd0, 0x2, 0x3af, 0x3b0, 0x5, 0x18d, 0xc7, 0x2, 0x3b0, 0x3b1, 0x5, 0x18d, - 0xc7, 0x2, 0x3b1, 0x84, 0x3, 0x2, 0x2, 0x2, 0x3b2, 0x3b3, 0x5, 0x181, - 0xc1, 0x2, 0x3b3, 0x3b4, 0x5, 0x19f, 0xd0, 0x2, 0x3b4, 0x3b5, 0x5, 0x191, - 0xc9, 0x2, 0x3b5, 0x3b6, 0x5, 0x17b, 0xbe, 0x2, 0x3b6, 0x3b7, 0x5, 0x19d, - 0xcf, 0x2, 0x3b7, 0x3b8, 0x5, 0x187, 0xc4, 0x2, 0x3b8, 0x3b9, 0x5, 0x193, - 0xca, 0x2, 0x3b9, 0x3ba, 0x5, 0x191, 0xc9, 0x2, 0x3ba, 0x86, 0x3, 0x2, - 0x2, 0x2, 0x3bb, 0x3bc, 0x5, 0x183, 0xc2, 0x2, 0x3bc, 0x3bd, 0x5, 0x18d, - 0xc7, 0x2, 0x3bd, 0x3be, 0x5, 0x193, 0xca, 0x2, 0x3be, 0x3bf, 0x5, 0x179, - 0xbd, 0x2, 0x3bf, 0x3c0, 0x5, 0x177, 0xbc, 0x2, 0x3c0, 0x3c1, 0x5, 0x18d, - 0xc7, 0x2, 0x3c1, 0x88, 0x3, 0x2, 0x2, 0x2, 0x3c2, 0x3c3, 0x5, 0x183, - 0xc2, 0x2, 0x3c3, 0x3c4, 0x5, 0x199, 0xcd, 0x2, 0x3c4, 0x3c5, 0x5, 0x177, - 0xbc, 0x2, 0x3c5, 0x3c6, 0x5, 0x191, 0xc9, 0x2, 0x3c6, 0x3c7, 0x5, 0x19f, - 0xd0, 0x2, 0x3c7, 0x3c8, 0x5, 0x18d, 0xc7, 0x2, 0x3c8, 0x3c9, 0x5, 0x177, - 0xbc, 0x2, 0x3c9, 0x3ca, 0x5, 0x199, 0xcd, 0x2, 0x3ca, 0x3cb, 0x5, 0x187, - 0xc4, 0x2, 0x3cb, 0x3cc, 0x5, 0x19d, 0xcf, 0x2, 0x3cc, 0x3cd, 0x5, 0x1a7, - 0xd4, 0x2, 0x3cd, 0x8a, 0x3, 0x2, 0x2, 0x2, 0x3ce, 0x3cf, 0x5, 0x183, - 0xc2, 0x2, 0x3cf, 0x3d0, 0x5, 0x199, 0xcd, 0x2, 0x3d0, 0x3d1, 0x5, 0x193, - 0xca, 0x2, 0x3d1, 0x3d2, 0x5, 0x19f, 0xd0, 0x2, 0x3d2, 0x3d3, 0x5, 0x195, - 0xcb, 0x2, 0x3d3, 0x8c, 0x3, 0x2, 0x2, 0x2, 0x3d4, 0x3d5, 0x5, 0x185, - 0xc3, 0x2, 0x3d5, 0x3d6, 0x5, 0x177, 0xbc, 0x2, 0x3d6, 0x3d7, 0x5, 0x1a1, - 0xd1, 0x2, 0x3d7, 0x3d8, 0x5, 0x187, 0xc4, 0x2, 0x3d8, 0x3d9, 0x5, 0x191, - 0xc9, 0x2, 0x3d9, 0x3da, 0x5, 0x183, 0xc2, 0x2, 0x3da, 0x8e, 0x3, 0x2, - 0x2, 0x2, 0x3db, 0x3dc, 0x5, 0x185, 0xc3, 0x2, 0x3dc, 0x3dd, 0x5, 0x187, - 0xc4, 0x2, 0x3dd, 0x3de, 0x5, 0x17f, 0xc0, 0x2, 0x3de, 0x3df, 0x5, 0x199, - 0xcd, 0x2, 0x3df, 0x3e0, 0x5, 0x177, 0xbc, 0x2, 0x3e0, 0x3e1, 0x5, 0x199, - 0xcd, 0x2, 0x3e1, 0x3e2, 0x5, 0x17b, 0xbe, 0x2, 0x3e2, 0x3e3, 0x5, 0x185, - 0xc3, 0x2, 0x3e3, 0x3e4, 0x5, 0x187, 0xc4, 0x2, 0x3e4, 0x3e5, 0x5, 0x17b, - 0xbe, 0x2, 0x3e5, 0x3e6, 0x5, 0x177, 0xbc, 0x2, 0x3e6, 0x3e7, 0x5, 0x18d, - 0xc7, 0x2, 0x3e7, 0x90, 0x3, 0x2, 0x2, 0x2, 0x3e8, 0x3e9, 0x5, 0x185, - 0xc3, 0x2, 0x3e9, 0x3ea, 0x5, 0x193, 0xca, 0x2, 0x3ea, 0x3eb, 0x5, 0x19f, - 0xd0, 0x2, 0x3eb, 0x3ec, 0x5, 0x199, 0xcd, 0x2, 0x3ec, 0x92, 0x3, 0x2, - 0x2, 0x2, 0x3ed, 0x3ee, 0x5, 0x187, 0xc4, 0x2, 0x3ee, 0x3ef, 0x5, 0x17d, - 0xbf, 0x2, 0x3ef, 0x94, 0x3, 0x2, 0x2, 0x2, 0x3f0, 0x3f1, 0x5, 0x187, - 0xc4, 0x2, 0x3f1, 0x3f2, 0x5, 0x181, 0xc1, 0x2, 0x3f2, 0x96, 0x3, 0x2, - 0x2, 0x2, 0x3f3, 0x3f4, 0x5, 0x187, 0xc4, 0x2, 0x3f4, 0x3f5, 0x5, 0x18d, - 0xc7, 0x2, 0x3f5, 0x3f6, 0x5, 0x187, 0xc4, 0x2, 0x3f6, 0x3f7, 0x5, 0x18b, - 0xc6, 0x2, 0x3f7, 0x3f8, 0x5, 0x17f, 0xc0, 0x2, 0x3f8, 0x98, 0x3, 0x2, - 0x2, 0x2, 0x3f9, 0x3fa, 0x5, 0x187, 0xc4, 0x2, 0x3fa, 0x3fb, 0x5, 0x191, - 0xc9, 0x2, 0x3fb, 0x9a, 0x3, 0x2, 0x2, 0x2, 0x3fc, 0x3fd, 0x5, 0x187, - 0xc4, 0x2, 0x3fd, 0x3fe, 0x5, 0x191, 0xc9, 0x2, 0x3fe, 0x3ff, 0x5, 0x17d, - 0xbf, 0x2, 0x3ff, 0x400, 0x5, 0x17f, 0xc0, 0x2, 0x400, 0x401, 0x5, 0x1a5, - 0xd3, 0x2, 0x401, 0x9c, 0x3, 0x2, 0x2, 0x2, 0x402, 0x403, 0x5, 0x187, - 0xc4, 0x2, 0x403, 0x404, 0x5, 0x191, 0xc9, 0x2, 0x404, 0x405, 0x5, 0x181, - 0xc1, 0x2, 0x405, 0x410, 0x3, 0x2, 0x2, 0x2, 0x406, 0x407, 0x5, 0x187, - 0xc4, 0x2, 0x407, 0x408, 0x5, 0x191, 0xc9, 0x2, 0x408, 0x409, 0x5, 0x181, - 0xc1, 0x2, 0x409, 0x40a, 0x5, 0x187, 0xc4, 0x2, 0x40a, 0x40b, 0x5, 0x191, - 0xc9, 0x2, 0x40b, 0x40c, 0x5, 0x187, 0xc4, 0x2, 0x40c, 0x40d, 0x5, 0x19d, - 0xcf, 0x2, 0x40d, 0x40e, 0x5, 0x1a7, 0xd4, 0x2, 0x40e, 0x410, 0x3, 0x2, - 0x2, 0x2, 0x40f, 0x402, 0x3, 0x2, 0x2, 0x2, 0x40f, 0x406, 0x3, 0x2, - 0x2, 0x2, 0x410, 0x9e, 0x3, 0x2, 0x2, 0x2, 0x411, 0x412, 0x5, 0x187, - 0xc4, 0x2, 0x412, 0x413, 0x5, 0x191, 0xc9, 0x2, 0x413, 0x414, 0x5, 0x189, - 0xc5, 0x2, 0x414, 0x415, 0x5, 0x17f, 0xc0, 0x2, 0x415, 0x416, 0x5, 0x17b, - 0xbe, 0x2, 0x416, 0x417, 0x5, 0x19d, 0xcf, 0x2, 0x417, 0x418, 0x5, 0x187, - 0xc4, 0x2, 0x418, 0x419, 0x5, 0x1a1, 0xd1, 0x2, 0x419, 0x41a, 0x5, 0x17f, - 0xc0, 0x2, 0x41a, 0xa0, 0x3, 0x2, 0x2, 0x2, 0x41b, 0x41c, 0x5, 0x187, - 0xc4, 0x2, 0x41c, 0x41d, 0x5, 0x191, 0xc9, 0x2, 0x41d, 0x41e, 0x5, 0x191, - 0xc9, 0x2, 0x41e, 0x41f, 0x5, 0x17f, 0xc0, 0x2, 0x41f, 0x420, 0x5, 0x199, - 0xcd, 0x2, 0x420, 0xa2, 0x3, 0x2, 0x2, 0x2, 0x421, 0x422, 0x5, 0x187, - 0xc4, 0x2, 0x422, 0x423, 0x5, 0x191, 0xc9, 0x2, 0x423, 0x424, 0x5, 0x19b, - 0xce, 0x2, 0x424, 0x425, 0x5, 0x17f, 0xc0, 0x2, 0x425, 0x426, 0x5, 0x199, - 0xcd, 0x2, 0x426, 0x427, 0x5, 0x19d, 0xcf, 0x2, 0x427, 0xa4, 0x3, 0x2, - 0x2, 0x2, 0x428, 0x429, 0x5, 0x187, 0xc4, 0x2, 0x429, 0x42a, 0x5, 0x191, - 0xc9, 0x2, 0x42a, 0x42b, 0x5, 0x19d, 0xcf, 0x2, 0x42b, 0x42c, 0x5, 0x17f, - 0xc0, 0x2, 0x42c, 0x42d, 0x5, 0x199, 0xcd, 0x2, 0x42d, 0x42e, 0x5, 0x1a1, - 0xd1, 0x2, 0x42e, 0x42f, 0x5, 0x177, 0xbc, 0x2, 0x42f, 0x430, 0x5, 0x18d, - 0xc7, 0x2, 0x430, 0xa6, 0x3, 0x2, 0x2, 0x2, 0x431, 0x432, 0x5, 0x187, - 0xc4, 0x2, 0x432, 0x433, 0x5, 0x191, 0xc9, 0x2, 0x433, 0x434, 0x5, 0x19d, - 0xcf, 0x2, 0x434, 0x435, 0x5, 0x193, 0xca, 0x2, 0x435, 0xa8, 0x3, 0x2, - 0x2, 0x2, 0x436, 0x437, 0x5, 0x187, 0xc4, 0x2, 0x437, 0x438, 0x5, 0x19b, - 0xce, 0x2, 0x438, 0xaa, 0x3, 0x2, 0x2, 0x2, 0x439, 0x43a, 0x5, 0x187, - 0xc4, 0x2, 0x43a, 0x43b, 0x5, 0x19b, 0xce, 0x2, 0x43b, 0x43c, 0x5, 0x1ed, - 0xf7, 0x2, 0x43c, 0x43d, 0x5, 0x193, 0xca, 0x2, 0x43d, 0x43e, 0x5, 0x179, - 0xbd, 0x2, 0x43e, 0x43f, 0x5, 0x189, 0xc5, 0x2, 0x43f, 0x440, 0x5, 0x17f, - 0xc0, 0x2, 0x440, 0x441, 0x5, 0x17b, 0xbe, 0x2, 0x441, 0x442, 0x5, 0x19d, - 0xcf, 0x2, 0x442, 0x443, 0x5, 0x1ed, 0xf7, 0x2, 0x443, 0x444, 0x5, 0x187, - 0xc4, 0x2, 0x444, 0x445, 0x5, 0x17d, 0xbf, 0x2, 0x445, 0xac, 0x3, 0x2, - 0x2, 0x2, 0x446, 0x447, 0x5, 0x189, 0xc5, 0x2, 0x447, 0x448, 0x5, 0x193, - 0xca, 0x2, 0x448, 0x449, 0x5, 0x187, 0xc4, 0x2, 0x449, 0x44a, 0x5, 0x191, - 0xc9, 0x2, 0x44a, 0xae, 0x3, 0x2, 0x2, 0x2, 0x44b, 0x44c, 0x5, 0x18b, - 0xc6, 0x2, 0x44c, 0x44d, 0x5, 0x17f, 0xc0, 0x2, 0x44d, 0x44e, 0x5, 0x1a7, - 0xd4, 0x2, 0x44e, 0xb0, 0x3, 0x2, 0x2, 0x2, 0x44f, 0x450, 0x5, 0x18b, - 0xc6, 0x2, 0x450, 0x451, 0x5, 0x187, 0xc4, 0x2, 0x451, 0x452, 0x5, 0x18d, - 0xc7, 0x2, 0x452, 0x453, 0x5, 0x18d, 0xc7, 0x2, 0x453, 0xb2, 0x3, 0x2, - 0x2, 0x2, 0x454, 0x455, 0x5, 0x18d, 0xc7, 0x2, 0x455, 0x456, 0x5, 0x177, - 0xbc, 0x2, 0x456, 0x457, 0x5, 0x19b, 0xce, 0x2, 0x457, 0x458, 0x5, 0x19d, - 0xcf, 0x2, 0x458, 0xb4, 0x3, 0x2, 0x2, 0x2, 0x459, 0x45a, 0x5, 0x18d, - 0xc7, 0x2, 0x45a, 0x45b, 0x5, 0x177, 0xbc, 0x2, 0x45b, 0x45c, 0x5, 0x1a7, - 0xd4, 0x2, 0x45c, 0x45d, 0x5, 0x193, 0xca, 0x2, 0x45d, 0x45e, 0x5, 0x19f, - 0xd0, 0x2, 0x45e, 0x45f, 0x5, 0x19d, 0xcf, 0x2, 0x45f, 0xb6, 0x3, 0x2, - 0x2, 0x2, 0x460, 0x461, 0x5, 0x18d, 0xc7, 0x2, 0x461, 0x462, 0x5, 0x17f, - 0xc0, 0x2, 0x462, 0x463, 0x5, 0x177, 0xbc, 0x2, 0x463, 0x464, 0x5, 0x17d, - 0xbf, 0x2, 0x464, 0x465, 0x5, 0x187, 0xc4, 0x2, 0x465, 0x466, 0x5, 0x191, - 0xc9, 0x2, 0x466, 0x467, 0x5, 0x183, 0xc2, 0x2, 0x467, 0xb8, 0x3, 0x2, - 0x2, 0x2, 0x468, 0x469, 0x5, 0x18d, 0xc7, 0x2, 0x469, 0x46a, 0x5, 0x17f, - 0xc0, 0x2, 0x46a, 0x46b, 0x5, 0x181, 0xc1, 0x2, 0x46b, 0x46c, 0x5, 0x19d, - 0xcf, 0x2, 0x46c, 0xba, 0x3, 0x2, 0x2, 0x2, 0x46d, 0x46e, 0x5, 0x18d, - 0xc7, 0x2, 0x46e, 0x46f, 0x5, 0x187, 0xc4, 0x2, 0x46f, 0x470, 0x5, 0x181, - 0xc1, 0x2, 0x470, 0x471, 0x5, 0x17f, 0xc0, 0x2, 0x471, 0x472, 0x5, 0x19d, - 0xcf, 0x2, 0x472, 0x473, 0x5, 0x187, 0xc4, 0x2, 0x473, 0x474, 0x5, 0x18f, - 0xc8, 0x2, 0x474, 0x475, 0x5, 0x17f, 0xc0, 0x2, 0x475, 0xbc, 0x3, 0x2, - 0x2, 0x2, 0x476, 0x477, 0x5, 0x18d, 0xc7, 0x2, 0x477, 0x478, 0x5, 0x187, - 0xc4, 0x2, 0x478, 0x479, 0x5, 0x18b, 0xc6, 0x2, 0x479, 0x47a, 0x5, 0x17f, - 0xc0, 0x2, 0x47a, 0xbe, 0x3, 0x2, 0x2, 0x2, 0x47b, 0x47c, 0x5, 0x18d, - 0xc7, 0x2, 0x47c, 0x47d, 0x5, 0x187, 0xc4, 0x2, 0x47d, 0x47e, 0x5, 0x18f, - 0xc8, 0x2, 0x47e, 0x47f, 0x5, 0x187, 0xc4, 0x2, 0x47f, 0x480, 0x5, 0x19d, - 0xcf, 0x2, 0x480, 0xc0, 0x3, 0x2, 0x2, 0x2, 0x481, 0x482, 0x5, 0x18d, - 0xc7, 0x2, 0x482, 0x483, 0x5, 0x187, 0xc4, 0x2, 0x483, 0x484, 0x5, 0x1a1, - 0xd1, 0x2, 0x484, 0x485, 0x5, 0x17f, 0xc0, 0x2, 0x485, 0xc2, 0x3, 0x2, - 0x2, 0x2, 0x486, 0x487, 0x5, 0x18d, 0xc7, 0x2, 0x487, 0x488, 0x5, 0x193, - 0xca, 0x2, 0x488, 0x489, 0x5, 0x17b, 0xbe, 0x2, 0x489, 0x48a, 0x5, 0x177, - 0xbc, 0x2, 0x48a, 0x48b, 0x5, 0x18d, 0xc7, 0x2, 0x48b, 0xc4, 0x3, 0x2, - 0x2, 0x2, 0x48c, 0x48d, 0x5, 0x18d, 0xc7, 0x2, 0x48d, 0x48e, 0x5, 0x193, - 0xca, 0x2, 0x48e, 0x48f, 0x5, 0x183, 0xc2, 0x2, 0x48f, 0x490, 0x5, 0x19b, - 0xce, 0x2, 0x490, 0xc6, 0x3, 0x2, 0x2, 0x2, 0x491, 0x492, 0x5, 0x18f, - 0xc8, 0x2, 0x492, 0x493, 0x5, 0x177, 0xbc, 0x2, 0x493, 0x494, 0x5, 0x19d, - 0xcf, 0x2, 0x494, 0x495, 0x5, 0x17f, 0xc0, 0x2, 0x495, 0x496, 0x5, 0x199, - 0xcd, 0x2, 0x496, 0x497, 0x5, 0x187, 0xc4, 0x2, 0x497, 0x498, 0x5, 0x177, - 0xbc, 0x2, 0x498, 0x499, 0x5, 0x18d, 0xc7, 0x2, 0x499, 0x49a, 0x5, 0x187, - 0xc4, 0x2, 0x49a, 0x49b, 0x5, 0x1a9, 0xd5, 0x2, 0x49b, 0x49c, 0x5, 0x17f, - 0xc0, 0x2, 0x49c, 0x49d, 0x5, 0x17d, 0xbf, 0x2, 0x49d, 0xc8, 0x3, 0x2, - 0x2, 0x2, 0x49e, 0x49f, 0x5, 0x18f, 0xc8, 0x2, 0x49f, 0x4a0, 0x5, 0x177, - 0xbc, 0x2, 0x4a0, 0x4a1, 0x5, 0x1a5, 0xd3, 0x2, 0x4a1, 0xca, 0x3, 0x2, - 0x2, 0x2, 0x4a2, 0x4a3, 0x5, 0x18f, 0xc8, 0x2, 0x4a3, 0x4a4, 0x5, 0x17f, - 0xc0, 0x2, 0x4a4, 0x4a5, 0x5, 0x199, 0xcd, 0x2, 0x4a5, 0x4a6, 0x5, 0x183, - 0xc2, 0x2, 0x4a6, 0x4a7, 0x5, 0x17f, 0xc0, 0x2, 0x4a7, 0x4a8, 0x5, 0x19b, - 0xce, 0x2, 0x4a8, 0xcc, 0x3, 0x2, 0x2, 0x2, 0x4a9, 0x4aa, 0x5, 0x18f, - 0xc8, 0x2, 0x4aa, 0x4ab, 0x5, 0x187, 0xc4, 0x2, 0x4ab, 0x4ac, 0x5, 0x191, - 0xc9, 0x2, 0x4ac, 0xce, 0x3, 0x2, 0x2, 0x2, 0x4ad, 0x4ae, 0x5, 0x18f, - 0xc8, 0x2, 0x4ae, 0x4af, 0x5, 0x187, 0xc4, 0x2, 0x4af, 0x4b0, 0x5, 0x191, - 0xc9, 0x2, 0x4b0, 0x4b1, 0x5, 0x19f, 0xd0, 0x2, 0x4b1, 0x4b2, 0x5, 0x19d, - 0xcf, 0x2, 0x4b2, 0x4b3, 0x5, 0x17f, 0xc0, 0x2, 0x4b3, 0xd0, 0x3, 0x2, - 0x2, 0x2, 0x4b4, 0x4b5, 0x5, 0x18f, 0xc8, 0x2, 0x4b5, 0x4b6, 0x5, 0x193, - 0xca, 0x2, 0x4b6, 0x4b7, 0x5, 0x17d, 0xbf, 0x2, 0x4b7, 0x4b8, 0x5, 0x187, - 0xc4, 0x2, 0x4b8, 0x4b9, 0x5, 0x181, 0xc1, 0x2, 0x4b9, 0x4ba, 0x5, 0x1a7, - 0xd4, 0x2, 0x4ba, 0xd2, 0x3, 0x2, 0x2, 0x2, 0x4bb, 0x4bc, 0x5, 0x18f, - 0xc8, 0x2, 0x4bc, 0x4bd, 0x5, 0x193, 0xca, 0x2, 0x4bd, 0x4be, 0x5, 0x191, - 0xc9, 0x2, 0x4be, 0x4bf, 0x5, 0x19d, 0xcf, 0x2, 0x4bf, 0x4c0, 0x5, 0x185, - 0xc3, 0x2, 0x4c0, 0xd4, 0x3, 0x2, 0x2, 0x2, 0x4c1, 0x4c2, 0x5, 0x18f, - 0xc8, 0x2, 0x4c2, 0x4c3, 0x5, 0x193, 0xca, 0x2, 0x4c3, 0x4c4, 0x5, 0x1a1, - 0xd1, 0x2, 0x4c4, 0x4c5, 0x5, 0x17f, 0xc0, 0x2, 0x4c5, 0xd6, 0x3, 0x2, - 0x2, 0x2, 0x4c6, 0x4c7, 0x5, 0x18f, 0xc8, 0x2, 0x4c7, 0x4c8, 0x5, 0x19f, - 0xd0, 0x2, 0x4c8, 0x4c9, 0x5, 0x19d, 0xcf, 0x2, 0x4c9, 0x4ca, 0x5, 0x177, - 0xbc, 0x2, 0x4ca, 0x4cb, 0x5, 0x19d, 0xcf, 0x2, 0x4cb, 0x4cc, 0x5, 0x187, - 0xc4, 0x2, 0x4cc, 0x4cd, 0x5, 0x193, 0xca, 0x2, 0x4cd, 0x4ce, 0x5, 0x191, - 0xc9, 0x2, 0x4ce, 0xd8, 0x3, 0x2, 0x2, 0x2, 0x4cf, 0x4d0, 0x5, 0x191, - 0xc9, 0x2, 0x4d0, 0x4d1, 0x5, 0x177, 0xbc, 0x2, 0x4d1, 0x4d2, 0x5, 0x191, - 0xc9, 0x2, 0x4d2, 0xda, 0x3, 0x2, 0x2, 0x2, 0x4d3, 0x4d4, 0x5, 0x191, - 0xc9, 0x2, 0x4d4, 0x4d5, 0x5, 0x193, 0xca, 0x2, 0x4d5, 0xdc, 0x3, 0x2, - 0x2, 0x2, 0x4d6, 0x4d7, 0x5, 0x191, 0xc9, 0x2, 0x4d7, 0x4d8, 0x5, 0x193, - 0xca, 0x2, 0x4d8, 0x4d9, 0x5, 0x19d, 0xcf, 0x2, 0x4d9, 0xde, 0x3, 0x2, - 0x2, 0x2, 0x4da, 0x4db, 0x5, 0x191, 0xc9, 0x2, 0x4db, 0x4dc, 0x5, 0x19f, - 0xd0, 0x2, 0x4dc, 0x4dd, 0x5, 0x18d, 0xc7, 0x2, 0x4dd, 0x4de, 0x5, 0x18d, - 0xc7, 0x2, 0x4de, 0xe0, 0x3, 0x2, 0x2, 0x2, 0x4df, 0x4e0, 0x5, 0x191, - 0xc9, 0x2, 0x4e0, 0x4e1, 0x5, 0x19f, 0xd0, 0x2, 0x4e1, 0x4e2, 0x5, 0x18d, - 0xc7, 0x2, 0x4e2, 0x4e3, 0x5, 0x18d, 0xc7, 0x2, 0x4e3, 0x4e4, 0x5, 0x19b, - 0xce, 0x2, 0x4e4, 0xe2, 0x3, 0x2, 0x2, 0x2, 0x4e5, 0x4e6, 0x5, 0x193, - 0xca, 0x2, 0x4e6, 0x4e7, 0x5, 0x181, 0xc1, 0x2, 0x4e7, 0x4e8, 0x5, 0x181, - 0xc1, 0x2, 0x4e8, 0x4e9, 0x5, 0x19b, 0xce, 0x2, 0x4e9, 0x4ea, 0x5, 0x17f, - 0xc0, 0x2, 0x4ea, 0x4eb, 0x5, 0x19d, 0xcf, 0x2, 0x4eb, 0xe4, 0x3, 0x2, - 0x2, 0x2, 0x4ec, 0x4ed, 0x5, 0x193, 0xca, 0x2, 0x4ed, 0x4ee, 0x5, 0x191, - 0xc9, 0x2, 0x4ee, 0xe6, 0x3, 0x2, 0x2, 0x2, 0x4ef, 0x4f0, 0x5, 0x193, - 0xca, 0x2, 0x4f0, 0x4f1, 0x5, 0x195, 0xcb, 0x2, 0x4f1, 0x4f2, 0x5, 0x19d, - 0xcf, 0x2, 0x4f2, 0x4f3, 0x5, 0x187, 0xc4, 0x2, 0x4f3, 0x4f4, 0x5, 0x18f, - 0xc8, 0x2, 0x4f4, 0x4f5, 0x5, 0x187, 0xc4, 0x2, 0x4f5, 0x4f6, 0x5, 0x1a9, - 0xd5, 0x2, 0x4f6, 0x4f7, 0x5, 0x17f, 0xc0, 0x2, 0x4f7, 0xe8, 0x3, 0x2, - 0x2, 0x2, 0x4f8, 0x4f9, 0x5, 0x193, 0xca, 0x2, 0x4f9, 0x4fa, 0x5, 0x199, - 0xcd, 0x2, 0x4fa, 0xea, 0x3, 0x2, 0x2, 0x2, 0x4fb, 0x4fc, 0x5, 0x193, - 0xca, 0x2, 0x4fc, 0x4fd, 0x5, 0x199, 0xcd, 0x2, 0x4fd, 0x4fe, 0x5, 0x17d, - 0xbf, 0x2, 0x4fe, 0x4ff, 0x5, 0x17f, 0xc0, 0x2, 0x4ff, 0x500, 0x5, 0x199, - 0xcd, 0x2, 0x500, 0xec, 0x3, 0x2, 0x2, 0x2, 0x501, 0x502, 0x5, 0x193, - 0xca, 0x2, 0x502, 0x503, 0x5, 0x19f, 0xd0, 0x2, 0x503, 0x504, 0x5, 0x19d, - 0xcf, 0x2, 0x504, 0x505, 0x5, 0x17f, 0xc0, 0x2, 0x505, 0x506, 0x5, 0x199, - 0xcd, 0x2, 0x506, 0xee, 0x3, 0x2, 0x2, 0x2, 0x507, 0x508, 0x5, 0x193, - 0xca, 0x2, 0x508, 0x509, 0x5, 0x19f, 0xd0, 0x2, 0x509, 0x50a, 0x5, 0x19d, - 0xcf, 0x2, 0x50a, 0x50b, 0x5, 0x181, 0xc1, 0x2, 0x50b, 0x50c, 0x5, 0x187, - 0xc4, 0x2, 0x50c, 0x50d, 0x5, 0x18d, 0xc7, 0x2, 0x50d, 0x50e, 0x5, 0x17f, - 0xc0, 0x2, 0x50e, 0xf0, 0x3, 0x2, 0x2, 0x2, 0x50f, 0x510, 0x5, 0x195, - 0xcb, 0x2, 0x510, 0x511, 0x5, 0x177, 0xbc, 0x2, 0x511, 0x512, 0x5, 0x199, - 0xcd, 0x2, 0x512, 0x513, 0x5, 0x19d, 0xcf, 0x2, 0x513, 0x514, 0x5, 0x187, - 0xc4, 0x2, 0x514, 0x515, 0x5, 0x19d, 0xcf, 0x2, 0x515, 0x516, 0x5, 0x187, - 0xc4, 0x2, 0x516, 0x517, 0x5, 0x193, 0xca, 0x2, 0x517, 0x518, 0x5, 0x191, - 0xc9, 0x2, 0x518, 0xf2, 0x3, 0x2, 0x2, 0x2, 0x519, 0x51a, 0x5, 0x195, - 0xcb, 0x2, 0x51a, 0x51b, 0x5, 0x193, 0xca, 0x2, 0x51b, 0x51c, 0x5, 0x195, - 0xcb, 0x2, 0x51c, 0x51d, 0x5, 0x19f, 0xd0, 0x2, 0x51d, 0x51e, 0x5, 0x18d, - 0xc7, 0x2, 0x51e, 0x51f, 0x5, 0x177, 0xbc, 0x2, 0x51f, 0x520, 0x5, 0x19d, - 0xcf, 0x2, 0x520, 0x521, 0x5, 0x17f, 0xc0, 0x2, 0x521, 0xf4, 0x3, 0x2, - 0x2, 0x2, 0x522, 0x523, 0x5, 0x195, 0xcb, 0x2, 0x523, 0x524, 0x5, 0x199, - 0xcd, 0x2, 0x524, 0x525, 0x5, 0x17f, 0xc0, 0x2, 0x525, 0x526, 0x5, 0x1a3, - 0xd2, 0x2, 0x526, 0x527, 0x5, 0x185, 0xc3, 0x2, 0x527, 0x528, 0x5, 0x17f, - 0xc0, 0x2, 0x528, 0x529, 0x5, 0x199, 0xcd, 0x2, 0x529, 0x52a, 0x5, 0x17f, - 0xc0, 0x2, 0x52a, 0xf6, 0x3, 0x2, 0x2, 0x2, 0x52b, 0x52c, 0x5, 0x195, - 0xcb, 0x2, 0x52c, 0x52d, 0x5, 0x199, 0xcd, 0x2, 0x52d, 0x52e, 0x5, 0x187, - 0xc4, 0x2, 0x52e, 0x52f, 0x5, 0x18f, 0xc8, 0x2, 0x52f, 0x530, 0x5, 0x177, - 0xbc, 0x2, 0x530, 0x531, 0x5, 0x199, 0xcd, 0x2, 0x531, 0x532, 0x5, 0x1a7, - 0xd4, 0x2, 0x532, 0xf8, 0x3, 0x2, 0x2, 0x2, 0x533, 0x534, 0x5, 0x197, - 0xcc, 0x2, 0x534, 0x535, 0x5, 0x19f, 0xd0, 0x2, 0x535, 0x536, 0x5, 0x177, - 0xbc, 0x2, 0x536, 0x537, 0x5, 0x199, 0xcd, 0x2, 0x537, 0x538, 0x5, 0x19d, - 0xcf, 0x2, 0x538, 0x539, 0x5, 0x17f, 0xc0, 0x2, 0x539, 0x53a, 0x5, 0x199, - 0xcd, 0x2, 0x53a, 0xfa, 0x3, 0x2, 0x2, 0x2, 0x53b, 0x53c, 0x5, 0x199, - 0xcd, 0x2, 0x53c, 0x53d, 0x5, 0x177, 0xbc, 0x2, 0x53d, 0x53e, 0x5, 0x191, - 0xc9, 0x2, 0x53e, 0x53f, 0x5, 0x183, 0xc2, 0x2, 0x53f, 0x540, 0x5, 0x17f, - 0xc0, 0x2, 0x540, 0xfc, 0x3, 0x2, 0x2, 0x2, 0x541, 0x542, 0x5, 0x199, - 0xcd, 0x2, 0x542, 0x543, 0x5, 0x17f, 0xc0, 0x2, 0x543, 0x544, 0x5, 0x18d, - 0xc7, 0x2, 0x544, 0x545, 0x5, 0x193, 0xca, 0x2, 0x545, 0x546, 0x5, 0x177, - 0xbc, 0x2, 0x546, 0x547, 0x5, 0x17d, 0xbf, 0x2, 0x547, 0xfe, 0x3, 0x2, - 0x2, 0x2, 0x548, 0x549, 0x5, 0x199, 0xcd, 0x2, 0x549, 0x54a, 0x5, 0x17f, - 0xc0, 0x2, 0x54a, 0x54b, 0x5, 0x18f, 0xc8, 0x2, 0x54b, 0x54c, 0x5, 0x193, - 0xca, 0x2, 0x54c, 0x54d, 0x5, 0x1a1, 0xd1, 0x2, 0x54d, 0x54e, 0x5, 0x17f, - 0xc0, 0x2, 0x54e, 0x100, 0x3, 0x2, 0x2, 0x2, 0x54f, 0x550, 0x5, 0x199, - 0xcd, 0x2, 0x550, 0x551, 0x5, 0x17f, 0xc0, 0x2, 0x551, 0x552, 0x5, 0x191, - 0xc9, 0x2, 0x552, 0x553, 0x5, 0x177, 0xbc, 0x2, 0x553, 0x554, 0x5, 0x18f, - 0xc8, 0x2, 0x554, 0x555, 0x5, 0x17f, 0xc0, 0x2, 0x555, 0x102, 0x3, 0x2, - 0x2, 0x2, 0x556, 0x557, 0x5, 0x199, 0xcd, 0x2, 0x557, 0x558, 0x5, 0x17f, - 0xc0, 0x2, 0x558, 0x559, 0x5, 0x195, 0xcb, 0x2, 0x559, 0x55a, 0x5, 0x18d, - 0xc7, 0x2, 0x55a, 0x55b, 0x5, 0x177, 0xbc, 0x2, 0x55b, 0x55c, 0x5, 0x17b, - 0xbe, 0x2, 0x55c, 0x55d, 0x5, 0x17f, 0xc0, 0x2, 0x55d, 0x104, 0x3, 0x2, - 0x2, 0x2, 0x55e, 0x55f, 0x5, 0x199, 0xcd, 0x2, 0x55f, 0x560, 0x5, 0x17f, - 0xc0, 0x2, 0x560, 0x561, 0x5, 0x195, 0xcb, 0x2, 0x561, 0x562, 0x5, 0x18d, - 0xc7, 0x2, 0x562, 0x563, 0x5, 0x187, 0xc4, 0x2, 0x563, 0x564, 0x5, 0x17b, - 0xbe, 0x2, 0x564, 0x565, 0x5, 0x177, 0xbc, 0x2, 0x565, 0x106, 0x3, 0x2, - 0x2, 0x2, 0x566, 0x567, 0x5, 0x199, 0xcd, 0x2, 0x567, 0x568, 0x5, 0x17f, - 0xc0, 0x2, 0x568, 0x569, 0x5, 0x195, 0xcb, 0x2, 0x569, 0x56a, 0x5, 0x18d, - 0xc7, 0x2, 0x56a, 0x56b, 0x5, 0x187, 0xc4, 0x2, 0x56b, 0x56c, 0x5, 0x17b, - 0xbe, 0x2, 0x56c, 0x56d, 0x5, 0x177, 0xbc, 0x2, 0x56d, 0x56e, 0x5, 0x19d, - 0xcf, 0x2, 0x56e, 0x56f, 0x5, 0x17f, 0xc0, 0x2, 0x56f, 0x570, 0x5, 0x17d, - 0xbf, 0x2, 0x570, 0x108, 0x3, 0x2, 0x2, 0x2, 0x571, 0x572, 0x5, 0x199, - 0xcd, 0x2, 0x572, 0x573, 0x5, 0x187, 0xc4, 0x2, 0x573, 0x574, 0x5, 0x183, - 0xc2, 0x2, 0x574, 0x575, 0x5, 0x185, 0xc3, 0x2, 0x575, 0x576, 0x5, 0x19d, - 0xcf, 0x2, 0x576, 0x10a, 0x3, 0x2, 0x2, 0x2, 0x577, 0x578, 0x5, 0x199, - 0xcd, 0x2, 0x578, 0x579, 0x5, 0x193, 0xca, 0x2, 0x579, 0x57a, 0x5, 0x18d, - 0xc7, 0x2, 0x57a, 0x57b, 0x5, 0x18d, 0xc7, 0x2, 0x57b, 0x57c, 0x5, 0x19f, - 0xd0, 0x2, 0x57c, 0x57d, 0x5, 0x195, 0xcb, 0x2, 0x57d, 0x10c, 0x3, 0x2, - 0x2, 0x2, 0x57e, 0x57f, 0x5, 0x19b, 0xce, 0x2, 0x57f, 0x580, 0x5, 0x177, - 0xbc, 0x2, 0x580, 0x581, 0x5, 0x18f, 0xc8, 0x2, 0x581, 0x582, 0x5, 0x195, - 0xcb, 0x2, 0x582, 0x583, 0x5, 0x18d, 0xc7, 0x2, 0x583, 0x584, 0x5, 0x17f, - 0xc0, 0x2, 0x584, 0x10e, 0x3, 0x2, 0x2, 0x2, 0x585, 0x586, 0x5, 0x19b, - 0xce, 0x2, 0x586, 0x587, 0x5, 0x17f, 0xc0, 0x2, 0x587, 0x588, 0x5, 0x17b, - 0xbe, 0x2, 0x588, 0x589, 0x5, 0x193, 0xca, 0x2, 0x589, 0x58a, 0x5, 0x191, - 0xc9, 0x2, 0x58a, 0x58b, 0x5, 0x17d, 0xbf, 0x2, 0x58b, 0x110, 0x3, 0x2, - 0x2, 0x2, 0x58c, 0x58d, 0x5, 0x19b, 0xce, 0x2, 0x58d, 0x58e, 0x5, 0x17f, - 0xc0, 0x2, 0x58e, 0x58f, 0x5, 0x18d, 0xc7, 0x2, 0x58f, 0x590, 0x5, 0x17f, - 0xc0, 0x2, 0x590, 0x591, 0x5, 0x17b, 0xbe, 0x2, 0x591, 0x592, 0x5, 0x19d, - 0xcf, 0x2, 0x592, 0x112, 0x3, 0x2, 0x2, 0x2, 0x593, 0x594, 0x5, 0x19b, - 0xce, 0x2, 0x594, 0x595, 0x5, 0x17f, 0xc0, 0x2, 0x595, 0x596, 0x5, 0x18f, - 0xc8, 0x2, 0x596, 0x597, 0x5, 0x187, 0xc4, 0x2, 0x597, 0x114, 0x3, 0x2, - 0x2, 0x2, 0x598, 0x599, 0x5, 0x19b, 0xce, 0x2, 0x599, 0x59a, 0x5, 0x17f, - 0xc0, 0x2, 0x59a, 0x59b, 0x5, 0x191, 0xc9, 0x2, 0x59b, 0x59c, 0x5, 0x17d, - 0xbf, 0x2, 0x59c, 0x59d, 0x5, 0x19b, 0xce, 0x2, 0x59d, 0x116, 0x3, 0x2, - 0x2, 0x2, 0x59e, 0x59f, 0x5, 0x19b, 0xce, 0x2, 0x59f, 0x5a0, 0x5, 0x17f, - 0xc0, 0x2, 0x5a0, 0x5a1, 0x5, 0x19d, 0xcf, 0x2, 0x5a1, 0x118, 0x3, 0x2, - 0x2, 0x2, 0x5a2, 0x5a3, 0x5, 0x19b, 0xce, 0x2, 0x5a3, 0x5a4, 0x5, 0x17f, - 0xc0, 0x2, 0x5a4, 0x5a5, 0x5, 0x19d, 0xcf, 0x2, 0x5a5, 0x5a6, 0x5, 0x19d, - 0xcf, 0x2, 0x5a6, 0x5a7, 0x5, 0x187, 0xc4, 0x2, 0x5a7, 0x5a8, 0x5, 0x191, - 0xc9, 0x2, 0x5a8, 0x5a9, 0x5, 0x183, 0xc2, 0x2, 0x5a9, 0x5aa, 0x5, 0x19b, - 0xce, 0x2, 0x5aa, 0x11a, 0x3, 0x2, 0x2, 0x2, 0x5ab, 0x5ac, 0x5, 0x19b, - 0xce, 0x2, 0x5ac, 0x5ad, 0x5, 0x185, 0xc3, 0x2, 0x5ad, 0x5ae, 0x5, 0x193, - 0xca, 0x2, 0x5ae, 0x5af, 0x5, 0x1a3, 0xd2, 0x2, 0x5af, 0x11c, 0x3, 0x2, - 0x2, 0x2, 0x5b0, 0x5b1, 0x5, 0x19b, 0xce, 0x2, 0x5b1, 0x5b2, 0x5, 0x193, - 0xca, 0x2, 0x5b2, 0x5b3, 0x5, 0x19f, 0xd0, 0x2, 0x5b3, 0x5b4, 0x5, 0x199, - 0xcd, 0x2, 0x5b4, 0x5b5, 0x5, 0x17b, 0xbe, 0x2, 0x5b5, 0x5b6, 0x5, 0x17f, - 0xc0, 0x2, 0x5b6, 0x11e, 0x3, 0x2, 0x2, 0x2, 0x5b7, 0x5b8, 0x5, 0x19b, - 0xce, 0x2, 0x5b8, 0x5b9, 0x5, 0x19d, 0xcf, 0x2, 0x5b9, 0x5ba, 0x5, 0x177, - 0xbc, 0x2, 0x5ba, 0x5bb, 0x5, 0x199, 0xcd, 0x2, 0x5bb, 0x5bc, 0x5, 0x19d, - 0xcf, 0x2, 0x5bc, 0x120, 0x3, 0x2, 0x2, 0x2, 0x5bd, 0x5be, 0x5, 0x19b, - 0xce, 0x2, 0x5be, 0x5bf, 0x5, 0x19d, 0xcf, 0x2, 0x5bf, 0x5c0, 0x5, 0x193, - 0xca, 0x2, 0x5c0, 0x5c1, 0x5, 0x195, 0xcb, 0x2, 0x5c1, 0x122, 0x3, 0x2, - 0x2, 0x2, 0x5c2, 0x5c3, 0x5, 0x19b, 0xce, 0x2, 0x5c3, 0x5c4, 0x5, 0x19f, - 0xd0, 0x2, 0x5c4, 0x5c5, 0x5, 0x179, 0xbd, 0x2, 0x5c5, 0x5c6, 0x5, 0x19b, - 0xce, 0x2, 0x5c6, 0x5c7, 0x5, 0x19d, 0xcf, 0x2, 0x5c7, 0x5c8, 0x5, 0x199, - 0xcd, 0x2, 0x5c8, 0x5c9, 0x5, 0x187, 0xc4, 0x2, 0x5c9, 0x5ca, 0x5, 0x191, - 0xc9, 0x2, 0x5ca, 0x5cb, 0x5, 0x183, 0xc2, 0x2, 0x5cb, 0x124, 0x3, 0x2, - 0x2, 0x2, 0x5cc, 0x5cd, 0x5, 0x19b, 0xce, 0x2, 0x5cd, 0x5ce, 0x5, 0x1a7, - 0xd4, 0x2, 0x5ce, 0x5cf, 0x5, 0x191, 0xc9, 0x2, 0x5cf, 0x5d0, 0x5, 0x17b, - 0xbe, 0x2, 0x5d0, 0x126, 0x3, 0x2, 0x2, 0x2, 0x5d1, 0x5d2, 0x5, 0x19b, - 0xce, 0x2, 0x5d2, 0x5d3, 0x5, 0x1a7, 0xd4, 0x2, 0x5d3, 0x5d4, 0x5, 0x191, - 0xc9, 0x2, 0x5d4, 0x5d5, 0x5, 0x19d, 0xcf, 0x2, 0x5d5, 0x5d6, 0x5, 0x177, - 0xbc, 0x2, 0x5d6, 0x5d7, 0x5, 0x1a5, 0xd3, 0x2, 0x5d7, 0x128, 0x3, 0x2, - 0x2, 0x2, 0x5d8, 0x5d9, 0x5, 0x19b, 0xce, 0x2, 0x5d9, 0x5da, 0x5, 0x1a7, - 0xd4, 0x2, 0x5da, 0x5db, 0x5, 0x19b, 0xce, 0x2, 0x5db, 0x5dc, 0x5, 0x19d, - 0xcf, 0x2, 0x5dc, 0x5dd, 0x5, 0x17f, 0xc0, 0x2, 0x5dd, 0x5de, 0x5, 0x18f, - 0xc8, 0x2, 0x5de, 0x12a, 0x3, 0x2, 0x2, 0x2, 0x5df, 0x5e0, 0x5, 0x19d, - 0xcf, 0x2, 0x5e0, 0x5e1, 0x5, 0x177, 0xbc, 0x2, 0x5e1, 0x5e2, 0x5, 0x179, - 0xbd, 0x2, 0x5e2, 0x5e3, 0x5, 0x18d, 0xc7, 0x2, 0x5e3, 0x5e4, 0x5, 0x17f, - 0xc0, 0x2, 0x5e4, 0x12c, 0x3, 0x2, 0x2, 0x2, 0x5e5, 0x5e6, 0x5, 0x19d, - 0xcf, 0x2, 0x5e6, 0x5e7, 0x5, 0x177, 0xbc, 0x2, 0x5e7, 0x5e8, 0x5, 0x179, - 0xbd, 0x2, 0x5e8, 0x5e9, 0x5, 0x18d, 0xc7, 0x2, 0x5e9, 0x5ea, 0x5, 0x17f, - 0xc0, 0x2, 0x5ea, 0x5eb, 0x5, 0x19b, 0xce, 0x2, 0x5eb, 0x12e, 0x3, 0x2, - 0x2, 0x2, 0x5ec, 0x5ed, 0x5, 0x19d, 0xcf, 0x2, 0x5ed, 0x5ee, 0x5, 0x17f, - 0xc0, 0x2, 0x5ee, 0x5ef, 0x5, 0x18f, 0xc8, 0x2, 0x5ef, 0x5f0, 0x5, 0x195, - 0xcb, 0x2, 0x5f0, 0x5f1, 0x5, 0x193, 0xca, 0x2, 0x5f1, 0x5f2, 0x5, 0x199, - 0xcd, 0x2, 0x5f2, 0x5f3, 0x5, 0x177, 0xbc, 0x2, 0x5f3, 0x5f4, 0x5, 0x199, - 0xcd, 0x2, 0x5f4, 0x5f5, 0x5, 0x1a7, 0xd4, 0x2, 0x5f5, 0x130, 0x3, 0x2, - 0x2, 0x2, 0x5f6, 0x5f7, 0x5, 0x19d, 0xcf, 0x2, 0x5f7, 0x5f8, 0x5, 0x17f, - 0xc0, 0x2, 0x5f8, 0x5f9, 0x5, 0x19b, 0xce, 0x2, 0x5f9, 0x5fa, 0x5, 0x19d, - 0xcf, 0x2, 0x5fa, 0x132, 0x3, 0x2, 0x2, 0x2, 0x5fb, 0x5fc, 0x5, 0x19d, - 0xcf, 0x2, 0x5fc, 0x5fd, 0x5, 0x185, 0xc3, 0x2, 0x5fd, 0x5fe, 0x5, 0x17f, - 0xc0, 0x2, 0x5fe, 0x5ff, 0x5, 0x191, 0xc9, 0x2, 0x5ff, 0x134, 0x3, 0x2, - 0x2, 0x2, 0x600, 0x601, 0x5, 0x19d, 0xcf, 0x2, 0x601, 0x602, 0x5, 0x187, - 0xc4, 0x2, 0x602, 0x603, 0x5, 0x17f, 0xc0, 0x2, 0x603, 0x604, 0x5, 0x19b, - 0xce, 0x2, 0x604, 0x136, 0x3, 0x2, 0x2, 0x2, 0x605, 0x606, 0x5, 0x19d, - 0xcf, 0x2, 0x606, 0x607, 0x5, 0x187, 0xc4, 0x2, 0x607, 0x608, 0x5, 0x18f, - 0xc8, 0x2, 0x608, 0x609, 0x5, 0x17f, 0xc0, 0x2, 0x609, 0x60a, 0x5, 0x193, - 0xca, 0x2, 0x60a, 0x60b, 0x5, 0x19f, 0xd0, 0x2, 0x60b, 0x60c, 0x5, 0x19d, - 0xcf, 0x2, 0x60c, 0x138, 0x3, 0x2, 0x2, 0x2, 0x60d, 0x60e, 0x5, 0x19d, - 0xcf, 0x2, 0x60e, 0x60f, 0x5, 0x187, 0xc4, 0x2, 0x60f, 0x610, 0x5, 0x18f, - 0xc8, 0x2, 0x610, 0x611, 0x5, 0x17f, 0xc0, 0x2, 0x611, 0x612, 0x5, 0x19b, - 0xce, 0x2, 0x612, 0x613, 0x5, 0x19d, 0xcf, 0x2, 0x613, 0x614, 0x5, 0x177, - 0xbc, 0x2, 0x614, 0x615, 0x5, 0x18f, 0xc8, 0x2, 0x615, 0x616, 0x5, 0x195, - 0xcb, 0x2, 0x616, 0x13a, 0x3, 0x2, 0x2, 0x2, 0x617, 0x618, 0x5, 0x19d, - 0xcf, 0x2, 0x618, 0x619, 0x5, 0x193, 0xca, 0x2, 0x619, 0x13c, 0x3, 0x2, - 0x2, 0x2, 0x61a, 0x61b, 0x5, 0x19d, 0xcf, 0x2, 0x61b, 0x61c, 0x5, 0x193, - 0xca, 0x2, 0x61c, 0x61d, 0x5, 0x195, 0xcb, 0x2, 0x61d, 0x13e, 0x3, 0x2, - 0x2, 0x2, 0x61e, 0x61f, 0x5, 0x19d, 0xcf, 0x2, 0x61f, 0x620, 0x5, 0x193, - 0xca, 0x2, 0x620, 0x621, 0x5, 0x19d, 0xcf, 0x2, 0x621, 0x622, 0x5, 0x177, - 0xbc, 0x2, 0x622, 0x623, 0x5, 0x18d, 0xc7, 0x2, 0x623, 0x624, 0x5, 0x19b, - 0xce, 0x2, 0x624, 0x140, 0x3, 0x2, 0x2, 0x2, 0x625, 0x626, 0x5, 0x19d, - 0xcf, 0x2, 0x626, 0x627, 0x5, 0x199, 0xcd, 0x2, 0x627, 0x628, 0x5, 0x177, - 0xbc, 0x2, 0x628, 0x629, 0x5, 0x187, 0xc4, 0x2, 0x629, 0x62a, 0x5, 0x18d, - 0xc7, 0x2, 0x62a, 0x62b, 0x5, 0x187, 0xc4, 0x2, 0x62b, 0x62c, 0x5, 0x191, - 0xc9, 0x2, 0x62c, 0x62d, 0x5, 0x183, 0xc2, 0x2, 0x62d, 0x142, 0x3, 0x2, - 0x2, 0x2, 0x62e, 0x62f, 0x5, 0x19d, 0xcf, 0x2, 0x62f, 0x630, 0x5, 0x199, - 0xcd, 0x2, 0x630, 0x631, 0x5, 0x187, 0xc4, 0x2, 0x631, 0x632, 0x5, 0x18f, - 0xc8, 0x2, 0x632, 0x144, 0x3, 0x2, 0x2, 0x2, 0x633, 0x634, 0x5, 0x19d, - 0xcf, 0x2, 0x634, 0x635, 0x5, 0x199, 0xcd, 0x2, 0x635, 0x636, 0x5, 0x19f, - 0xd0, 0x2, 0x636, 0x637, 0x5, 0x191, 0xc9, 0x2, 0x637, 0x638, 0x5, 0x17b, - 0xbe, 0x2, 0x638, 0x639, 0x5, 0x177, 0xbc, 0x2, 0x639, 0x63a, 0x5, 0x19d, - 0xcf, 0x2, 0x63a, 0x63b, 0x5, 0x17f, 0xc0, 0x2, 0x63b, 0x146, 0x3, 0x2, - 0x2, 0x2, 0x63c, 0x63d, 0x5, 0x19d, 0xcf, 0x2, 0x63d, 0x63e, 0x5, 0x19d, - 0xcf, 0x2, 0x63e, 0x63f, 0x5, 0x18d, 0xc7, 0x2, 0x63f, 0x148, 0x3, 0x2, - 0x2, 0x2, 0x640, 0x641, 0x5, 0x19d, 0xcf, 0x2, 0x641, 0x642, 0x5, 0x1a7, - 0xd4, 0x2, 0x642, 0x643, 0x5, 0x195, 0xcb, 0x2, 0x643, 0x644, 0x5, 0x17f, - 0xc0, 0x2, 0x644, 0x14a, 0x3, 0x2, 0x2, 0x2, 0x645, 0x646, 0x5, 0x19f, - 0xd0, 0x2, 0x646, 0x647, 0x5, 0x191, 0xc9, 0x2, 0x647, 0x648, 0x5, 0x187, - 0xc4, 0x2, 0x648, 0x649, 0x5, 0x193, 0xca, 0x2, 0x649, 0x64a, 0x5, 0x191, - 0xc9, 0x2, 0x64a, 0x14c, 0x3, 0x2, 0x2, 0x2, 0x64b, 0x64c, 0x5, 0x19f, - 0xd0, 0x2, 0x64c, 0x64d, 0x5, 0x195, 0xcb, 0x2, 0x64d, 0x64e, 0x5, 0x17d, - 0xbf, 0x2, 0x64e, 0x64f, 0x5, 0x177, 0xbc, 0x2, 0x64f, 0x650, 0x5, 0x19d, - 0xcf, 0x2, 0x650, 0x651, 0x5, 0x17f, 0xc0, 0x2, 0x651, 0x14e, 0x3, 0x2, - 0x2, 0x2, 0x652, 0x653, 0x5, 0x19f, 0xd0, 0x2, 0x653, 0x654, 0x5, 0x19b, - 0xce, 0x2, 0x654, 0x655, 0x5, 0x17f, 0xc0, 0x2, 0x655, 0x150, 0x3, 0x2, - 0x2, 0x2, 0x656, 0x657, 0x5, 0x19f, 0xd0, 0x2, 0x657, 0x658, 0x5, 0x19b, - 0xce, 0x2, 0x658, 0x659, 0x5, 0x187, 0xc4, 0x2, 0x659, 0x65a, 0x5, 0x191, - 0xc9, 0x2, 0x65a, 0x65b, 0x5, 0x183, 0xc2, 0x2, 0x65b, 0x152, 0x3, 0x2, - 0x2, 0x2, 0x65c, 0x65d, 0x5, 0x19f, 0xd0, 0x2, 0x65d, 0x65e, 0x5, 0x19f, - 0xd0, 0x2, 0x65e, 0x65f, 0x5, 0x187, 0xc4, 0x2, 0x65f, 0x660, 0x5, 0x17d, - 0xbf, 0x2, 0x660, 0x154, 0x3, 0x2, 0x2, 0x2, 0x661, 0x662, 0x5, 0x1a1, - 0xd1, 0x2, 0x662, 0x663, 0x5, 0x177, 0xbc, 0x2, 0x663, 0x664, 0x5, 0x18d, - 0xc7, 0x2, 0x664, 0x665, 0x5, 0x19f, 0xd0, 0x2, 0x665, 0x666, 0x5, 0x17f, - 0xc0, 0x2, 0x666, 0x667, 0x5, 0x19b, 0xce, 0x2, 0x667, 0x156, 0x3, 0x2, - 0x2, 0x2, 0x668, 0x669, 0x5, 0x1a1, 0xd1, 0x2, 0x669, 0x66a, 0x5, 0x187, - 0xc4, 0x2, 0x66a, 0x66b, 0x5, 0x17f, 0xc0, 0x2, 0x66b, 0x66c, 0x5, 0x1a3, - 0xd2, 0x2, 0x66c, 0x158, 0x3, 0x2, 0x2, 0x2, 0x66d, 0x66e, 0x5, 0x1a1, - 0xd1, 0x2, 0x66e, 0x66f, 0x5, 0x193, 0xca, 0x2, 0x66f, 0x670, 0x5, 0x18d, - 0xc7, 0x2, 0x670, 0x671, 0x5, 0x19f, 0xd0, 0x2, 0x671, 0x672, 0x5, 0x18f, - 0xc8, 0x2, 0x672, 0x673, 0x5, 0x17f, 0xc0, 0x2, 0x673, 0x15a, 0x3, 0x2, - 0x2, 0x2, 0x674, 0x675, 0x5, 0x1a3, 0xd2, 0x2, 0x675, 0x676, 0x5, 0x177, - 0xbc, 0x2, 0x676, 0x677, 0x5, 0x19d, 0xcf, 0x2, 0x677, 0x678, 0x5, 0x17b, - 0xbe, 0x2, 0x678, 0x679, 0x5, 0x185, 0xc3, 0x2, 0x679, 0x15c, 0x3, 0x2, - 0x2, 0x2, 0x67a, 0x67b, 0x5, 0x1a3, 0xd2, 0x2, 0x67b, 0x67c, 0x5, 0x17f, - 0xc0, 0x2, 0x67c, 0x67d, 0x5, 0x17f, 0xc0, 0x2, 0x67d, 0x67e, 0x5, 0x18b, - 0xc6, 0x2, 0x67e, 0x15e, 0x3, 0x2, 0x2, 0x2, 0x67f, 0x680, 0x5, 0x1a3, - 0xd2, 0x2, 0x680, 0x681, 0x5, 0x185, 0xc3, 0x2, 0x681, 0x682, 0x5, 0x17f, - 0xc0, 0x2, 0x682, 0x683, 0x5, 0x191, 0xc9, 0x2, 0x683, 0x160, 0x3, 0x2, - 0x2, 0x2, 0x684, 0x685, 0x5, 0x1a3, 0xd2, 0x2, 0x685, 0x686, 0x5, 0x185, - 0xc3, 0x2, 0x686, 0x687, 0x5, 0x17f, 0xc0, 0x2, 0x687, 0x688, 0x5, 0x199, - 0xcd, 0x2, 0x688, 0x689, 0x5, 0x17f, 0xc0, 0x2, 0x689, 0x162, 0x3, 0x2, - 0x2, 0x2, 0x68a, 0x68b, 0x5, 0x1a3, 0xd2, 0x2, 0x68b, 0x68c, 0x5, 0x187, - 0xc4, 0x2, 0x68c, 0x68d, 0x5, 0x19d, 0xcf, 0x2, 0x68d, 0x68e, 0x5, 0x185, - 0xc3, 0x2, 0x68e, 0x164, 0x3, 0x2, 0x2, 0x2, 0x68f, 0x690, 0x5, 0x1a7, - 0xd4, 0x2, 0x690, 0x691, 0x5, 0x17f, 0xc0, 0x2, 0x691, 0x692, 0x5, 0x177, - 0xbc, 0x2, 0x692, 0x693, 0x5, 0x199, 0xcd, 0x2, 0x693, 0x69a, 0x3, 0x2, - 0x2, 0x2, 0x694, 0x695, 0x5, 0x1a7, 0xd4, 0x2, 0x695, 0x696, 0x5, 0x1a7, - 0xd4, 0x2, 0x696, 0x697, 0x5, 0x1a7, 0xd4, 0x2, 0x697, 0x698, 0x5, 0x1a7, - 0xd4, 0x2, 0x698, 0x69a, 0x3, 0x2, 0x2, 0x2, 0x699, 0x68f, 0x3, 0x2, - 0x2, 0x2, 0x699, 0x694, 0x3, 0x2, 0x2, 0x2, 0x69a, 0x166, 0x3, 0x2, - 0x2, 0x2, 0x69b, 0x69c, 0x7, 0x68, 0x2, 0x2, 0x69c, 0x69d, 0x7, 0x63, - 0x2, 0x2, 0x69d, 0x69e, 0x7, 0x6e, 0x2, 0x2, 0x69e, 0x69f, 0x7, 0x75, - 0x2, 0x2, 0x69f, 0x6a0, 0x7, 0x67, 0x2, 0x2, 0x6a0, 0x168, 0x3, 0x2, - 0x2, 0x2, 0x6a1, 0x6a2, 0x7, 0x76, 0x2, 0x2, 0x6a2, 0x6a3, 0x7, 0x74, - 0x2, 0x2, 0x6a3, 0x6a4, 0x7, 0x77, 0x2, 0x2, 0x6a4, 0x6a5, 0x7, 0x67, - 0x2, 0x2, 0x6a5, 0x16a, 0x3, 0x2, 0x2, 0x2, 0x6a6, 0x6a9, 0x5, 0x1ab, - 0xd6, 0x2, 0x6a7, 0x6a9, 0x5, 0x1ed, 0xf7, 0x2, 0x6a8, 0x6a6, 0x3, 0x2, - 0x2, 0x2, 0x6a8, 0x6a7, 0x3, 0x2, 0x2, 0x2, 0x6a9, 0x6af, 0x3, 0x2, - 0x2, 0x2, 0x6aa, 0x6ae, 0x5, 0x1ab, 0xd6, 0x2, 0x6ab, 0x6ae, 0x5, 0x1ed, - 0xf7, 0x2, 0x6ac, 0x6ae, 0x5, 0x1af, 0xd8, 0x2, 0x6ad, 0x6aa, 0x3, 0x2, - 0x2, 0x2, 0x6ad, 0x6ab, 0x3, 0x2, 0x2, 0x2, 0x6ad, 0x6ac, 0x3, 0x2, - 0x2, 0x2, 0x6ae, 0x6b1, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x6ad, 0x3, 0x2, - 0x2, 0x2, 0x6af, 0x6b0, 0x3, 0x2, 0x2, 0x2, 0x6b0, 0x6d1, 0x3, 0x2, - 0x2, 0x2, 0x6b1, 0x6af, 0x3, 0x2, 0x2, 0x2, 0x6b2, 0x6bc, 0x5, 0x1b7, - 0xdc, 0x2, 0x6b3, 0x6bb, 0xa, 0x2, 0x2, 0x2, 0x6b4, 0x6b5, 0x5, 0x1b9, - 0xdd, 0x2, 0x6b5, 0x6b6, 0xb, 0x2, 0x2, 0x2, 0x6b6, 0x6bb, 0x3, 0x2, - 0x2, 0x2, 0x6b7, 0x6b8, 0x5, 0x1b7, 0xdc, 0x2, 0x6b8, 0x6b9, 0x5, 0x1b7, - 0xdc, 0x2, 0x6b9, 0x6bb, 0x3, 0x2, 0x2, 0x2, 0x6ba, 0x6b3, 0x3, 0x2, - 0x2, 0x2, 0x6ba, 0x6b4, 0x3, 0x2, 0x2, 0x2, 0x6ba, 0x6b7, 0x3, 0x2, - 0x2, 0x2, 0x6bb, 0x6be, 0x3, 0x2, 0x2, 0x2, 0x6bc, 0x6ba, 0x3, 0x2, - 0x2, 0x2, 0x6bc, 0x6bd, 0x3, 0x2, 0x2, 0x2, 0x6bd, 0x6bf, 0x3, 0x2, - 0x2, 0x2, 0x6be, 0x6bc, 0x3, 0x2, 0x2, 0x2, 0x6bf, 0x6c0, 0x5, 0x1b7, - 0xdc, 0x2, 0x6c0, 0x6d1, 0x3, 0x2, 0x2, 0x2, 0x6c1, 0x6cb, 0x5, 0x1df, - 0xf0, 0x2, 0x6c2, 0x6ca, 0xa, 0x3, 0x2, 0x2, 0x6c3, 0x6c4, 0x5, 0x1b9, - 0xdd, 0x2, 0x6c4, 0x6c5, 0xb, 0x2, 0x2, 0x2, 0x6c5, 0x6ca, 0x3, 0x2, - 0x2, 0x2, 0x6c6, 0x6c7, 0x5, 0x1df, 0xf0, 0x2, 0x6c7, 0x6c8, 0x5, 0x1df, - 0xf0, 0x2, 0x6c8, 0x6ca, 0x3, 0x2, 0x2, 0x2, 0x6c9, 0x6c2, 0x3, 0x2, - 0x2, 0x2, 0x6c9, 0x6c3, 0x3, 0x2, 0x2, 0x2, 0x6c9, 0x6c6, 0x3, 0x2, - 0x2, 0x2, 0x6ca, 0x6cd, 0x3, 0x2, 0x2, 0x2, 0x6cb, 0x6c9, 0x3, 0x2, - 0x2, 0x2, 0x6cb, 0x6cc, 0x3, 0x2, 0x2, 0x2, 0x6cc, 0x6ce, 0x3, 0x2, - 0x2, 0x2, 0x6cd, 0x6cb, 0x3, 0x2, 0x2, 0x2, 0x6ce, 0x6cf, 0x5, 0x1df, - 0xf0, 0x2, 0x6cf, 0x6d1, 0x3, 0x2, 0x2, 0x2, 0x6d0, 0x6a8, 0x3, 0x2, - 0x2, 0x2, 0x6d0, 0x6b2, 0x3, 0x2, 0x2, 0x2, 0x6d0, 0x6c1, 0x3, 0x2, - 0x2, 0x2, 0x6d1, 0x16c, 0x3, 0x2, 0x2, 0x2, 0x6d2, 0x6d3, 0x5, 0x173, - 0xba, 0x2, 0x6d3, 0x6d7, 0x5, 0x1c3, 0xe2, 0x2, 0x6d4, 0x6d6, 0x5, 0x1b1, - 0xd9, 0x2, 0x6d5, 0x6d4, 0x3, 0x2, 0x2, 0x2, 0x6d6, 0x6d9, 0x3, 0x2, - 0x2, 0x2, 0x6d7, 0x6d5, 0x3, 0x2, 0x2, 0x2, 0x6d7, 0x6d8, 0x3, 0x2, - 0x2, 0x2, 0x6d8, 0x6dc, 0x3, 0x2, 0x2, 0x2, 0x6d9, 0x6d7, 0x3, 0x2, - 0x2, 0x2, 0x6da, 0x6dd, 0x5, 0x195, 0xcb, 0x2, 0x6db, 0x6dd, 0x5, 0x17f, - 0xc0, 0x2, 0x6dc, 0x6da, 0x3, 0x2, 0x2, 0x2, 0x6dc, 0x6db, 0x3, 0x2, - 0x2, 0x2, 0x6dd, 0x6e0, 0x3, 0x2, 0x2, 0x2, 0x6de, 0x6e1, 0x5, 0x1db, - 0xee, 0x2, 0x6df, 0x6e1, 0x5, 0x1c1, 0xe1, 0x2, 0x6e0, 0x6de, 0x3, 0x2, - 0x2, 0x2, 0x6e0, 0x6df, 0x3, 0x2, 0x2, 0x2, 0x6e0, 0x6e1, 0x3, 0x2, - 0x2, 0x2, 0x6e1, 0x6e3, 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6e4, 0x5, 0x1af, - 0xd8, 0x2, 0x6e3, 0x6e2, 0x3, 0x2, 0x2, 0x2, 0x6e4, 0x6e5, 0x3, 0x2, - 0x2, 0x2, 0x6e5, 0x6e3, 0x3, 0x2, 0x2, 0x2, 0x6e5, 0x6e6, 0x3, 0x2, - 0x2, 0x2, 0x6e6, 0x71f, 0x3, 0x2, 0x2, 0x2, 0x6e7, 0x6ea, 0x5, 0x173, - 0xba, 0x2, 0x6e8, 0x6eb, 0x5, 0x195, 0xcb, 0x2, 0x6e9, 0x6eb, 0x5, 0x17f, - 0xc0, 0x2, 0x6ea, 0x6e8, 0x3, 0x2, 0x2, 0x2, 0x6ea, 0x6e9, 0x3, 0x2, - 0x2, 0x2, 0x6eb, 0x6ee, 0x3, 0x2, 0x2, 0x2, 0x6ec, 0x6ef, 0x5, 0x1db, - 0xee, 0x2, 0x6ed, 0x6ef, 0x5, 0x1c1, 0xe1, 0x2, 0x6ee, 0x6ec, 0x3, 0x2, - 0x2, 0x2, 0x6ee, 0x6ed, 0x3, 0x2, 0x2, 0x2, 0x6ee, 0x6ef, 0x3, 0x2, - 0x2, 0x2, 0x6ef, 0x6f1, 0x3, 0x2, 0x2, 0x2, 0x6f0, 0x6f2, 0x5, 0x1af, - 0xd8, 0x2, 0x6f1, 0x6f0, 0x3, 0x2, 0x2, 0x2, 0x6f2, 0x6f3, 0x3, 0x2, - 0x2, 0x2, 0x6f3, 0x6f1, 0x3, 0x2, 0x2, 0x2, 0x6f3, 0x6f4, 0x3, 0x2, - 0x2, 0x2, 0x6f4, 0x71f, 0x3, 0x2, 0x2, 0x2, 0x6f5, 0x6f6, 0x5, 0x171, - 0xb9, 0x2, 0x6f6, 0x6fa, 0x5, 0x1c3, 0xe2, 0x2, 0x6f7, 0x6f9, 0x5, 0x1af, - 0xd8, 0x2, 0x6f8, 0x6f7, 0x3, 0x2, 0x2, 0x2, 0x6f9, 0x6fc, 0x3, 0x2, - 0x2, 0x2, 0x6fa, 0x6f8, 0x3, 0x2, 0x2, 0x2, 0x6fa, 0x6fb, 0x3, 0x2, - 0x2, 0x2, 0x6fb, 0x6fd, 0x3, 0x2, 0x2, 0x2, 0x6fc, 0x6fa, 0x3, 0x2, - 0x2, 0x2, 0x6fd, 0x700, 0x5, 0x17f, 0xc0, 0x2, 0x6fe, 0x701, 0x5, 0x1db, - 0xee, 0x2, 0x6ff, 0x701, 0x5, 0x1c1, 0xe1, 0x2, 0x700, 0x6fe, 0x3, 0x2, - 0x2, 0x2, 0x700, 0x6ff, 0x3, 0x2, 0x2, 0x2, 0x700, 0x701, 0x3, 0x2, - 0x2, 0x2, 0x701, 0x703, 0x3, 0x2, 0x2, 0x2, 0x702, 0x704, 0x5, 0x1af, - 0xd8, 0x2, 0x703, 0x702, 0x3, 0x2, 0x2, 0x2, 0x704, 0x705, 0x3, 0x2, - 0x2, 0x2, 0x705, 0x703, 0x3, 0x2, 0x2, 0x2, 0x705, 0x706, 0x3, 0x2, - 0x2, 0x2, 0x706, 0x71f, 0x3, 0x2, 0x2, 0x2, 0x707, 0x708, 0x5, 0x1c3, - 0xe2, 0x2, 0x708, 0x709, 0x5, 0x171, 0xb9, 0x2, 0x709, 0x70c, 0x5, 0x17f, - 0xc0, 0x2, 0x70a, 0x70d, 0x5, 0x1db, 0xee, 0x2, 0x70b, 0x70d, 0x5, 0x1c1, - 0xe1, 0x2, 0x70c, 0x70a, 0x3, 0x2, 0x2, 0x2, 0x70c, 0x70b, 0x3, 0x2, - 0x2, 0x2, 0x70c, 0x70d, 0x3, 0x2, 0x2, 0x2, 0x70d, 0x70f, 0x3, 0x2, - 0x2, 0x2, 0x70e, 0x710, 0x5, 0x1af, 0xd8, 0x2, 0x70f, 0x70e, 0x3, 0x2, - 0x2, 0x2, 0x710, 0x711, 0x3, 0x2, 0x2, 0x2, 0x711, 0x70f, 0x3, 0x2, - 0x2, 0x2, 0x711, 0x712, 0x3, 0x2, 0x2, 0x2, 0x712, 0x71f, 0x3, 0x2, - 0x2, 0x2, 0x713, 0x714, 0x5, 0x171, 0xb9, 0x2, 0x714, 0x717, 0x5, 0x17f, - 0xc0, 0x2, 0x715, 0x718, 0x5, 0x1db, 0xee, 0x2, 0x716, 0x718, 0x5, 0x1c1, - 0xe1, 0x2, 0x717, 0x715, 0x3, 0x2, 0x2, 0x2, 0x717, 0x716, 0x3, 0x2, - 0x2, 0x2, 0x717, 0x718, 0x3, 0x2, 0x2, 0x2, 0x718, 0x71a, 0x3, 0x2, - 0x2, 0x2, 0x719, 0x71b, 0x5, 0x1af, 0xd8, 0x2, 0x71a, 0x719, 0x3, 0x2, - 0x2, 0x2, 0x71b, 0x71c, 0x3, 0x2, 0x2, 0x2, 0x71c, 0x71a, 0x3, 0x2, - 0x2, 0x2, 0x71c, 0x71d, 0x3, 0x2, 0x2, 0x2, 0x71d, 0x71f, 0x3, 0x2, - 0x2, 0x2, 0x71e, 0x6d2, 0x3, 0x2, 0x2, 0x2, 0x71e, 0x6e7, 0x3, 0x2, - 0x2, 0x2, 0x71e, 0x6f5, 0x3, 0x2, 0x2, 0x2, 0x71e, 0x707, 0x3, 0x2, - 0x2, 0x2, 0x71e, 0x713, 0x3, 0x2, 0x2, 0x2, 0x71f, 0x16e, 0x3, 0x2, - 0x2, 0x2, 0x720, 0x722, 0x7, 0x32, 0x2, 0x2, 0x721, 0x723, 0x5, 0x1ad, - 0xd7, 0x2, 0x722, 0x721, 0x3, 0x2, 0x2, 0x2, 0x723, 0x724, 0x3, 0x2, - 0x2, 0x2, 0x724, 0x722, 0x3, 0x2, 0x2, 0x2, 0x724, 0x725, 0x3, 0x2, - 0x2, 0x2, 0x725, 0x170, 0x3, 0x2, 0x2, 0x2, 0x726, 0x728, 0x5, 0x1af, - 0xd8, 0x2, 0x727, 0x726, 0x3, 0x2, 0x2, 0x2, 0x728, 0x729, 0x3, 0x2, - 0x2, 0x2, 0x729, 0x727, 0x3, 0x2, 0x2, 0x2, 0x729, 0x72a, 0x3, 0x2, - 0x2, 0x2, 0x72a, 0x172, 0x3, 0x2, 0x2, 0x2, 0x72b, 0x72c, 0x7, 0x32, - 0x2, 0x2, 0x72c, 0x72e, 0x5, 0x1a5, 0xd3, 0x2, 0x72d, 0x72f, 0x5, 0x1b1, - 0xd9, 0x2, 0x72e, 0x72d, 0x3, 0x2, 0x2, 0x2, 0x72f, 0x730, 0x3, 0x2, - 0x2, 0x2, 0x730, 0x72e, 0x3, 0x2, 0x2, 0x2, 0x730, 0x731, 0x3, 0x2, - 0x2, 0x2, 0x731, 0x174, 0x3, 0x2, 0x2, 0x2, 0x732, 0x73c, 0x5, 0x1e1, - 0xf1, 0x2, 0x733, 0x73b, 0xa, 0x4, 0x2, 0x2, 0x734, 0x735, 0x5, 0x1b9, - 0xdd, 0x2, 0x735, 0x736, 0xb, 0x2, 0x2, 0x2, 0x736, 0x73b, 0x3, 0x2, - 0x2, 0x2, 0x737, 0x738, 0x5, 0x1e1, 0xf1, 0x2, 0x738, 0x739, 0x5, 0x1e1, - 0xf1, 0x2, 0x739, 0x73b, 0x3, 0x2, 0x2, 0x2, 0x73a, 0x733, 0x3, 0x2, - 0x2, 0x2, 0x73a, 0x734, 0x3, 0x2, 0x2, 0x2, 0x73a, 0x737, 0x3, 0x2, - 0x2, 0x2, 0x73b, 0x73e, 0x3, 0x2, 0x2, 0x2, 0x73c, 0x73a, 0x3, 0x2, - 0x2, 0x2, 0x73c, 0x73d, 0x3, 0x2, 0x2, 0x2, 0x73d, 0x73f, 0x3, 0x2, - 0x2, 0x2, 0x73e, 0x73c, 0x3, 0x2, 0x2, 0x2, 0x73f, 0x740, 0x5, 0x1e1, - 0xf1, 0x2, 0x740, 0x176, 0x3, 0x2, 0x2, 0x2, 0x741, 0x742, 0x9, 0x5, - 0x2, 0x2, 0x742, 0x178, 0x3, 0x2, 0x2, 0x2, 0x743, 0x744, 0x9, 0x6, - 0x2, 0x2, 0x744, 0x17a, 0x3, 0x2, 0x2, 0x2, 0x745, 0x746, 0x9, 0x7, - 0x2, 0x2, 0x746, 0x17c, 0x3, 0x2, 0x2, 0x2, 0x747, 0x748, 0x9, 0x8, - 0x2, 0x2, 0x748, 0x17e, 0x3, 0x2, 0x2, 0x2, 0x749, 0x74a, 0x9, 0x9, - 0x2, 0x2, 0x74a, 0x180, 0x3, 0x2, 0x2, 0x2, 0x74b, 0x74c, 0x9, 0xa, - 0x2, 0x2, 0x74c, 0x182, 0x3, 0x2, 0x2, 0x2, 0x74d, 0x74e, 0x9, 0xb, - 0x2, 0x2, 0x74e, 0x184, 0x3, 0x2, 0x2, 0x2, 0x74f, 0x750, 0x9, 0xc, - 0x2, 0x2, 0x750, 0x186, 0x3, 0x2, 0x2, 0x2, 0x751, 0x752, 0x9, 0xd, - 0x2, 0x2, 0x752, 0x188, 0x3, 0x2, 0x2, 0x2, 0x753, 0x754, 0x9, 0xe, - 0x2, 0x2, 0x754, 0x18a, 0x3, 0x2, 0x2, 0x2, 0x755, 0x756, 0x9, 0xf, - 0x2, 0x2, 0x756, 0x18c, 0x3, 0x2, 0x2, 0x2, 0x757, 0x758, 0x9, 0x10, - 0x2, 0x2, 0x758, 0x18e, 0x3, 0x2, 0x2, 0x2, 0x759, 0x75a, 0x9, 0x11, - 0x2, 0x2, 0x75a, 0x190, 0x3, 0x2, 0x2, 0x2, 0x75b, 0x75c, 0x9, 0x12, - 0x2, 0x2, 0x75c, 0x192, 0x3, 0x2, 0x2, 0x2, 0x75d, 0x75e, 0x9, 0x13, - 0x2, 0x2, 0x75e, 0x194, 0x3, 0x2, 0x2, 0x2, 0x75f, 0x760, 0x9, 0x14, - 0x2, 0x2, 0x760, 0x196, 0x3, 0x2, 0x2, 0x2, 0x761, 0x762, 0x9, 0x15, - 0x2, 0x2, 0x762, 0x198, 0x3, 0x2, 0x2, 0x2, 0x763, 0x764, 0x9, 0x16, - 0x2, 0x2, 0x764, 0x19a, 0x3, 0x2, 0x2, 0x2, 0x765, 0x766, 0x9, 0x17, - 0x2, 0x2, 0x766, 0x19c, 0x3, 0x2, 0x2, 0x2, 0x767, 0x768, 0x9, 0x18, - 0x2, 0x2, 0x768, 0x19e, 0x3, 0x2, 0x2, 0x2, 0x769, 0x76a, 0x9, 0x19, - 0x2, 0x2, 0x76a, 0x1a0, 0x3, 0x2, 0x2, 0x2, 0x76b, 0x76c, 0x9, 0x1a, - 0x2, 0x2, 0x76c, 0x1a2, 0x3, 0x2, 0x2, 0x2, 0x76d, 0x76e, 0x9, 0x1b, - 0x2, 0x2, 0x76e, 0x1a4, 0x3, 0x2, 0x2, 0x2, 0x76f, 0x770, 0x9, 0x1c, - 0x2, 0x2, 0x770, 0x1a6, 0x3, 0x2, 0x2, 0x2, 0x771, 0x772, 0x9, 0x1d, - 0x2, 0x2, 0x772, 0x1a8, 0x3, 0x2, 0x2, 0x2, 0x773, 0x774, 0x9, 0x1e, - 0x2, 0x2, 0x774, 0x1aa, 0x3, 0x2, 0x2, 0x2, 0x775, 0x776, 0x9, 0x1f, - 0x2, 0x2, 0x776, 0x1ac, 0x3, 0x2, 0x2, 0x2, 0x777, 0x778, 0x9, 0x20, - 0x2, 0x2, 0x778, 0x1ae, 0x3, 0x2, 0x2, 0x2, 0x779, 0x77a, 0x9, 0x21, - 0x2, 0x2, 0x77a, 0x1b0, 0x3, 0x2, 0x2, 0x2, 0x77b, 0x77c, 0x9, 0x22, - 0x2, 0x2, 0x77c, 0x1b2, 0x3, 0x2, 0x2, 0x2, 0x77d, 0x77e, 0x7, 0x2f, - 0x2, 0x2, 0x77e, 0x77f, 0x7, 0x40, 0x2, 0x2, 0x77f, 0x1b4, 0x3, 0x2, - 0x2, 0x2, 0x780, 0x781, 0x7, 0x2c, 0x2, 0x2, 0x781, 0x1b6, 0x3, 0x2, - 0x2, 0x2, 0x782, 0x783, 0x7, 0x62, 0x2, 0x2, 0x783, 0x1b8, 0x3, 0x2, - 0x2, 0x2, 0x784, 0x785, 0x7, 0x5e, 0x2, 0x2, 0x785, 0x1ba, 0x3, 0x2, - 0x2, 0x2, 0x786, 0x787, 0x7, 0x3c, 0x2, 0x2, 0x787, 0x1bc, 0x3, 0x2, - 0x2, 0x2, 0x788, 0x789, 0x7, 0x2e, 0x2, 0x2, 0x789, 0x1be, 0x3, 0x2, - 0x2, 0x2, 0x78a, 0x78b, 0x7, 0x7e, 0x2, 0x2, 0x78b, 0x78c, 0x7, 0x7e, - 0x2, 0x2, 0x78c, 0x1c0, 0x3, 0x2, 0x2, 0x2, 0x78d, 0x78e, 0x7, 0x2f, - 0x2, 0x2, 0x78e, 0x1c2, 0x3, 0x2, 0x2, 0x2, 0x78f, 0x790, 0x7, 0x30, - 0x2, 0x2, 0x790, 0x1c4, 0x3, 0x2, 0x2, 0x2, 0x791, 0x792, 0x7, 0x3f, - 0x2, 0x2, 0x792, 0x793, 0x7, 0x3f, 0x2, 0x2, 0x793, 0x1c6, 0x3, 0x2, - 0x2, 0x2, 0x794, 0x795, 0x7, 0x3f, 0x2, 0x2, 0x795, 0x1c8, 0x3, 0x2, - 0x2, 0x2, 0x796, 0x797, 0x7, 0x40, 0x2, 0x2, 0x797, 0x798, 0x7, 0x3f, - 0x2, 0x2, 0x798, 0x1ca, 0x3, 0x2, 0x2, 0x2, 0x799, 0x79a, 0x7, 0x40, - 0x2, 0x2, 0x79a, 0x1cc, 0x3, 0x2, 0x2, 0x2, 0x79b, 0x79c, 0x7, 0x7d, - 0x2, 0x2, 0x79c, 0x1ce, 0x3, 0x2, 0x2, 0x2, 0x79d, 0x79e, 0x7, 0x5d, - 0x2, 0x2, 0x79e, 0x1d0, 0x3, 0x2, 0x2, 0x2, 0x79f, 0x7a0, 0x7, 0x3e, - 0x2, 0x2, 0x7a0, 0x7a1, 0x7, 0x3f, 0x2, 0x2, 0x7a1, 0x1d2, 0x3, 0x2, - 0x2, 0x2, 0x7a2, 0x7a3, 0x7, 0x2a, 0x2, 0x2, 0x7a3, 0x1d4, 0x3, 0x2, - 0x2, 0x2, 0x7a4, 0x7a5, 0x7, 0x3e, 0x2, 0x2, 0x7a5, 0x1d6, 0x3, 0x2, - 0x2, 0x2, 0x7a6, 0x7a7, 0x7, 0x23, 0x2, 0x2, 0x7a7, 0x7ab, 0x7, 0x3f, - 0x2, 0x2, 0x7a8, 0x7a9, 0x7, 0x3e, 0x2, 0x2, 0x7a9, 0x7ab, 0x7, 0x40, - 0x2, 0x2, 0x7aa, 0x7a6, 0x3, 0x2, 0x2, 0x2, 0x7aa, 0x7a8, 0x3, 0x2, - 0x2, 0x2, 0x7ab, 0x1d8, 0x3, 0x2, 0x2, 0x2, 0x7ac, 0x7ad, 0x7, 0x27, - 0x2, 0x2, 0x7ad, 0x1da, 0x3, 0x2, 0x2, 0x2, 0x7ae, 0x7af, 0x7, 0x2d, - 0x2, 0x2, 0x7af, 0x1dc, 0x3, 0x2, 0x2, 0x2, 0x7b0, 0x7b1, 0x7, 0x41, - 0x2, 0x2, 0x7b1, 0x1de, 0x3, 0x2, 0x2, 0x2, 0x7b2, 0x7b3, 0x7, 0x24, - 0x2, 0x2, 0x7b3, 0x1e0, 0x3, 0x2, 0x2, 0x2, 0x7b4, 0x7b5, 0x7, 0x29, - 0x2, 0x2, 0x7b5, 0x1e2, 0x3, 0x2, 0x2, 0x2, 0x7b6, 0x7b7, 0x7, 0x7f, - 0x2, 0x2, 0x7b7, 0x1e4, 0x3, 0x2, 0x2, 0x2, 0x7b8, 0x7b9, 0x7, 0x5f, - 0x2, 0x2, 0x7b9, 0x1e6, 0x3, 0x2, 0x2, 0x2, 0x7ba, 0x7bb, 0x7, 0x2b, - 0x2, 0x2, 0x7bb, 0x1e8, 0x3, 0x2, 0x2, 0x2, 0x7bc, 0x7bd, 0x7, 0x3d, - 0x2, 0x2, 0x7bd, 0x1ea, 0x3, 0x2, 0x2, 0x2, 0x7be, 0x7bf, 0x7, 0x31, - 0x2, 0x2, 0x7bf, 0x1ec, 0x3, 0x2, 0x2, 0x2, 0x7c0, 0x7c1, 0x7, 0x61, - 0x2, 0x2, 0x7c1, 0x1ee, 0x3, 0x2, 0x2, 0x2, 0x7c2, 0x7c3, 0x7, 0x31, - 0x2, 0x2, 0x7c3, 0x7c4, 0x7, 0x2c, 0x2, 0x2, 0x7c4, 0x7c8, 0x3, 0x2, - 0x2, 0x2, 0x7c5, 0x7c7, 0xb, 0x2, 0x2, 0x2, 0x7c6, 0x7c5, 0x3, 0x2, - 0x2, 0x2, 0x7c7, 0x7ca, 0x3, 0x2, 0x2, 0x2, 0x7c8, 0x7c9, 0x3, 0x2, - 0x2, 0x2, 0x7c8, 0x7c6, 0x3, 0x2, 0x2, 0x2, 0x7c9, 0x7cb, 0x3, 0x2, - 0x2, 0x2, 0x7ca, 0x7c8, 0x3, 0x2, 0x2, 0x2, 0x7cb, 0x7cc, 0x7, 0x2c, - 0x2, 0x2, 0x7cc, 0x7cd, 0x7, 0x31, 0x2, 0x2, 0x7cd, 0x7ce, 0x3, 0x2, - 0x2, 0x2, 0x7ce, 0x7cf, 0x8, 0xf8, 0x2, 0x2, 0x7cf, 0x1f0, 0x3, 0x2, - 0x2, 0x2, 0x7d0, 0x7d1, 0x7, 0x2f, 0x2, 0x2, 0x7d1, 0x7d2, 0x7, 0x2f, - 0x2, 0x2, 0x7d2, 0x7d6, 0x3, 0x2, 0x2, 0x2, 0x7d3, 0x7d5, 0xa, 0x23, - 0x2, 0x2, 0x7d4, 0x7d3, 0x3, 0x2, 0x2, 0x2, 0x7d5, 0x7d8, 0x3, 0x2, - 0x2, 0x2, 0x7d6, 0x7d4, 0x3, 0x2, 0x2, 0x2, 0x7d6, 0x7d7, 0x3, 0x2, - 0x2, 0x2, 0x7d7, 0x7da, 0x3, 0x2, 0x2, 0x2, 0x7d8, 0x7d6, 0x3, 0x2, - 0x2, 0x2, 0x7d9, 0x7db, 0x9, 0x24, 0x2, 0x2, 0x7da, 0x7d9, 0x3, 0x2, - 0x2, 0x2, 0x7db, 0x7dc, 0x3, 0x2, 0x2, 0x2, 0x7dc, 0x7dd, 0x8, 0xf9, - 0x2, 0x2, 0x7dd, 0x1f2, 0x3, 0x2, 0x2, 0x2, 0x7de, 0x7df, 0x9, 0x25, - 0x2, 0x2, 0x7df, 0x7e0, 0x3, 0x2, 0x2, 0x2, 0x7e0, 0x7e1, 0x8, 0xfa, - 0x2, 0x2, 0x7e1, 0x1f4, 0x3, 0x2, 0x2, 0x2, 0x26, 0x2, 0x233, 0x40f, - 0x699, 0x6a8, 0x6ad, 0x6af, 0x6ba, 0x6bc, 0x6c9, 0x6cb, 0x6d0, 0x6d7, - 0x6dc, 0x6e0, 0x6e5, 0x6ea, 0x6ee, 0x6f3, 0x6fa, 0x700, 0x705, 0x70c, - 0x711, 0x717, 0x71c, 0x71e, 0x724, 0x729, 0x730, 0x73a, 0x73c, 0x7aa, - 0x7c8, 0x7d6, 0x7da, 0x3, 0x8, 0x2, 0x2, + 0x3, 0x7a, 0x3, 0x7a, 0x3, 0x7b, 0x3, 0x7b, 0x3, 0x7b, 0x3, 0x7b, 0x3, + 0x7b, 0x3, 0x7b, 0x3, 0x7b, 0x3, 0x7b, 0x3, 0x7b, 0x3, 0x7c, 0x3, 0x7c, + 0x3, 0x7c, 0x3, 0x7c, 0x3, 0x7c, 0x3, 0x7c, 0x3, 0x7c, 0x3, 0x7c, 0x3, + 0x7c, 0x3, 0x7d, 0x3, 0x7d, 0x3, 0x7d, 0x3, 0x7d, 0x3, 0x7d, 0x3, 0x7d, + 0x3, 0x7d, 0x3, 0x7d, 0x3, 0x7e, 0x3, 0x7e, 0x3, 0x7e, 0x3, 0x7e, 0x3, + 0x7e, 0x3, 0x7e, 0x3, 0x7e, 0x3, 0x7e, 0x3, 0x7e, 0x3, 0x7e, 0x3, 0x7e, + 0x3, 0x7f, 0x3, 0x7f, 0x3, 0x7f, 0x3, 0x7f, 0x3, 0x7f, 0x3, 0x7f, 0x3, + 0x7f, 0x3, 0x7f, 0x3, 0x80, 0x3, 0x80, 0x3, 0x80, 0x3, 0x80, 0x3, 0x80, + 0x3, 0x80, 0x3, 0x81, 0x3, 0x81, 0x3, 0x81, 0x3, 0x81, 0x3, 0x81, 0x3, + 0x81, 0x3, 0x81, 0x3, 0x82, 0x3, 0x82, 0x3, 0x82, 0x3, 0x82, 0x3, 0x82, + 0x3, 0x82, 0x3, 0x82, 0x3, 0x83, 0x3, 0x83, 0x3, 0x83, 0x3, 0x83, 0x3, + 0x83, 0x3, 0x83, 0x3, 0x83, 0x3, 0x84, 0x3, 0x84, 0x3, 0x84, 0x3, 0x84, + 0x3, 0x84, 0x3, 0x84, 0x3, 0x84, 0x3, 0x84, 0x3, 0x85, 0x3, 0x85, 0x3, + 0x85, 0x3, 0x85, 0x3, 0x85, 0x3, 0x85, 0x3, 0x85, 0x3, 0x85, 0x3, 0x86, + 0x3, 0x86, 0x3, 0x86, 0x3, 0x86, 0x3, 0x86, 0x3, 0x86, 0x3, 0x86, 0x3, + 0x86, 0x3, 0x86, 0x3, 0x86, 0x3, 0x86, 0x3, 0x87, 0x3, 0x87, 0x3, 0x87, + 0x3, 0x87, 0x3, 0x87, 0x3, 0x87, 0x3, 0x88, 0x3, 0x88, 0x3, 0x88, 0x3, + 0x88, 0x3, 0x88, 0x3, 0x88, 0x3, 0x88, 0x3, 0x89, 0x3, 0x89, 0x3, 0x89, + 0x3, 0x89, 0x3, 0x89, 0x3, 0x89, 0x3, 0x89, 0x3, 0x8a, 0x3, 0x8a, 0x3, + 0x8a, 0x3, 0x8a, 0x3, 0x8a, 0x3, 0x8a, 0x3, 0x8a, 0x3, 0x8b, 0x3, 0x8b, + 0x3, 0x8b, 0x3, 0x8b, 0x3, 0x8b, 0x3, 0x8b, 0x3, 0x8b, 0x3, 0x8c, 0x3, + 0x8c, 0x3, 0x8c, 0x3, 0x8c, 0x3, 0x8c, 0x3, 0x8d, 0x3, 0x8d, 0x3, 0x8d, + 0x3, 0x8d, 0x3, 0x8d, 0x3, 0x8d, 0x3, 0x8e, 0x3, 0x8e, 0x3, 0x8e, 0x3, + 0x8e, 0x3, 0x8f, 0x3, 0x8f, 0x3, 0x8f, 0x3, 0x8f, 0x3, 0x8f, 0x3, 0x8f, + 0x3, 0x8f, 0x3, 0x8f, 0x3, 0x8f, 0x3, 0x90, 0x3, 0x90, 0x3, 0x90, 0x3, + 0x90, 0x3, 0x90, 0x3, 0x91, 0x3, 0x91, 0x3, 0x91, 0x3, 0x91, 0x3, 0x91, + 0x3, 0x91, 0x3, 0x91, 0x3, 0x92, 0x3, 0x92, 0x3, 0x92, 0x3, 0x92, 0x3, + 0x92, 0x3, 0x92, 0x3, 0x93, 0x3, 0x93, 0x3, 0x93, 0x3, 0x93, 0x3, 0x93, + 0x3, 0x94, 0x3, 0x94, 0x3, 0x94, 0x3, 0x94, 0x3, 0x94, 0x3, 0x94, 0x3, + 0x94, 0x3, 0x94, 0x3, 0x94, 0x3, 0x94, 0x3, 0x95, 0x3, 0x95, 0x3, 0x95, + 0x3, 0x95, 0x3, 0x95, 0x3, 0x96, 0x3, 0x96, 0x3, 0x96, 0x3, 0x96, 0x3, + 0x96, 0x3, 0x96, 0x3, 0x96, 0x3, 0x97, 0x3, 0x97, 0x3, 0x97, 0x3, 0x97, + 0x3, 0x97, 0x3, 0x97, 0x3, 0x97, 0x3, 0x98, 0x3, 0x98, 0x3, 0x98, 0x3, + 0x98, 0x3, 0x98, 0x3, 0x98, 0x3, 0x99, 0x3, 0x99, 0x3, 0x99, 0x3, 0x99, + 0x3, 0x99, 0x3, 0x99, 0x3, 0x99, 0x3, 0x9a, 0x3, 0x9a, 0x3, 0x9a, 0x3, + 0x9a, 0x3, 0x9a, 0x3, 0x9a, 0x3, 0x9a, 0x3, 0x9a, 0x3, 0x9a, 0x3, 0x9a, + 0x3, 0x9b, 0x3, 0x9b, 0x3, 0x9b, 0x3, 0x9b, 0x3, 0x9b, 0x3, 0x9c, 0x3, + 0x9c, 0x3, 0x9c, 0x3, 0x9c, 0x3, 0x9c, 0x3, 0x9d, 0x3, 0x9d, 0x3, 0x9d, + 0x3, 0x9d, 0x3, 0x9d, 0x3, 0x9e, 0x3, 0x9e, 0x3, 0x9e, 0x3, 0x9e, 0x3, + 0x9e, 0x3, 0x9e, 0x3, 0x9e, 0x3, 0x9e, 0x3, 0x9f, 0x3, 0x9f, 0x3, 0x9f, + 0x3, 0x9f, 0x3, 0x9f, 0x3, 0x9f, 0x3, 0x9f, 0x3, 0x9f, 0x3, 0x9f, 0x3, + 0x9f, 0x3, 0xa0, 0x3, 0xa0, 0x3, 0xa0, 0x3, 0xa1, 0x3, 0xa1, 0x3, 0xa1, + 0x3, 0xa1, 0x3, 0xa2, 0x3, 0xa2, 0x3, 0xa2, 0x3, 0xa2, 0x3, 0xa2, 0x3, + 0xa2, 0x3, 0xa2, 0x3, 0xa3, 0x3, 0xa3, 0x3, 0xa3, 0x3, 0xa3, 0x3, 0xa3, + 0x3, 0xa3, 0x3, 0xa3, 0x3, 0xa3, 0x3, 0xa3, 0x3, 0xa4, 0x3, 0xa4, 0x3, + 0xa4, 0x3, 0xa4, 0x3, 0xa4, 0x3, 0xa5, 0x3, 0xa5, 0x3, 0xa5, 0x3, 0xa5, + 0x3, 0xa5, 0x3, 0xa5, 0x3, 0xa5, 0x3, 0xa5, 0x3, 0xa5, 0x3, 0xa6, 0x3, + 0xa6, 0x3, 0xa6, 0x3, 0xa6, 0x3, 0xa7, 0x3, 0xa7, 0x3, 0xa7, 0x3, 0xa7, + 0x3, 0xa7, 0x3, 0xa8, 0x3, 0xa8, 0x3, 0xa8, 0x3, 0xa8, 0x3, 0xa8, 0x3, + 0xa8, 0x3, 0xa9, 0x3, 0xa9, 0x3, 0xa9, 0x3, 0xa9, 0x3, 0xa9, 0x3, 0xa9, + 0x3, 0xa9, 0x3, 0xaa, 0x3, 0xaa, 0x3, 0xaa, 0x3, 0xaa, 0x3, 0xab, 0x3, + 0xab, 0x3, 0xab, 0x3, 0xab, 0x3, 0xab, 0x3, 0xab, 0x3, 0xac, 0x3, 0xac, + 0x3, 0xac, 0x3, 0xac, 0x3, 0xac, 0x3, 0xad, 0x3, 0xad, 0x3, 0xad, 0x3, + 0xad, 0x3, 0xad, 0x3, 0xad, 0x3, 0xad, 0x3, 0xae, 0x3, 0xae, 0x3, 0xae, + 0x3, 0xae, 0x3, 0xae, 0x3, 0xaf, 0x3, 0xaf, 0x3, 0xaf, 0x3, 0xaf, 0x3, + 0xaf, 0x3, 0xaf, 0x3, 0xaf, 0x3, 0xb0, 0x3, 0xb0, 0x3, 0xb0, 0x3, 0xb0, + 0x3, 0xb0, 0x3, 0xb0, 0x3, 0xb1, 0x3, 0xb1, 0x3, 0xb1, 0x3, 0xb1, 0x3, + 0xb1, 0x3, 0xb2, 0x3, 0xb2, 0x3, 0xb2, 0x3, 0xb2, 0x3, 0xb2, 0x3, 0xb3, + 0x3, 0xb3, 0x3, 0xb3, 0x3, 0xb3, 0x3, 0xb3, 0x3, 0xb3, 0x3, 0xb4, 0x3, + 0xb4, 0x3, 0xb4, 0x3, 0xb4, 0x3, 0xb4, 0x3, 0xb5, 0x3, 0xb5, 0x3, 0xb5, + 0x3, 0xb5, 0x3, 0xb5, 0x3, 0xb5, 0x3, 0xb5, 0x3, 0xb5, 0x3, 0xb5, 0x3, + 0xb5, 0x5, 0xb5, 0x6b5, 0xa, 0xb5, 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb6, + 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb6, 0x3, 0xb7, 0x3, 0xb7, 0x3, 0xb7, 0x3, + 0xb7, 0x3, 0xb7, 0x3, 0xb8, 0x3, 0xb8, 0x5, 0xb8, 0x6c4, 0xa, 0xb8, + 0x3, 0xb8, 0x3, 0xb8, 0x3, 0xb8, 0x7, 0xb8, 0x6c9, 0xa, 0xb8, 0xc, 0xb8, + 0xe, 0xb8, 0x6cc, 0xb, 0xb8, 0x3, 0xb8, 0x3, 0xb8, 0x3, 0xb8, 0x3, 0xb8, + 0x3, 0xb8, 0x3, 0xb8, 0x3, 0xb8, 0x3, 0xb8, 0x7, 0xb8, 0x6d6, 0xa, 0xb8, + 0xc, 0xb8, 0xe, 0xb8, 0x6d9, 0xb, 0xb8, 0x3, 0xb8, 0x3, 0xb8, 0x3, 0xb8, + 0x3, 0xb8, 0x3, 0xb8, 0x3, 0xb8, 0x3, 0xb8, 0x3, 0xb8, 0x3, 0xb8, 0x3, + 0xb8, 0x7, 0xb8, 0x6e5, 0xa, 0xb8, 0xc, 0xb8, 0xe, 0xb8, 0x6e8, 0xb, + 0xb8, 0x3, 0xb8, 0x3, 0xb8, 0x5, 0xb8, 0x6ec, 0xa, 0xb8, 0x3, 0xb9, + 0x3, 0xb9, 0x3, 0xb9, 0x7, 0xb9, 0x6f1, 0xa, 0xb9, 0xc, 0xb9, 0xe, 0xb9, + 0x6f4, 0xb, 0xb9, 0x3, 0xb9, 0x3, 0xb9, 0x5, 0xb9, 0x6f8, 0xa, 0xb9, + 0x3, 0xb9, 0x3, 0xb9, 0x5, 0xb9, 0x6fc, 0xa, 0xb9, 0x3, 0xb9, 0x6, 0xb9, + 0x6ff, 0xa, 0xb9, 0xd, 0xb9, 0xe, 0xb9, 0x700, 0x3, 0xb9, 0x3, 0xb9, + 0x3, 0xb9, 0x5, 0xb9, 0x706, 0xa, 0xb9, 0x3, 0xb9, 0x3, 0xb9, 0x5, 0xb9, + 0x70a, 0xa, 0xb9, 0x3, 0xb9, 0x6, 0xb9, 0x70d, 0xa, 0xb9, 0xd, 0xb9, + 0xe, 0xb9, 0x70e, 0x3, 0xb9, 0x3, 0xb9, 0x3, 0xb9, 0x7, 0xb9, 0x714, + 0xa, 0xb9, 0xc, 0xb9, 0xe, 0xb9, 0x717, 0xb, 0xb9, 0x3, 0xb9, 0x3, 0xb9, + 0x3, 0xb9, 0x5, 0xb9, 0x71c, 0xa, 0xb9, 0x3, 0xb9, 0x6, 0xb9, 0x71f, + 0xa, 0xb9, 0xd, 0xb9, 0xe, 0xb9, 0x720, 0x3, 0xb9, 0x3, 0xb9, 0x3, 0xb9, + 0x3, 0xb9, 0x3, 0xb9, 0x5, 0xb9, 0x728, 0xa, 0xb9, 0x3, 0xb9, 0x6, 0xb9, + 0x72b, 0xa, 0xb9, 0xd, 0xb9, 0xe, 0xb9, 0x72c, 0x3, 0xb9, 0x3, 0xb9, + 0x3, 0xb9, 0x3, 0xb9, 0x5, 0xb9, 0x733, 0xa, 0xb9, 0x3, 0xb9, 0x6, 0xb9, + 0x736, 0xa, 0xb9, 0xd, 0xb9, 0xe, 0xb9, 0x737, 0x5, 0xb9, 0x73a, 0xa, + 0xb9, 0x3, 0xba, 0x3, 0xba, 0x6, 0xba, 0x73e, 0xa, 0xba, 0xd, 0xba, + 0xe, 0xba, 0x73f, 0x3, 0xbb, 0x6, 0xbb, 0x743, 0xa, 0xbb, 0xd, 0xbb, + 0xe, 0xbb, 0x744, 0x3, 0xbc, 0x3, 0xbc, 0x3, 0xbc, 0x6, 0xbc, 0x74a, + 0xa, 0xbc, 0xd, 0xbc, 0xe, 0xbc, 0x74b, 0x3, 0xbd, 0x3, 0xbd, 0x3, 0xbd, + 0x3, 0xbd, 0x3, 0xbd, 0x3, 0xbd, 0x3, 0xbd, 0x3, 0xbd, 0x7, 0xbd, 0x756, + 0xa, 0xbd, 0xc, 0xbd, 0xe, 0xbd, 0x759, 0xb, 0xbd, 0x3, 0xbd, 0x3, 0xbd, + 0x3, 0xbe, 0x3, 0xbe, 0x3, 0xbf, 0x3, 0xbf, 0x3, 0xc0, 0x3, 0xc0, 0x3, + 0xc1, 0x3, 0xc1, 0x3, 0xc2, 0x3, 0xc2, 0x3, 0xc3, 0x3, 0xc3, 0x3, 0xc4, + 0x3, 0xc4, 0x3, 0xc5, 0x3, 0xc5, 0x3, 0xc6, 0x3, 0xc6, 0x3, 0xc7, 0x3, + 0xc7, 0x3, 0xc8, 0x3, 0xc8, 0x3, 0xc9, 0x3, 0xc9, 0x3, 0xca, 0x3, 0xca, + 0x3, 0xcb, 0x3, 0xcb, 0x3, 0xcc, 0x3, 0xcc, 0x3, 0xcd, 0x3, 0xcd, 0x3, + 0xce, 0x3, 0xce, 0x3, 0xcf, 0x3, 0xcf, 0x3, 0xd0, 0x3, 0xd0, 0x3, 0xd1, + 0x3, 0xd1, 0x3, 0xd2, 0x3, 0xd2, 0x3, 0xd3, 0x3, 0xd3, 0x3, 0xd4, 0x3, + 0xd4, 0x3, 0xd5, 0x3, 0xd5, 0x3, 0xd6, 0x3, 0xd6, 0x3, 0xd7, 0x3, 0xd7, + 0x3, 0xd8, 0x3, 0xd8, 0x3, 0xd9, 0x3, 0xd9, 0x3, 0xda, 0x3, 0xda, 0x3, + 0xdb, 0x3, 0xdb, 0x3, 0xdc, 0x3, 0xdc, 0x3, 0xdc, 0x3, 0xdd, 0x3, 0xdd, + 0x3, 0xde, 0x3, 0xde, 0x3, 0xdf, 0x3, 0xdf, 0x3, 0xe0, 0x3, 0xe0, 0x3, + 0xe1, 0x3, 0xe1, 0x3, 0xe2, 0x3, 0xe2, 0x3, 0xe2, 0x3, 0xe3, 0x3, 0xe3, + 0x3, 0xe4, 0x3, 0xe4, 0x3, 0xe5, 0x3, 0xe5, 0x3, 0xe5, 0x3, 0xe6, 0x3, + 0xe6, 0x3, 0xe7, 0x3, 0xe7, 0x3, 0xe7, 0x3, 0xe8, 0x3, 0xe8, 0x3, 0xe9, + 0x3, 0xe9, 0x3, 0xea, 0x3, 0xea, 0x3, 0xeb, 0x3, 0xeb, 0x3, 0xeb, 0x3, + 0xec, 0x3, 0xec, 0x3, 0xed, 0x3, 0xed, 0x3, 0xee, 0x3, 0xee, 0x3, 0xee, + 0x3, 0xee, 0x5, 0xee, 0x7c6, 0xa, 0xee, 0x3, 0xef, 0x3, 0xef, 0x3, 0xf0, + 0x3, 0xf0, 0x3, 0xf1, 0x3, 0xf1, 0x3, 0xf2, 0x3, 0xf2, 0x3, 0xf3, 0x3, + 0xf3, 0x3, 0xf4, 0x3, 0xf4, 0x3, 0xf5, 0x3, 0xf5, 0x3, 0xf6, 0x3, 0xf6, + 0x3, 0xf7, 0x3, 0xf7, 0x3, 0xf8, 0x3, 0xf8, 0x3, 0xf9, 0x3, 0xf9, 0x3, + 0xfa, 0x3, 0xfa, 0x3, 0xfa, 0x3, 0xfa, 0x7, 0xfa, 0x7e2, 0xa, 0xfa, + 0xc, 0xfa, 0xe, 0xfa, 0x7e5, 0xb, 0xfa, 0x3, 0xfa, 0x3, 0xfa, 0x3, 0xfa, + 0x3, 0xfa, 0x3, 0xfa, 0x3, 0xfb, 0x3, 0xfb, 0x3, 0xfb, 0x3, 0xfb, 0x7, + 0xfb, 0x7f0, 0xa, 0xfb, 0xc, 0xfb, 0xe, 0xfb, 0x7f3, 0xb, 0xfb, 0x3, + 0xfb, 0x5, 0xfb, 0x7f6, 0xa, 0xfb, 0x3, 0xfb, 0x3, 0xfb, 0x3, 0xfc, + 0x3, 0xfc, 0x3, 0xfc, 0x3, 0xfc, 0x3, 0x7e3, 0x2, 0xfd, 0x3, 0x3, 0x5, + 0x4, 0x7, 0x5, 0x9, 0x6, 0xb, 0x7, 0xd, 0x8, 0xf, 0x9, 0x11, 0xa, 0x13, + 0xb, 0x15, 0xc, 0x17, 0xd, 0x19, 0xe, 0x1b, 0xf, 0x1d, 0x10, 0x1f, 0x11, + 0x21, 0x12, 0x23, 0x13, 0x25, 0x14, 0x27, 0x15, 0x29, 0x16, 0x2b, 0x17, + 0x2d, 0x18, 0x2f, 0x19, 0x31, 0x1a, 0x33, 0x1b, 0x35, 0x1c, 0x37, 0x1d, + 0x39, 0x1e, 0x3b, 0x1f, 0x3d, 0x20, 0x3f, 0x21, 0x41, 0x22, 0x43, 0x23, + 0x45, 0x24, 0x47, 0x25, 0x49, 0x26, 0x4b, 0x27, 0x4d, 0x28, 0x4f, 0x29, + 0x51, 0x2a, 0x53, 0x2b, 0x55, 0x2c, 0x57, 0x2d, 0x59, 0x2e, 0x5b, 0x2f, + 0x5d, 0x30, 0x5f, 0x31, 0x61, 0x32, 0x63, 0x33, 0x65, 0x34, 0x67, 0x35, + 0x69, 0x36, 0x6b, 0x37, 0x6d, 0x38, 0x6f, 0x39, 0x71, 0x3a, 0x73, 0x3b, + 0x75, 0x3c, 0x77, 0x3d, 0x79, 0x3e, 0x7b, 0x3f, 0x7d, 0x40, 0x7f, 0x41, + 0x81, 0x42, 0x83, 0x43, 0x85, 0x44, 0x87, 0x45, 0x89, 0x46, 0x8b, 0x47, + 0x8d, 0x48, 0x8f, 0x49, 0x91, 0x4a, 0x93, 0x4b, 0x95, 0x4c, 0x97, 0x4d, + 0x99, 0x4e, 0x9b, 0x4f, 0x9d, 0x50, 0x9f, 0x51, 0xa1, 0x52, 0xa3, 0x53, + 0xa5, 0x54, 0xa7, 0x55, 0xa9, 0x56, 0xab, 0x57, 0xad, 0x58, 0xaf, 0x59, + 0xb1, 0x5a, 0xb3, 0x5b, 0xb5, 0x5c, 0xb7, 0x5d, 0xb9, 0x5e, 0xbb, 0x5f, + 0xbd, 0x60, 0xbf, 0x61, 0xc1, 0x62, 0xc3, 0x63, 0xc5, 0x64, 0xc7, 0x65, + 0xc9, 0x66, 0xcb, 0x67, 0xcd, 0x68, 0xcf, 0x69, 0xd1, 0x6a, 0xd3, 0x6b, + 0xd5, 0x6c, 0xd7, 0x6d, 0xd9, 0x6e, 0xdb, 0x6f, 0xdd, 0x70, 0xdf, 0x71, + 0xe1, 0x72, 0xe3, 0x73, 0xe5, 0x74, 0xe7, 0x75, 0xe9, 0x76, 0xeb, 0x77, + 0xed, 0x78, 0xef, 0x79, 0xf1, 0x7a, 0xf3, 0x7b, 0xf5, 0x7c, 0xf7, 0x7d, + 0xf9, 0x7e, 0xfb, 0x7f, 0xfd, 0x80, 0xff, 0x81, 0x101, 0x82, 0x103, + 0x83, 0x105, 0x84, 0x107, 0x85, 0x109, 0x86, 0x10b, 0x87, 0x10d, 0x88, + 0x10f, 0x89, 0x111, 0x8a, 0x113, 0x8b, 0x115, 0x8c, 0x117, 0x8d, 0x119, + 0x8e, 0x11b, 0x8f, 0x11d, 0x90, 0x11f, 0x91, 0x121, 0x92, 0x123, 0x93, + 0x125, 0x94, 0x127, 0x95, 0x129, 0x96, 0x12b, 0x97, 0x12d, 0x98, 0x12f, + 0x99, 0x131, 0x9a, 0x133, 0x9b, 0x135, 0x9c, 0x137, 0x9d, 0x139, 0x9e, + 0x13b, 0x9f, 0x13d, 0xa0, 0x13f, 0xa1, 0x141, 0xa2, 0x143, 0xa3, 0x145, + 0xa4, 0x147, 0xa5, 0x149, 0xa6, 0x14b, 0xa7, 0x14d, 0xa8, 0x14f, 0xa9, + 0x151, 0xaa, 0x153, 0xab, 0x155, 0xac, 0x157, 0xad, 0x159, 0xae, 0x15b, + 0xaf, 0x15d, 0xb0, 0x15f, 0xb1, 0x161, 0xb2, 0x163, 0xb3, 0x165, 0xb4, + 0x167, 0xb5, 0x169, 0xb6, 0x16b, 0xb7, 0x16d, 0xb8, 0x16f, 0xb9, 0x171, + 0xba, 0x173, 0xbb, 0x175, 0xbc, 0x177, 0xbd, 0x179, 0xbe, 0x17b, 0x2, + 0x17d, 0x2, 0x17f, 0x2, 0x181, 0x2, 0x183, 0x2, 0x185, 0x2, 0x187, 0x2, + 0x189, 0x2, 0x18b, 0x2, 0x18d, 0x2, 0x18f, 0x2, 0x191, 0x2, 0x193, 0x2, + 0x195, 0x2, 0x197, 0x2, 0x199, 0x2, 0x19b, 0x2, 0x19d, 0x2, 0x19f, 0x2, + 0x1a1, 0x2, 0x1a3, 0x2, 0x1a5, 0x2, 0x1a7, 0x2, 0x1a9, 0x2, 0x1ab, 0x2, + 0x1ad, 0x2, 0x1af, 0x2, 0x1b1, 0x2, 0x1b3, 0x2, 0x1b5, 0x2, 0x1b7, 0xbf, + 0x1b9, 0xc0, 0x1bb, 0xc1, 0x1bd, 0xc2, 0x1bf, 0xc3, 0x1c1, 0xc4, 0x1c3, + 0xc5, 0x1c5, 0xc6, 0x1c7, 0xc7, 0x1c9, 0xc8, 0x1cb, 0xc9, 0x1cd, 0xca, + 0x1cf, 0xcb, 0x1d1, 0xcc, 0x1d3, 0xcd, 0x1d5, 0xce, 0x1d7, 0xcf, 0x1d9, + 0xd0, 0x1db, 0xd1, 0x1dd, 0xd2, 0x1df, 0xd3, 0x1e1, 0xd4, 0x1e3, 0xd5, + 0x1e5, 0xd6, 0x1e7, 0xd7, 0x1e9, 0xd8, 0x1eb, 0xd9, 0x1ed, 0xda, 0x1ef, + 0xdb, 0x1f1, 0xdc, 0x1f3, 0xdd, 0x1f5, 0xde, 0x1f7, 0xdf, 0x3, 0x2, + 0x26, 0x4, 0x2, 0x5e, 0x5e, 0x62, 0x62, 0x4, 0x2, 0x24, 0x24, 0x5e, + 0x5e, 0x4, 0x2, 0x29, 0x29, 0x5e, 0x5e, 0x4, 0x2, 0x43, 0x43, 0x63, + 0x63, 0x4, 0x2, 0x44, 0x44, 0x64, 0x64, 0x4, 0x2, 0x45, 0x45, 0x65, + 0x65, 0x4, 0x2, 0x46, 0x46, 0x66, 0x66, 0x4, 0x2, 0x47, 0x47, 0x67, + 0x67, 0x4, 0x2, 0x48, 0x48, 0x68, 0x68, 0x4, 0x2, 0x49, 0x49, 0x69, + 0x69, 0x4, 0x2, 0x4a, 0x4a, 0x6a, 0x6a, 0x4, 0x2, 0x4b, 0x4b, 0x6b, + 0x6b, 0x4, 0x2, 0x4c, 0x4c, 0x6c, 0x6c, 0x4, 0x2, 0x4d, 0x4d, 0x6d, + 0x6d, 0x4, 0x2, 0x4e, 0x4e, 0x6e, 0x6e, 0x4, 0x2, 0x4f, 0x4f, 0x6f, + 0x6f, 0x4, 0x2, 0x50, 0x50, 0x70, 0x70, 0x4, 0x2, 0x51, 0x51, 0x71, + 0x71, 0x4, 0x2, 0x52, 0x52, 0x72, 0x72, 0x4, 0x2, 0x53, 0x53, 0x73, + 0x73, 0x4, 0x2, 0x54, 0x54, 0x74, 0x74, 0x4, 0x2, 0x55, 0x55, 0x75, + 0x75, 0x4, 0x2, 0x56, 0x56, 0x76, 0x76, 0x4, 0x2, 0x57, 0x57, 0x77, + 0x77, 0x4, 0x2, 0x58, 0x58, 0x78, 0x78, 0x4, 0x2, 0x59, 0x59, 0x79, + 0x79, 0x4, 0x2, 0x5a, 0x5a, 0x7a, 0x7a, 0x4, 0x2, 0x5b, 0x5b, 0x7b, + 0x7b, 0x4, 0x2, 0x5c, 0x5c, 0x7c, 0x7c, 0x4, 0x2, 0x43, 0x5c, 0x63, + 0x7c, 0x3, 0x2, 0x32, 0x39, 0x3, 0x2, 0x32, 0x3b, 0x5, 0x2, 0x32, 0x3b, + 0x43, 0x48, 0x63, 0x68, 0x4, 0x2, 0xc, 0xc, 0xf, 0xf, 0x4, 0x3, 0xc, + 0xc, 0xf, 0xf, 0x4, 0x2, 0xb, 0xf, 0x22, 0x22, 0x2, 0x80d, 0x2, 0x3, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x5, 0x3, 0x2, 0x2, 0x2, 0x2, 0x7, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x9, 0x3, 0x2, 0x2, 0x2, 0x2, 0xb, 0x3, 0x2, 0x2, 0x2, + 0x2, 0xd, 0x3, 0x2, 0x2, 0x2, 0x2, 0xf, 0x3, 0x2, 0x2, 0x2, 0x2, 0x11, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x13, 0x3, 0x2, 0x2, 0x2, 0x2, 0x15, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x17, 0x3, 0x2, 0x2, 0x2, 0x2, 0x19, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x1b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1f, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x21, 0x3, 0x2, 0x2, 0x2, 0x2, 0x23, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x25, 0x3, 0x2, 0x2, 0x2, 0x2, 0x27, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x29, 0x3, 0x2, 0x2, 0x2, 0x2, 0x2b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x2d, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x2f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x31, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x33, 0x3, 0x2, 0x2, 0x2, 0x2, 0x35, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x37, 0x3, 0x2, 0x2, 0x2, 0x2, 0x39, 0x3, 0x2, 0x2, 0x2, 0x2, 0x3b, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x3d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x3f, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x41, 0x3, 0x2, 0x2, 0x2, 0x2, 0x43, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x45, 0x3, 0x2, 0x2, 0x2, 0x2, 0x47, 0x3, 0x2, 0x2, 0x2, 0x2, 0x49, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x4b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x4d, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x4f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x51, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x53, 0x3, 0x2, 0x2, 0x2, 0x2, 0x55, 0x3, 0x2, 0x2, 0x2, 0x2, 0x57, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x59, 0x3, 0x2, 0x2, 0x2, 0x2, 0x5b, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x5d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x5f, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x61, 0x3, 0x2, 0x2, 0x2, 0x2, 0x63, 0x3, 0x2, 0x2, 0x2, 0x2, 0x65, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x67, 0x3, 0x2, 0x2, 0x2, 0x2, 0x69, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x6b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x6d, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x6f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x71, 0x3, 0x2, 0x2, 0x2, 0x2, 0x73, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x75, 0x3, 0x2, 0x2, 0x2, 0x2, 0x77, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x79, 0x3, 0x2, 0x2, 0x2, 0x2, 0x7b, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x7d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x7f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x81, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x83, 0x3, 0x2, 0x2, 0x2, 0x2, 0x85, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x87, 0x3, 0x2, 0x2, 0x2, 0x2, 0x89, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x8b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x8d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x8f, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x91, 0x3, 0x2, 0x2, 0x2, 0x2, 0x93, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x95, 0x3, 0x2, 0x2, 0x2, 0x2, 0x97, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x99, 0x3, 0x2, 0x2, 0x2, 0x2, 0x9b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x9d, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x9f, 0x3, 0x2, 0x2, 0x2, 0x2, 0xa1, 0x3, 0x2, + 0x2, 0x2, 0x2, 0xa3, 0x3, 0x2, 0x2, 0x2, 0x2, 0xa5, 0x3, 0x2, 0x2, 0x2, + 0x2, 0xa7, 0x3, 0x2, 0x2, 0x2, 0x2, 0xa9, 0x3, 0x2, 0x2, 0x2, 0x2, 0xab, + 0x3, 0x2, 0x2, 0x2, 0x2, 0xad, 0x3, 0x2, 0x2, 0x2, 0x2, 0xaf, 0x3, 0x2, + 0x2, 0x2, 0x2, 0xb1, 0x3, 0x2, 0x2, 0x2, 0x2, 0xb3, 0x3, 0x2, 0x2, 0x2, + 0x2, 0xb5, 0x3, 0x2, 0x2, 0x2, 0x2, 0xb7, 0x3, 0x2, 0x2, 0x2, 0x2, 0xb9, + 0x3, 0x2, 0x2, 0x2, 0x2, 0xbb, 0x3, 0x2, 0x2, 0x2, 0x2, 0xbd, 0x3, 0x2, + 0x2, 0x2, 0x2, 0xbf, 0x3, 0x2, 0x2, 0x2, 0x2, 0xc1, 0x3, 0x2, 0x2, 0x2, + 0x2, 0xc3, 0x3, 0x2, 0x2, 0x2, 0x2, 0xc5, 0x3, 0x2, 0x2, 0x2, 0x2, 0xc7, + 0x3, 0x2, 0x2, 0x2, 0x2, 0xc9, 0x3, 0x2, 0x2, 0x2, 0x2, 0xcb, 0x3, 0x2, + 0x2, 0x2, 0x2, 0xcd, 0x3, 0x2, 0x2, 0x2, 0x2, 0xcf, 0x3, 0x2, 0x2, 0x2, + 0x2, 0xd1, 0x3, 0x2, 0x2, 0x2, 0x2, 0xd3, 0x3, 0x2, 0x2, 0x2, 0x2, 0xd5, + 0x3, 0x2, 0x2, 0x2, 0x2, 0xd7, 0x3, 0x2, 0x2, 0x2, 0x2, 0xd9, 0x3, 0x2, + 0x2, 0x2, 0x2, 0xdb, 0x3, 0x2, 0x2, 0x2, 0x2, 0xdd, 0x3, 0x2, 0x2, 0x2, + 0x2, 0xdf, 0x3, 0x2, 0x2, 0x2, 0x2, 0xe1, 0x3, 0x2, 0x2, 0x2, 0x2, 0xe3, + 0x3, 0x2, 0x2, 0x2, 0x2, 0xe5, 0x3, 0x2, 0x2, 0x2, 0x2, 0xe7, 0x3, 0x2, + 0x2, 0x2, 0x2, 0xe9, 0x3, 0x2, 0x2, 0x2, 0x2, 0xeb, 0x3, 0x2, 0x2, 0x2, + 0x2, 0xed, 0x3, 0x2, 0x2, 0x2, 0x2, 0xef, 0x3, 0x2, 0x2, 0x2, 0x2, 0xf1, + 0x3, 0x2, 0x2, 0x2, 0x2, 0xf3, 0x3, 0x2, 0x2, 0x2, 0x2, 0xf5, 0x3, 0x2, + 0x2, 0x2, 0x2, 0xf7, 0x3, 0x2, 0x2, 0x2, 0x2, 0xf9, 0x3, 0x2, 0x2, 0x2, + 0x2, 0xfb, 0x3, 0x2, 0x2, 0x2, 0x2, 0xfd, 0x3, 0x2, 0x2, 0x2, 0x2, 0xff, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x101, 0x3, 0x2, 0x2, 0x2, 0x2, 0x103, 0x3, + 0x2, 0x2, 0x2, 0x2, 0x105, 0x3, 0x2, 0x2, 0x2, 0x2, 0x107, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x109, 0x3, 0x2, 0x2, 0x2, 0x2, 0x10b, 0x3, 0x2, 0x2, + 0x2, 0x2, 0x10d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x10f, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x111, 0x3, 0x2, 0x2, 0x2, 0x2, 0x113, 0x3, 0x2, 0x2, 0x2, 0x2, + 0x115, 0x3, 0x2, 0x2, 0x2, 0x2, 0x117, 0x3, 0x2, 0x2, 0x2, 0x2, 0x119, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x11b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x11d, 0x3, + 0x2, 0x2, 0x2, 0x2, 0x11f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x121, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x123, 0x3, 0x2, 0x2, 0x2, 0x2, 0x125, 0x3, 0x2, 0x2, + 0x2, 0x2, 0x127, 0x3, 0x2, 0x2, 0x2, 0x2, 0x129, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x12b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x12d, 0x3, 0x2, 0x2, 0x2, 0x2, + 0x12f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x131, 0x3, 0x2, 0x2, 0x2, 0x2, 0x133, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x135, 0x3, 0x2, 0x2, 0x2, 0x2, 0x137, 0x3, + 0x2, 0x2, 0x2, 0x2, 0x139, 0x3, 0x2, 0x2, 0x2, 0x2, 0x13b, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x13d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x13f, 0x3, 0x2, 0x2, + 0x2, 0x2, 0x141, 0x3, 0x2, 0x2, 0x2, 0x2, 0x143, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x145, 0x3, 0x2, 0x2, 0x2, 0x2, 0x147, 0x3, 0x2, 0x2, 0x2, 0x2, + 0x149, 0x3, 0x2, 0x2, 0x2, 0x2, 0x14b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x14d, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x14f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x151, 0x3, + 0x2, 0x2, 0x2, 0x2, 0x153, 0x3, 0x2, 0x2, 0x2, 0x2, 0x155, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x157, 0x3, 0x2, 0x2, 0x2, 0x2, 0x159, 0x3, 0x2, 0x2, + 0x2, 0x2, 0x15b, 0x3, 0x2, 0x2, 0x2, 0x2, 0x15d, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x15f, 0x3, 0x2, 0x2, 0x2, 0x2, 0x161, 0x3, 0x2, 0x2, 0x2, 0x2, + 0x163, 0x3, 0x2, 0x2, 0x2, 0x2, 0x165, 0x3, 0x2, 0x2, 0x2, 0x2, 0x167, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x169, 0x3, 0x2, 0x2, 0x2, 0x2, 0x16b, 0x3, + 0x2, 0x2, 0x2, 0x2, 0x16d, 0x3, 0x2, 0x2, 0x2, 0x2, 0x16f, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x171, 0x3, 0x2, 0x2, 0x2, 0x2, 0x173, 0x3, 0x2, 0x2, + 0x2, 0x2, 0x175, 0x3, 0x2, 0x2, 0x2, 0x2, 0x177, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x179, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1b7, 0x3, 0x2, 0x2, 0x2, 0x2, + 0x1b9, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1bb, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1bd, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x1bf, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1c1, 0x3, + 0x2, 0x2, 0x2, 0x2, 0x1c3, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1c5, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x1c7, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1c9, 0x3, 0x2, 0x2, + 0x2, 0x2, 0x1cb, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1cd, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x1cf, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1d1, 0x3, 0x2, 0x2, 0x2, 0x2, + 0x1d3, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1d5, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1d7, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x1d9, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1db, 0x3, + 0x2, 0x2, 0x2, 0x2, 0x1dd, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1df, 0x3, 0x2, + 0x2, 0x2, 0x2, 0x1e1, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1e3, 0x3, 0x2, 0x2, + 0x2, 0x2, 0x1e5, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1e7, 0x3, 0x2, 0x2, 0x2, + 0x2, 0x1e9, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1eb, 0x3, 0x2, 0x2, 0x2, 0x2, + 0x1ed, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1ef, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1f1, + 0x3, 0x2, 0x2, 0x2, 0x2, 0x1f3, 0x3, 0x2, 0x2, 0x2, 0x2, 0x1f5, 0x3, + 0x2, 0x2, 0x2, 0x2, 0x1f7, 0x3, 0x2, 0x2, 0x2, 0x3, 0x1f9, 0x3, 0x2, + 0x2, 0x2, 0x5, 0x1fd, 0x3, 0x2, 0x2, 0x2, 0x7, 0x203, 0x3, 0x2, 0x2, + 0x2, 0x9, 0x209, 0x3, 0x2, 0x2, 0x2, 0xb, 0x20d, 0x3, 0x2, 0x2, 0x2, + 0xd, 0x213, 0x3, 0x2, 0x2, 0x2, 0xf, 0x217, 0x3, 0x2, 0x2, 0x2, 0x11, + 0x21c, 0x3, 0x2, 0x2, 0x2, 0x13, 0x220, 0x3, 0x2, 0x2, 0x2, 0x15, 0x226, + 0x3, 0x2, 0x2, 0x2, 0x17, 0x237, 0x3, 0x2, 0x2, 0x2, 0x19, 0x239, 0x3, + 0x2, 0x2, 0x2, 0x1b, 0x23e, 0x3, 0x2, 0x2, 0x2, 0x1d, 0x244, 0x3, 0x2, + 0x2, 0x2, 0x1f, 0x24b, 0x3, 0x2, 0x2, 0x2, 0x21, 0x253, 0x3, 0x2, 0x2, + 0x2, 0x23, 0x258, 0x3, 0x2, 0x2, 0x2, 0x25, 0x25b, 0x3, 0x2, 0x2, 0x2, + 0x27, 0x260, 0x3, 0x2, 0x2, 0x2, 0x29, 0x265, 0x3, 0x2, 0x2, 0x2, 0x2b, + 0x26b, 0x3, 0x2, 0x2, 0x2, 0x2d, 0x271, 0x3, 0x2, 0x2, 0x2, 0x2f, 0x279, + 0x3, 0x2, 0x2, 0x2, 0x31, 0x27f, 0x3, 0x2, 0x2, 0x2, 0x33, 0x287, 0x3, + 0x2, 0x2, 0x2, 0x35, 0x28e, 0x3, 0x2, 0x2, 0x2, 0x37, 0x296, 0x3, 0x2, + 0x2, 0x2, 0x39, 0x2a1, 0x3, 0x2, 0x2, 0x2, 0x3b, 0x2a8, 0x3, 0x2, 0x2, + 0x2, 0x3d, 0x2ae, 0x3, 0x2, 0x2, 0x2, 0x3f, 0x2b3, 0x3, 0x2, 0x2, 0x2, + 0x41, 0x2bc, 0x3, 0x2, 0x2, 0x2, 0x43, 0x2c6, 0x3, 0x2, 0x2, 0x2, 0x45, + 0x2cb, 0x3, 0x2, 0x2, 0x2, 0x47, 0x2cf, 0x3, 0x2, 0x2, 0x2, 0x49, 0x2db, + 0x3, 0x2, 0x2, 0x2, 0x4b, 0x2e3, 0x3, 0x2, 0x2, 0x2, 0x4d, 0x2e9, 0x3, + 0x2, 0x2, 0x2, 0x4f, 0x2f0, 0x3, 0x2, 0x2, 0x2, 0x51, 0x2f5, 0x3, 0x2, + 0x2, 0x2, 0x53, 0x300, 0x3, 0x2, 0x2, 0x2, 0x55, 0x309, 0x3, 0x2, 0x2, + 0x2, 0x57, 0x310, 0x3, 0x2, 0x2, 0x2, 0x59, 0x31d, 0x3, 0x2, 0x2, 0x2, + 0x5b, 0x328, 0x3, 0x2, 0x2, 0x2, 0x5d, 0x32d, 0x3, 0x2, 0x2, 0x2, 0x5f, + 0x336, 0x3, 0x2, 0x2, 0x2, 0x61, 0x342, 0x3, 0x2, 0x2, 0x2, 0x63, 0x347, + 0x3, 0x2, 0x2, 0x2, 0x65, 0x34c, 0x3, 0x2, 0x2, 0x2, 0x67, 0x350, 0x3, + 0x2, 0x2, 0x2, 0x69, 0x357, 0x3, 0x2, 0x2, 0x2, 0x6b, 0x35e, 0x3, 0x2, + 0x2, 0x2, 0x6d, 0x365, 0x3, 0x2, 0x2, 0x2, 0x6f, 0x36d, 0x3, 0x2, 0x2, + 0x2, 0x71, 0x378, 0x3, 0x2, 0x2, 0x2, 0x73, 0x380, 0x3, 0x2, 0x2, 0x2, + 0x75, 0x388, 0x3, 0x2, 0x2, 0x2, 0x77, 0x38e, 0x3, 0x2, 0x2, 0x2, 0x79, + 0x394, 0x3, 0x2, 0x2, 0x2, 0x7b, 0x39a, 0x3, 0x2, 0x2, 0x2, 0x7d, 0x39e, + 0x3, 0x2, 0x2, 0x2, 0x7f, 0x3a5, 0x3, 0x2, 0x2, 0x2, 0x81, 0x3ac, 0x3, + 0x2, 0x2, 0x2, 0x83, 0x3b1, 0x3, 0x2, 0x2, 0x2, 0x85, 0x3b6, 0x3, 0x2, + 0x2, 0x2, 0x87, 0x3bf, 0x3, 0x2, 0x2, 0x2, 0x89, 0x3c6, 0x3, 0x2, 0x2, + 0x2, 0x8b, 0x3d2, 0x3, 0x2, 0x2, 0x2, 0x8d, 0x3d8, 0x3, 0x2, 0x2, 0x2, + 0x8f, 0x3df, 0x3, 0x2, 0x2, 0x2, 0x91, 0x3ec, 0x3, 0x2, 0x2, 0x2, 0x93, + 0x3f1, 0x3, 0x2, 0x2, 0x2, 0x95, 0x3f4, 0x3, 0x2, 0x2, 0x2, 0x97, 0x3f7, + 0x3, 0x2, 0x2, 0x2, 0x99, 0x3fd, 0x3, 0x2, 0x2, 0x2, 0x9b, 0x400, 0x3, + 0x2, 0x2, 0x2, 0x9d, 0x413, 0x3, 0x2, 0x2, 0x2, 0x9f, 0x415, 0x3, 0x2, + 0x2, 0x2, 0xa1, 0x41f, 0x3, 0x2, 0x2, 0x2, 0xa3, 0x425, 0x3, 0x2, 0x2, + 0x2, 0xa5, 0x42c, 0x3, 0x2, 0x2, 0x2, 0xa7, 0x435, 0x3, 0x2, 0x2, 0x2, + 0xa9, 0x43a, 0x3, 0x2, 0x2, 0x2, 0xab, 0x43d, 0x3, 0x2, 0x2, 0x2, 0xad, + 0x44a, 0x3, 0x2, 0x2, 0x2, 0xaf, 0x44f, 0x3, 0x2, 0x2, 0x2, 0xb1, 0x453, + 0x3, 0x2, 0x2, 0x2, 0xb3, 0x458, 0x3, 0x2, 0x2, 0x2, 0xb5, 0x45d, 0x3, + 0x2, 0x2, 0x2, 0xb7, 0x464, 0x3, 0x2, 0x2, 0x2, 0xb9, 0x46c, 0x3, 0x2, + 0x2, 0x2, 0xbb, 0x471, 0x3, 0x2, 0x2, 0x2, 0xbd, 0x47a, 0x3, 0x2, 0x2, + 0x2, 0xbf, 0x47f, 0x3, 0x2, 0x2, 0x2, 0xc1, 0x485, 0x3, 0x2, 0x2, 0x2, + 0xc3, 0x48a, 0x3, 0x2, 0x2, 0x2, 0xc5, 0x490, 0x3, 0x2, 0x2, 0x2, 0xc7, + 0x495, 0x3, 0x2, 0x2, 0x2, 0xc9, 0x4a2, 0x3, 0x2, 0x2, 0x2, 0xcb, 0x4ae, + 0x3, 0x2, 0x2, 0x2, 0xcd, 0x4b2, 0x3, 0x2, 0x2, 0x2, 0xcf, 0x4b9, 0x3, + 0x2, 0x2, 0x2, 0xd1, 0x4bd, 0x3, 0x2, 0x2, 0x2, 0xd3, 0x4c4, 0x3, 0x2, + 0x2, 0x2, 0xd5, 0x4cb, 0x3, 0x2, 0x2, 0x2, 0xd7, 0x4d1, 0x3, 0x2, 0x2, + 0x2, 0xd9, 0x4d6, 0x3, 0x2, 0x2, 0x2, 0xdb, 0x4df, 0x3, 0x2, 0x2, 0x2, + 0xdd, 0x4e3, 0x3, 0x2, 0x2, 0x2, 0xdf, 0x4e6, 0x3, 0x2, 0x2, 0x2, 0xe1, + 0x4ea, 0x3, 0x2, 0x2, 0x2, 0xe3, 0x4ef, 0x3, 0x2, 0x2, 0x2, 0xe5, 0x4f5, + 0x3, 0x2, 0x2, 0x2, 0xe7, 0x4fc, 0x3, 0x2, 0x2, 0x2, 0xe9, 0x4ff, 0x3, + 0x2, 0x2, 0x2, 0xeb, 0x508, 0x3, 0x2, 0x2, 0x2, 0xed, 0x50b, 0x3, 0x2, + 0x2, 0x2, 0xef, 0x511, 0x3, 0x2, 0x2, 0x2, 0xf1, 0x517, 0x3, 0x2, 0x2, + 0x2, 0xf3, 0x51f, 0x3, 0x2, 0x2, 0x2, 0xf5, 0x529, 0x3, 0x2, 0x2, 0x2, + 0xf7, 0x532, 0x3, 0x2, 0x2, 0x2, 0xf9, 0x53b, 0x3, 0x2, 0x2, 0x2, 0xfb, + 0x543, 0x3, 0x2, 0x2, 0x2, 0xfd, 0x54e, 0x3, 0x2, 0x2, 0x2, 0xff, 0x556, + 0x3, 0x2, 0x2, 0x2, 0x101, 0x55c, 0x3, 0x2, 0x2, 0x2, 0x103, 0x563, + 0x3, 0x2, 0x2, 0x2, 0x105, 0x56a, 0x3, 0x2, 0x2, 0x2, 0x107, 0x571, + 0x3, 0x2, 0x2, 0x2, 0x109, 0x579, 0x3, 0x2, 0x2, 0x2, 0x10b, 0x581, + 0x3, 0x2, 0x2, 0x2, 0x10d, 0x58c, 0x3, 0x2, 0x2, 0x2, 0x10f, 0x592, + 0x3, 0x2, 0x2, 0x2, 0x111, 0x599, 0x3, 0x2, 0x2, 0x2, 0x113, 0x5a0, + 0x3, 0x2, 0x2, 0x2, 0x115, 0x5a7, 0x3, 0x2, 0x2, 0x2, 0x117, 0x5ae, + 0x3, 0x2, 0x2, 0x2, 0x119, 0x5b3, 0x3, 0x2, 0x2, 0x2, 0x11b, 0x5b9, + 0x3, 0x2, 0x2, 0x2, 0x11d, 0x5bd, 0x3, 0x2, 0x2, 0x2, 0x11f, 0x5c6, + 0x3, 0x2, 0x2, 0x2, 0x121, 0x5cb, 0x3, 0x2, 0x2, 0x2, 0x123, 0x5d2, + 0x3, 0x2, 0x2, 0x2, 0x125, 0x5d8, 0x3, 0x2, 0x2, 0x2, 0x127, 0x5dd, + 0x3, 0x2, 0x2, 0x2, 0x129, 0x5e7, 0x3, 0x2, 0x2, 0x2, 0x12b, 0x5ec, + 0x3, 0x2, 0x2, 0x2, 0x12d, 0x5f3, 0x3, 0x2, 0x2, 0x2, 0x12f, 0x5fa, + 0x3, 0x2, 0x2, 0x2, 0x131, 0x600, 0x3, 0x2, 0x2, 0x2, 0x133, 0x607, + 0x3, 0x2, 0x2, 0x2, 0x135, 0x611, 0x3, 0x2, 0x2, 0x2, 0x137, 0x616, + 0x3, 0x2, 0x2, 0x2, 0x139, 0x61b, 0x3, 0x2, 0x2, 0x2, 0x13b, 0x620, + 0x3, 0x2, 0x2, 0x2, 0x13d, 0x628, 0x3, 0x2, 0x2, 0x2, 0x13f, 0x632, + 0x3, 0x2, 0x2, 0x2, 0x141, 0x635, 0x3, 0x2, 0x2, 0x2, 0x143, 0x639, + 0x3, 0x2, 0x2, 0x2, 0x145, 0x640, 0x3, 0x2, 0x2, 0x2, 0x147, 0x649, + 0x3, 0x2, 0x2, 0x2, 0x149, 0x64e, 0x3, 0x2, 0x2, 0x2, 0x14b, 0x657, + 0x3, 0x2, 0x2, 0x2, 0x14d, 0x65b, 0x3, 0x2, 0x2, 0x2, 0x14f, 0x660, + 0x3, 0x2, 0x2, 0x2, 0x151, 0x666, 0x3, 0x2, 0x2, 0x2, 0x153, 0x66d, + 0x3, 0x2, 0x2, 0x2, 0x155, 0x671, 0x3, 0x2, 0x2, 0x2, 0x157, 0x677, + 0x3, 0x2, 0x2, 0x2, 0x159, 0x67c, 0x3, 0x2, 0x2, 0x2, 0x15b, 0x683, + 0x3, 0x2, 0x2, 0x2, 0x15d, 0x688, 0x3, 0x2, 0x2, 0x2, 0x15f, 0x68f, + 0x3, 0x2, 0x2, 0x2, 0x161, 0x695, 0x3, 0x2, 0x2, 0x2, 0x163, 0x69a, + 0x3, 0x2, 0x2, 0x2, 0x165, 0x69f, 0x3, 0x2, 0x2, 0x2, 0x167, 0x6a5, + 0x3, 0x2, 0x2, 0x2, 0x169, 0x6b4, 0x3, 0x2, 0x2, 0x2, 0x16b, 0x6b6, + 0x3, 0x2, 0x2, 0x2, 0x16d, 0x6bc, 0x3, 0x2, 0x2, 0x2, 0x16f, 0x6eb, + 0x3, 0x2, 0x2, 0x2, 0x171, 0x739, 0x3, 0x2, 0x2, 0x2, 0x173, 0x73b, + 0x3, 0x2, 0x2, 0x2, 0x175, 0x742, 0x3, 0x2, 0x2, 0x2, 0x177, 0x746, + 0x3, 0x2, 0x2, 0x2, 0x179, 0x74d, 0x3, 0x2, 0x2, 0x2, 0x17b, 0x75c, + 0x3, 0x2, 0x2, 0x2, 0x17d, 0x75e, 0x3, 0x2, 0x2, 0x2, 0x17f, 0x760, + 0x3, 0x2, 0x2, 0x2, 0x181, 0x762, 0x3, 0x2, 0x2, 0x2, 0x183, 0x764, + 0x3, 0x2, 0x2, 0x2, 0x185, 0x766, 0x3, 0x2, 0x2, 0x2, 0x187, 0x768, + 0x3, 0x2, 0x2, 0x2, 0x189, 0x76a, 0x3, 0x2, 0x2, 0x2, 0x18b, 0x76c, + 0x3, 0x2, 0x2, 0x2, 0x18d, 0x76e, 0x3, 0x2, 0x2, 0x2, 0x18f, 0x770, + 0x3, 0x2, 0x2, 0x2, 0x191, 0x772, 0x3, 0x2, 0x2, 0x2, 0x193, 0x774, + 0x3, 0x2, 0x2, 0x2, 0x195, 0x776, 0x3, 0x2, 0x2, 0x2, 0x197, 0x778, + 0x3, 0x2, 0x2, 0x2, 0x199, 0x77a, 0x3, 0x2, 0x2, 0x2, 0x19b, 0x77c, + 0x3, 0x2, 0x2, 0x2, 0x19d, 0x77e, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x780, + 0x3, 0x2, 0x2, 0x2, 0x1a1, 0x782, 0x3, 0x2, 0x2, 0x2, 0x1a3, 0x784, + 0x3, 0x2, 0x2, 0x2, 0x1a5, 0x786, 0x3, 0x2, 0x2, 0x2, 0x1a7, 0x788, + 0x3, 0x2, 0x2, 0x2, 0x1a9, 0x78a, 0x3, 0x2, 0x2, 0x2, 0x1ab, 0x78c, + 0x3, 0x2, 0x2, 0x2, 0x1ad, 0x78e, 0x3, 0x2, 0x2, 0x2, 0x1af, 0x790, + 0x3, 0x2, 0x2, 0x2, 0x1b1, 0x792, 0x3, 0x2, 0x2, 0x2, 0x1b3, 0x794, + 0x3, 0x2, 0x2, 0x2, 0x1b5, 0x796, 0x3, 0x2, 0x2, 0x2, 0x1b7, 0x798, + 0x3, 0x2, 0x2, 0x2, 0x1b9, 0x79b, 0x3, 0x2, 0x2, 0x2, 0x1bb, 0x79d, + 0x3, 0x2, 0x2, 0x2, 0x1bd, 0x79f, 0x3, 0x2, 0x2, 0x2, 0x1bf, 0x7a1, + 0x3, 0x2, 0x2, 0x2, 0x1c1, 0x7a3, 0x3, 0x2, 0x2, 0x2, 0x1c3, 0x7a5, + 0x3, 0x2, 0x2, 0x2, 0x1c5, 0x7a8, 0x3, 0x2, 0x2, 0x2, 0x1c7, 0x7aa, + 0x3, 0x2, 0x2, 0x2, 0x1c9, 0x7ac, 0x3, 0x2, 0x2, 0x2, 0x1cb, 0x7af, + 0x3, 0x2, 0x2, 0x2, 0x1cd, 0x7b1, 0x3, 0x2, 0x2, 0x2, 0x1cf, 0x7b4, + 0x3, 0x2, 0x2, 0x2, 0x1d1, 0x7b6, 0x3, 0x2, 0x2, 0x2, 0x1d3, 0x7b8, + 0x3, 0x2, 0x2, 0x2, 0x1d5, 0x7ba, 0x3, 0x2, 0x2, 0x2, 0x1d7, 0x7bd, + 0x3, 0x2, 0x2, 0x2, 0x1d9, 0x7bf, 0x3, 0x2, 0x2, 0x2, 0x1db, 0x7c5, + 0x3, 0x2, 0x2, 0x2, 0x1dd, 0x7c7, 0x3, 0x2, 0x2, 0x2, 0x1df, 0x7c9, + 0x3, 0x2, 0x2, 0x2, 0x1e1, 0x7cb, 0x3, 0x2, 0x2, 0x2, 0x1e3, 0x7cd, + 0x3, 0x2, 0x2, 0x2, 0x1e5, 0x7cf, 0x3, 0x2, 0x2, 0x2, 0x1e7, 0x7d1, + 0x3, 0x2, 0x2, 0x2, 0x1e9, 0x7d3, 0x3, 0x2, 0x2, 0x2, 0x1eb, 0x7d5, + 0x3, 0x2, 0x2, 0x2, 0x1ed, 0x7d7, 0x3, 0x2, 0x2, 0x2, 0x1ef, 0x7d9, + 0x3, 0x2, 0x2, 0x2, 0x1f1, 0x7db, 0x3, 0x2, 0x2, 0x2, 0x1f3, 0x7dd, + 0x3, 0x2, 0x2, 0x2, 0x1f5, 0x7eb, 0x3, 0x2, 0x2, 0x2, 0x1f7, 0x7f9, + 0x3, 0x2, 0x2, 0x2, 0x1f9, 0x1fa, 0x5, 0x17b, 0xbe, 0x2, 0x1fa, 0x1fb, + 0x5, 0x181, 0xc1, 0x2, 0x1fb, 0x1fc, 0x5, 0x181, 0xc1, 0x2, 0x1fc, 0x4, + 0x3, 0x2, 0x2, 0x2, 0x1fd, 0x1fe, 0x5, 0x17b, 0xbe, 0x2, 0x1fe, 0x1ff, + 0x5, 0x185, 0xc3, 0x2, 0x1ff, 0x200, 0x5, 0x1a1, 0xd1, 0x2, 0x200, 0x201, + 0x5, 0x183, 0xc2, 0x2, 0x201, 0x202, 0x5, 0x19d, 0xcf, 0x2, 0x202, 0x6, + 0x3, 0x2, 0x2, 0x2, 0x203, 0x204, 0x5, 0x17b, 0xbe, 0x2, 0x204, 0x205, + 0x5, 0x191, 0xc9, 0x2, 0x205, 0x206, 0x5, 0x18b, 0xc6, 0x2, 0x206, 0x207, + 0x5, 0x17b, 0xbe, 0x2, 0x207, 0x208, 0x5, 0x19f, 0xd0, 0x2, 0x208, 0x8, + 0x3, 0x2, 0x2, 0x2, 0x209, 0x20a, 0x5, 0x17b, 0xbe, 0x2, 0x20a, 0x20b, + 0x5, 0x191, 0xc9, 0x2, 0x20b, 0x20c, 0x5, 0x191, 0xc9, 0x2, 0x20c, 0xa, + 0x3, 0x2, 0x2, 0x2, 0x20d, 0x20e, 0x5, 0x17b, 0xbe, 0x2, 0x20e, 0x20f, + 0x5, 0x191, 0xc9, 0x2, 0x20f, 0x210, 0x5, 0x1a1, 0xd1, 0x2, 0x210, 0x211, + 0x5, 0x183, 0xc2, 0x2, 0x211, 0x212, 0x5, 0x19d, 0xcf, 0x2, 0x212, 0xc, + 0x3, 0x2, 0x2, 0x2, 0x213, 0x214, 0x5, 0x17b, 0xbe, 0x2, 0x214, 0x215, + 0x5, 0x195, 0xcb, 0x2, 0x215, 0x216, 0x5, 0x181, 0xc1, 0x2, 0x216, 0xe, + 0x3, 0x2, 0x2, 0x2, 0x217, 0x218, 0x5, 0x17b, 0xbe, 0x2, 0x218, 0x219, + 0x5, 0x195, 0xcb, 0x2, 0x219, 0x21a, 0x5, 0x1a1, 0xd1, 0x2, 0x21a, 0x21b, + 0x5, 0x18b, 0xc6, 0x2, 0x21b, 0x10, 0x3, 0x2, 0x2, 0x2, 0x21c, 0x21d, + 0x5, 0x17b, 0xbe, 0x2, 0x21d, 0x21e, 0x5, 0x195, 0xcb, 0x2, 0x21e, 0x21f, + 0x5, 0x1ab, 0xd6, 0x2, 0x21f, 0x12, 0x3, 0x2, 0x2, 0x2, 0x220, 0x221, + 0x5, 0x17b, 0xbe, 0x2, 0x221, 0x222, 0x5, 0x19d, 0xcf, 0x2, 0x222, 0x223, + 0x5, 0x19d, 0xcf, 0x2, 0x223, 0x224, 0x5, 0x17b, 0xbe, 0x2, 0x224, 0x225, + 0x5, 0x1ab, 0xd6, 0x2, 0x225, 0x14, 0x3, 0x2, 0x2, 0x2, 0x226, 0x227, + 0x5, 0x17b, 0xbe, 0x2, 0x227, 0x228, 0x5, 0x19f, 0xd0, 0x2, 0x228, 0x16, + 0x3, 0x2, 0x2, 0x2, 0x229, 0x22a, 0x5, 0x17b, 0xbe, 0x2, 0x22a, 0x22b, + 0x5, 0x19f, 0xd0, 0x2, 0x22b, 0x22c, 0x5, 0x17f, 0xc0, 0x2, 0x22c, 0x238, + 0x3, 0x2, 0x2, 0x2, 0x22d, 0x22e, 0x5, 0x17b, 0xbe, 0x2, 0x22e, 0x22f, + 0x5, 0x19f, 0xd0, 0x2, 0x22f, 0x230, 0x5, 0x17f, 0xc0, 0x2, 0x230, 0x231, + 0x5, 0x183, 0xc2, 0x2, 0x231, 0x232, 0x5, 0x195, 0xcb, 0x2, 0x232, 0x233, + 0x5, 0x181, 0xc1, 0x2, 0x233, 0x234, 0x5, 0x18b, 0xc6, 0x2, 0x234, 0x235, + 0x5, 0x195, 0xcb, 0x2, 0x235, 0x236, 0x5, 0x187, 0xc4, 0x2, 0x236, 0x238, + 0x3, 0x2, 0x2, 0x2, 0x237, 0x229, 0x3, 0x2, 0x2, 0x2, 0x237, 0x22d, + 0x3, 0x2, 0x2, 0x2, 0x238, 0x18, 0x3, 0x2, 0x2, 0x2, 0x239, 0x23a, 0x5, + 0x17b, 0xbe, 0x2, 0x23a, 0x23b, 0x5, 0x19f, 0xd0, 0x2, 0x23b, 0x23c, + 0x5, 0x197, 0xcc, 0x2, 0x23c, 0x23d, 0x5, 0x185, 0xc3, 0x2, 0x23d, 0x1a, + 0x3, 0x2, 0x2, 0x2, 0x23e, 0x23f, 0x5, 0x17b, 0xbe, 0x2, 0x23f, 0x240, + 0x5, 0x19f, 0xd0, 0x2, 0x240, 0x241, 0x5, 0x1ab, 0xd6, 0x2, 0x241, 0x242, + 0x5, 0x195, 0xcb, 0x2, 0x242, 0x243, 0x5, 0x17f, 0xc0, 0x2, 0x243, 0x1c, + 0x3, 0x2, 0x2, 0x2, 0x244, 0x245, 0x5, 0x17b, 0xbe, 0x2, 0x245, 0x246, + 0x5, 0x1a1, 0xd1, 0x2, 0x246, 0x247, 0x5, 0x1a1, 0xd1, 0x2, 0x247, 0x248, + 0x5, 0x17b, 0xbe, 0x2, 0x248, 0x249, 0x5, 0x17f, 0xc0, 0x2, 0x249, 0x24a, + 0x5, 0x189, 0xc5, 0x2, 0x24a, 0x1e, 0x3, 0x2, 0x2, 0x2, 0x24b, 0x24c, + 0x5, 0x17d, 0xbf, 0x2, 0x24c, 0x24d, 0x5, 0x183, 0xc2, 0x2, 0x24d, 0x24e, + 0x5, 0x1a1, 0xd1, 0x2, 0x24e, 0x24f, 0x5, 0x1a7, 0xd4, 0x2, 0x24f, 0x250, + 0x5, 0x183, 0xc2, 0x2, 0x250, 0x251, 0x5, 0x183, 0xc2, 0x2, 0x251, 0x252, + 0x5, 0x195, 0xcb, 0x2, 0x252, 0x20, 0x3, 0x2, 0x2, 0x2, 0x253, 0x254, + 0x5, 0x17d, 0xbf, 0x2, 0x254, 0x255, 0x5, 0x197, 0xcc, 0x2, 0x255, 0x256, + 0x5, 0x1a1, 0xd1, 0x2, 0x256, 0x257, 0x5, 0x189, 0xc5, 0x2, 0x257, 0x22, + 0x3, 0x2, 0x2, 0x2, 0x258, 0x259, 0x5, 0x17d, 0xbf, 0x2, 0x259, 0x25a, + 0x5, 0x1ab, 0xd6, 0x2, 0x25a, 0x24, 0x3, 0x2, 0x2, 0x2, 0x25b, 0x25c, + 0x5, 0x17f, 0xc0, 0x2, 0x25c, 0x25d, 0x5, 0x17b, 0xbe, 0x2, 0x25d, 0x25e, + 0x5, 0x19f, 0xd0, 0x2, 0x25e, 0x25f, 0x5, 0x183, 0xc2, 0x2, 0x25f, 0x26, + 0x3, 0x2, 0x2, 0x2, 0x260, 0x261, 0x5, 0x17f, 0xc0, 0x2, 0x261, 0x262, + 0x5, 0x17b, 0xbe, 0x2, 0x262, 0x263, 0x5, 0x19f, 0xd0, 0x2, 0x263, 0x264, + 0x5, 0x1a1, 0xd1, 0x2, 0x264, 0x28, 0x3, 0x2, 0x2, 0x2, 0x265, 0x266, + 0x5, 0x17f, 0xc0, 0x2, 0x266, 0x267, 0x5, 0x189, 0xc5, 0x2, 0x267, 0x268, + 0x5, 0x183, 0xc2, 0x2, 0x268, 0x269, 0x5, 0x17f, 0xc0, 0x2, 0x269, 0x26a, + 0x5, 0x18f, 0xc8, 0x2, 0x26a, 0x2a, 0x3, 0x2, 0x2, 0x2, 0x26b, 0x26c, + 0x5, 0x17f, 0xc0, 0x2, 0x26c, 0x26d, 0x5, 0x191, 0xc9, 0x2, 0x26d, 0x26e, + 0x5, 0x183, 0xc2, 0x2, 0x26e, 0x26f, 0x5, 0x17b, 0xbe, 0x2, 0x26f, 0x270, + 0x5, 0x19d, 0xcf, 0x2, 0x270, 0x2c, 0x3, 0x2, 0x2, 0x2, 0x271, 0x272, + 0x5, 0x17f, 0xc0, 0x2, 0x272, 0x273, 0x5, 0x191, 0xc9, 0x2, 0x273, 0x274, + 0x5, 0x1a3, 0xd2, 0x2, 0x274, 0x275, 0x5, 0x19f, 0xd0, 0x2, 0x275, 0x276, + 0x5, 0x1a1, 0xd1, 0x2, 0x276, 0x277, 0x5, 0x183, 0xc2, 0x2, 0x277, 0x278, + 0x5, 0x19d, 0xcf, 0x2, 0x278, 0x2e, 0x3, 0x2, 0x2, 0x2, 0x279, 0x27a, + 0x5, 0x17f, 0xc0, 0x2, 0x27a, 0x27b, 0x5, 0x197, 0xcc, 0x2, 0x27b, 0x27c, + 0x5, 0x181, 0xc1, 0x2, 0x27c, 0x27d, 0x5, 0x183, 0xc2, 0x2, 0x27d, 0x27e, + 0x5, 0x17f, 0xc0, 0x2, 0x27e, 0x30, 0x3, 0x2, 0x2, 0x2, 0x27f, 0x280, + 0x5, 0x17f, 0xc0, 0x2, 0x280, 0x281, 0x5, 0x197, 0xcc, 0x2, 0x281, 0x282, + 0x5, 0x191, 0xc9, 0x2, 0x282, 0x283, 0x5, 0x191, 0xc9, 0x2, 0x283, 0x284, + 0x5, 0x17b, 0xbe, 0x2, 0x284, 0x285, 0x5, 0x1a1, 0xd1, 0x2, 0x285, 0x286, + 0x5, 0x183, 0xc2, 0x2, 0x286, 0x32, 0x3, 0x2, 0x2, 0x2, 0x287, 0x288, + 0x5, 0x17f, 0xc0, 0x2, 0x288, 0x289, 0x5, 0x197, 0xcc, 0x2, 0x289, 0x28a, + 0x5, 0x191, 0xc9, 0x2, 0x28a, 0x28b, 0x5, 0x1a3, 0xd2, 0x2, 0x28b, 0x28c, + 0x5, 0x193, 0xca, 0x2, 0x28c, 0x28d, 0x5, 0x195, 0xcb, 0x2, 0x28d, 0x34, + 0x3, 0x2, 0x2, 0x2, 0x28e, 0x28f, 0x5, 0x17f, 0xc0, 0x2, 0x28f, 0x290, + 0x5, 0x197, 0xcc, 0x2, 0x290, 0x291, 0x5, 0x193, 0xca, 0x2, 0x291, 0x292, + 0x5, 0x193, 0xca, 0x2, 0x292, 0x293, 0x5, 0x183, 0xc2, 0x2, 0x293, 0x294, + 0x5, 0x195, 0xcb, 0x2, 0x294, 0x295, 0x5, 0x1a1, 0xd1, 0x2, 0x295, 0x36, + 0x3, 0x2, 0x2, 0x2, 0x296, 0x297, 0x5, 0x17f, 0xc0, 0x2, 0x297, 0x298, + 0x5, 0x197, 0xcc, 0x2, 0x298, 0x299, 0x5, 0x195, 0xcb, 0x2, 0x299, 0x29a, + 0x5, 0x19f, 0xd0, 0x2, 0x29a, 0x29b, 0x5, 0x1a1, 0xd1, 0x2, 0x29b, 0x29c, + 0x5, 0x19d, 0xcf, 0x2, 0x29c, 0x29d, 0x5, 0x17b, 0xbe, 0x2, 0x29d, 0x29e, + 0x5, 0x18b, 0xc6, 0x2, 0x29e, 0x29f, 0x5, 0x195, 0xcb, 0x2, 0x29f, 0x2a0, + 0x5, 0x1a1, 0xd1, 0x2, 0x2a0, 0x38, 0x3, 0x2, 0x2, 0x2, 0x2a1, 0x2a2, + 0x5, 0x17f, 0xc0, 0x2, 0x2a2, 0x2a3, 0x5, 0x19d, 0xcf, 0x2, 0x2a3, 0x2a4, + 0x5, 0x183, 0xc2, 0x2, 0x2a4, 0x2a5, 0x5, 0x17b, 0xbe, 0x2, 0x2a5, 0x2a6, + 0x5, 0x1a1, 0xd1, 0x2, 0x2a6, 0x2a7, 0x5, 0x183, 0xc2, 0x2, 0x2a7, 0x3a, + 0x3, 0x2, 0x2, 0x2, 0x2a8, 0x2a9, 0x5, 0x17f, 0xc0, 0x2, 0x2a9, 0x2aa, + 0x5, 0x19d, 0xcf, 0x2, 0x2aa, 0x2ab, 0x5, 0x197, 0xcc, 0x2, 0x2ab, 0x2ac, + 0x5, 0x19f, 0xd0, 0x2, 0x2ac, 0x2ad, 0x5, 0x19f, 0xd0, 0x2, 0x2ad, 0x3c, + 0x3, 0x2, 0x2, 0x2, 0x2ae, 0x2af, 0x5, 0x17f, 0xc0, 0x2, 0x2af, 0x2b0, + 0x5, 0x1a3, 0xd2, 0x2, 0x2b0, 0x2b1, 0x5, 0x17d, 0xbf, 0x2, 0x2b1, 0x2b2, + 0x5, 0x183, 0xc2, 0x2, 0x2b2, 0x3e, 0x3, 0x2, 0x2, 0x2, 0x2b3, 0x2b4, + 0x5, 0x181, 0xc1, 0x2, 0x2b4, 0x2b5, 0x5, 0x17b, 0xbe, 0x2, 0x2b5, 0x2b6, + 0x5, 0x1a1, 0xd1, 0x2, 0x2b6, 0x2b7, 0x5, 0x17b, 0xbe, 0x2, 0x2b7, 0x2b8, + 0x5, 0x17d, 0xbf, 0x2, 0x2b8, 0x2b9, 0x5, 0x17b, 0xbe, 0x2, 0x2b9, 0x2ba, + 0x5, 0x19f, 0xd0, 0x2, 0x2ba, 0x2bb, 0x5, 0x183, 0xc2, 0x2, 0x2bb, 0x40, + 0x3, 0x2, 0x2, 0x2, 0x2bc, 0x2bd, 0x5, 0x181, 0xc1, 0x2, 0x2bd, 0x2be, + 0x5, 0x17b, 0xbe, 0x2, 0x2be, 0x2bf, 0x5, 0x1a1, 0xd1, 0x2, 0x2bf, 0x2c0, + 0x5, 0x17b, 0xbe, 0x2, 0x2c0, 0x2c1, 0x5, 0x17d, 0xbf, 0x2, 0x2c1, 0x2c2, + 0x5, 0x17b, 0xbe, 0x2, 0x2c2, 0x2c3, 0x5, 0x19f, 0xd0, 0x2, 0x2c3, 0x2c4, + 0x5, 0x183, 0xc2, 0x2, 0x2c4, 0x2c5, 0x5, 0x19f, 0xd0, 0x2, 0x2c5, 0x42, + 0x3, 0x2, 0x2, 0x2, 0x2c6, 0x2c7, 0x5, 0x181, 0xc1, 0x2, 0x2c7, 0x2c8, + 0x5, 0x17b, 0xbe, 0x2, 0x2c8, 0x2c9, 0x5, 0x1a1, 0xd1, 0x2, 0x2c9, 0x2ca, + 0x5, 0x183, 0xc2, 0x2, 0x2ca, 0x44, 0x3, 0x2, 0x2, 0x2, 0x2cb, 0x2cc, + 0x5, 0x181, 0xc1, 0x2, 0x2cc, 0x2cd, 0x5, 0x17b, 0xbe, 0x2, 0x2cd, 0x2ce, + 0x5, 0x1ab, 0xd6, 0x2, 0x2ce, 0x46, 0x3, 0x2, 0x2, 0x2, 0x2cf, 0x2d0, + 0x5, 0x181, 0xc1, 0x2, 0x2d0, 0x2d1, 0x5, 0x183, 0xc2, 0x2, 0x2d1, 0x2d2, + 0x5, 0x181, 0xc1, 0x2, 0x2d2, 0x2d3, 0x5, 0x1a3, 0xd2, 0x2, 0x2d3, 0x2d4, + 0x5, 0x199, 0xcd, 0x2, 0x2d4, 0x2d5, 0x5, 0x191, 0xc9, 0x2, 0x2d5, 0x2d6, + 0x5, 0x18b, 0xc6, 0x2, 0x2d6, 0x2d7, 0x5, 0x17f, 0xc0, 0x2, 0x2d7, 0x2d8, + 0x5, 0x17b, 0xbe, 0x2, 0x2d8, 0x2d9, 0x5, 0x1a1, 0xd1, 0x2, 0x2d9, 0x2da, + 0x5, 0x183, 0xc2, 0x2, 0x2da, 0x48, 0x3, 0x2, 0x2, 0x2, 0x2db, 0x2dc, + 0x5, 0x181, 0xc1, 0x2, 0x2dc, 0x2dd, 0x5, 0x183, 0xc2, 0x2, 0x2dd, 0x2de, + 0x5, 0x185, 0xc3, 0x2, 0x2de, 0x2df, 0x5, 0x17b, 0xbe, 0x2, 0x2df, 0x2e0, + 0x5, 0x1a3, 0xd2, 0x2, 0x2e0, 0x2e1, 0x5, 0x191, 0xc9, 0x2, 0x2e1, 0x2e2, + 0x5, 0x1a1, 0xd1, 0x2, 0x2e2, 0x4a, 0x3, 0x2, 0x2, 0x2, 0x2e3, 0x2e4, + 0x5, 0x181, 0xc1, 0x2, 0x2e4, 0x2e5, 0x5, 0x183, 0xc2, 0x2, 0x2e5, 0x2e6, + 0x5, 0x191, 0xc9, 0x2, 0x2e6, 0x2e7, 0x5, 0x17b, 0xbe, 0x2, 0x2e7, 0x2e8, + 0x5, 0x1ab, 0xd6, 0x2, 0x2e8, 0x4c, 0x3, 0x2, 0x2, 0x2, 0x2e9, 0x2ea, + 0x5, 0x181, 0xc1, 0x2, 0x2ea, 0x2eb, 0x5, 0x183, 0xc2, 0x2, 0x2eb, 0x2ec, + 0x5, 0x191, 0xc9, 0x2, 0x2ec, 0x2ed, 0x5, 0x183, 0xc2, 0x2, 0x2ed, 0x2ee, + 0x5, 0x1a1, 0xd1, 0x2, 0x2ee, 0x2ef, 0x5, 0x183, 0xc2, 0x2, 0x2ef, 0x4e, + 0x3, 0x2, 0x2, 0x2, 0x2f0, 0x2f1, 0x5, 0x181, 0xc1, 0x2, 0x2f1, 0x2f2, + 0x5, 0x183, 0xc2, 0x2, 0x2f2, 0x2f3, 0x5, 0x19f, 0xd0, 0x2, 0x2f3, 0x2f4, + 0x5, 0x17f, 0xc0, 0x2, 0x2f4, 0x50, 0x3, 0x2, 0x2, 0x2, 0x2f5, 0x2f6, + 0x5, 0x181, 0xc1, 0x2, 0x2f6, 0x2f7, 0x5, 0x183, 0xc2, 0x2, 0x2f7, 0x2f8, + 0x5, 0x19f, 0xd0, 0x2, 0x2f8, 0x2f9, 0x5, 0x17f, 0xc0, 0x2, 0x2f9, 0x2fa, + 0x5, 0x183, 0xc2, 0x2, 0x2fa, 0x2fb, 0x5, 0x195, 0xcb, 0x2, 0x2fb, 0x2fc, + 0x5, 0x181, 0xc1, 0x2, 0x2fc, 0x2fd, 0x5, 0x18b, 0xc6, 0x2, 0x2fd, 0x2fe, + 0x5, 0x195, 0xcb, 0x2, 0x2fe, 0x2ff, 0x5, 0x187, 0xc4, 0x2, 0x2ff, 0x52, + 0x3, 0x2, 0x2, 0x2, 0x300, 0x301, 0x5, 0x181, 0xc1, 0x2, 0x301, 0x302, + 0x5, 0x183, 0xc2, 0x2, 0x302, 0x303, 0x5, 0x19f, 0xd0, 0x2, 0x303, 0x304, + 0x5, 0x17f, 0xc0, 0x2, 0x304, 0x305, 0x5, 0x19d, 0xcf, 0x2, 0x305, 0x306, + 0x5, 0x18b, 0xc6, 0x2, 0x306, 0x307, 0x5, 0x17d, 0xbf, 0x2, 0x307, 0x308, + 0x5, 0x183, 0xc2, 0x2, 0x308, 0x54, 0x3, 0x2, 0x2, 0x2, 0x309, 0x30a, + 0x5, 0x181, 0xc1, 0x2, 0x30a, 0x30b, 0x5, 0x183, 0xc2, 0x2, 0x30b, 0x30c, + 0x5, 0x1a1, 0xd1, 0x2, 0x30c, 0x30d, 0x5, 0x17b, 0xbe, 0x2, 0x30d, 0x30e, + 0x5, 0x17f, 0xc0, 0x2, 0x30e, 0x30f, 0x5, 0x189, 0xc5, 0x2, 0x30f, 0x56, + 0x3, 0x2, 0x2, 0x2, 0x310, 0x311, 0x5, 0x181, 0xc1, 0x2, 0x311, 0x312, + 0x5, 0x18b, 0xc6, 0x2, 0x312, 0x313, 0x5, 0x17f, 0xc0, 0x2, 0x313, 0x314, + 0x5, 0x1a1, 0xd1, 0x2, 0x314, 0x315, 0x5, 0x18b, 0xc6, 0x2, 0x315, 0x316, + 0x5, 0x197, 0xcc, 0x2, 0x316, 0x317, 0x5, 0x195, 0xcb, 0x2, 0x317, 0x318, + 0x5, 0x17b, 0xbe, 0x2, 0x318, 0x319, 0x5, 0x19d, 0xcf, 0x2, 0x319, 0x31a, + 0x5, 0x18b, 0xc6, 0x2, 0x31a, 0x31b, 0x5, 0x183, 0xc2, 0x2, 0x31b, 0x31c, + 0x5, 0x19f, 0xd0, 0x2, 0x31c, 0x58, 0x3, 0x2, 0x2, 0x2, 0x31d, 0x31e, + 0x5, 0x181, 0xc1, 0x2, 0x31e, 0x31f, 0x5, 0x18b, 0xc6, 0x2, 0x31f, 0x320, + 0x5, 0x17f, 0xc0, 0x2, 0x320, 0x321, 0x5, 0x1a1, 0xd1, 0x2, 0x321, 0x322, + 0x5, 0x18b, 0xc6, 0x2, 0x322, 0x323, 0x5, 0x197, 0xcc, 0x2, 0x323, 0x324, + 0x5, 0x195, 0xcb, 0x2, 0x324, 0x325, 0x5, 0x17b, 0xbe, 0x2, 0x325, 0x326, + 0x5, 0x19d, 0xcf, 0x2, 0x326, 0x327, 0x5, 0x1ab, 0xd6, 0x2, 0x327, 0x5a, + 0x3, 0x2, 0x2, 0x2, 0x328, 0x329, 0x5, 0x181, 0xc1, 0x2, 0x329, 0x32a, + 0x5, 0x18b, 0xc6, 0x2, 0x32a, 0x32b, 0x5, 0x19f, 0xd0, 0x2, 0x32b, 0x32c, + 0x5, 0x18f, 0xc8, 0x2, 0x32c, 0x5c, 0x3, 0x2, 0x2, 0x2, 0x32d, 0x32e, + 0x5, 0x181, 0xc1, 0x2, 0x32e, 0x32f, 0x5, 0x18b, 0xc6, 0x2, 0x32f, 0x330, + 0x5, 0x19f, 0xd0, 0x2, 0x330, 0x331, 0x5, 0x1a1, 0xd1, 0x2, 0x331, 0x332, + 0x5, 0x18b, 0xc6, 0x2, 0x332, 0x333, 0x5, 0x195, 0xcb, 0x2, 0x333, 0x334, + 0x5, 0x17f, 0xc0, 0x2, 0x334, 0x335, 0x5, 0x1a1, 0xd1, 0x2, 0x335, 0x5e, + 0x3, 0x2, 0x2, 0x2, 0x336, 0x337, 0x5, 0x181, 0xc1, 0x2, 0x337, 0x338, + 0x5, 0x18b, 0xc6, 0x2, 0x338, 0x339, 0x5, 0x19f, 0xd0, 0x2, 0x339, 0x33a, + 0x5, 0x1a1, 0xd1, 0x2, 0x33a, 0x33b, 0x5, 0x19d, 0xcf, 0x2, 0x33b, 0x33c, + 0x5, 0x18b, 0xc6, 0x2, 0x33c, 0x33d, 0x5, 0x17d, 0xbf, 0x2, 0x33d, 0x33e, + 0x5, 0x1a3, 0xd2, 0x2, 0x33e, 0x33f, 0x5, 0x1a1, 0xd1, 0x2, 0x33f, 0x340, + 0x5, 0x183, 0xc2, 0x2, 0x340, 0x341, 0x5, 0x181, 0xc1, 0x2, 0x341, 0x60, + 0x3, 0x2, 0x2, 0x2, 0x342, 0x343, 0x5, 0x181, 0xc1, 0x2, 0x343, 0x344, + 0x5, 0x19d, 0xcf, 0x2, 0x344, 0x345, 0x5, 0x197, 0xcc, 0x2, 0x345, 0x346, + 0x5, 0x199, 0xcd, 0x2, 0x346, 0x62, 0x3, 0x2, 0x2, 0x2, 0x347, 0x348, + 0x5, 0x183, 0xc2, 0x2, 0x348, 0x349, 0x5, 0x191, 0xc9, 0x2, 0x349, 0x34a, + 0x5, 0x19f, 0xd0, 0x2, 0x34a, 0x34b, 0x5, 0x183, 0xc2, 0x2, 0x34b, 0x64, + 0x3, 0x2, 0x2, 0x2, 0x34c, 0x34d, 0x5, 0x183, 0xc2, 0x2, 0x34d, 0x34e, + 0x5, 0x195, 0xcb, 0x2, 0x34e, 0x34f, 0x5, 0x181, 0xc1, 0x2, 0x34f, 0x66, + 0x3, 0x2, 0x2, 0x2, 0x350, 0x351, 0x5, 0x183, 0xc2, 0x2, 0x351, 0x352, + 0x5, 0x195, 0xcb, 0x2, 0x352, 0x353, 0x5, 0x187, 0xc4, 0x2, 0x353, 0x354, + 0x5, 0x18b, 0xc6, 0x2, 0x354, 0x355, 0x5, 0x195, 0xcb, 0x2, 0x355, 0x356, + 0x5, 0x183, 0xc2, 0x2, 0x356, 0x68, 0x3, 0x2, 0x2, 0x2, 0x357, 0x358, + 0x5, 0x183, 0xc2, 0x2, 0x358, 0x359, 0x5, 0x1a5, 0xd3, 0x2, 0x359, 0x35a, + 0x5, 0x183, 0xc2, 0x2, 0x35a, 0x35b, 0x5, 0x195, 0xcb, 0x2, 0x35b, 0x35c, + 0x5, 0x1a1, 0xd1, 0x2, 0x35c, 0x35d, 0x5, 0x19f, 0xd0, 0x2, 0x35d, 0x6a, + 0x3, 0x2, 0x2, 0x2, 0x35e, 0x35f, 0x5, 0x183, 0xc2, 0x2, 0x35f, 0x360, + 0x5, 0x1a9, 0xd5, 0x2, 0x360, 0x361, 0x5, 0x18b, 0xc6, 0x2, 0x361, 0x362, + 0x5, 0x19f, 0xd0, 0x2, 0x362, 0x363, 0x5, 0x1a1, 0xd1, 0x2, 0x363, 0x364, + 0x5, 0x19f, 0xd0, 0x2, 0x364, 0x6c, 0x3, 0x2, 0x2, 0x2, 0x365, 0x366, + 0x5, 0x183, 0xc2, 0x2, 0x366, 0x367, 0x5, 0x1a9, 0xd5, 0x2, 0x367, 0x368, + 0x5, 0x199, 0xcd, 0x2, 0x368, 0x369, 0x5, 0x191, 0xc9, 0x2, 0x369, 0x36a, + 0x5, 0x17b, 0xbe, 0x2, 0x36a, 0x36b, 0x5, 0x18b, 0xc6, 0x2, 0x36b, 0x36c, + 0x5, 0x195, 0xcb, 0x2, 0x36c, 0x6e, 0x3, 0x2, 0x2, 0x2, 0x36d, 0x36e, + 0x5, 0x183, 0xc2, 0x2, 0x36e, 0x36f, 0x5, 0x1a9, 0xd5, 0x2, 0x36f, 0x370, + 0x5, 0x199, 0xcd, 0x2, 0x370, 0x371, 0x5, 0x19d, 0xcf, 0x2, 0x371, 0x372, + 0x5, 0x183, 0xc2, 0x2, 0x372, 0x373, 0x5, 0x19f, 0xd0, 0x2, 0x373, 0x374, + 0x5, 0x19f, 0xd0, 0x2, 0x374, 0x375, 0x5, 0x18b, 0xc6, 0x2, 0x375, 0x376, + 0x5, 0x197, 0xcc, 0x2, 0x376, 0x377, 0x5, 0x195, 0xcb, 0x2, 0x377, 0x70, + 0x3, 0x2, 0x2, 0x2, 0x378, 0x379, 0x5, 0x183, 0xc2, 0x2, 0x379, 0x37a, + 0x5, 0x1a9, 0xd5, 0x2, 0x37a, 0x37b, 0x5, 0x1a1, 0xd1, 0x2, 0x37b, 0x37c, + 0x5, 0x19d, 0xcf, 0x2, 0x37c, 0x37d, 0x5, 0x17b, 0xbe, 0x2, 0x37d, 0x37e, + 0x5, 0x17f, 0xc0, 0x2, 0x37e, 0x37f, 0x5, 0x1a1, 0xd1, 0x2, 0x37f, 0x72, + 0x3, 0x2, 0x2, 0x2, 0x380, 0x381, 0x5, 0x185, 0xc3, 0x2, 0x381, 0x382, + 0x5, 0x183, 0xc2, 0x2, 0x382, 0x383, 0x5, 0x1a1, 0xd1, 0x2, 0x383, 0x384, + 0x5, 0x17f, 0xc0, 0x2, 0x384, 0x385, 0x5, 0x189, 0xc5, 0x2, 0x385, 0x386, + 0x5, 0x183, 0xc2, 0x2, 0x386, 0x387, 0x5, 0x19f, 0xd0, 0x2, 0x387, 0x74, + 0x3, 0x2, 0x2, 0x2, 0x388, 0x389, 0x5, 0x185, 0xc3, 0x2, 0x389, 0x38a, + 0x5, 0x18b, 0xc6, 0x2, 0x38a, 0x38b, 0x5, 0x195, 0xcb, 0x2, 0x38b, 0x38c, + 0x5, 0x17b, 0xbe, 0x2, 0x38c, 0x38d, 0x5, 0x191, 0xc9, 0x2, 0x38d, 0x76, + 0x3, 0x2, 0x2, 0x2, 0x38e, 0x38f, 0x5, 0x185, 0xc3, 0x2, 0x38f, 0x390, + 0x5, 0x18b, 0xc6, 0x2, 0x390, 0x391, 0x5, 0x19d, 0xcf, 0x2, 0x391, 0x392, + 0x5, 0x19f, 0xd0, 0x2, 0x392, 0x393, 0x5, 0x1a1, 0xd1, 0x2, 0x393, 0x78, + 0x3, 0x2, 0x2, 0x2, 0x394, 0x395, 0x5, 0x185, 0xc3, 0x2, 0x395, 0x396, + 0x5, 0x191, 0xc9, 0x2, 0x396, 0x397, 0x5, 0x1a3, 0xd2, 0x2, 0x397, 0x398, + 0x5, 0x19f, 0xd0, 0x2, 0x398, 0x399, 0x5, 0x189, 0xc5, 0x2, 0x399, 0x7a, + 0x3, 0x2, 0x2, 0x2, 0x39a, 0x39b, 0x5, 0x185, 0xc3, 0x2, 0x39b, 0x39c, + 0x5, 0x197, 0xcc, 0x2, 0x39c, 0x39d, 0x5, 0x19d, 0xcf, 0x2, 0x39d, 0x7c, + 0x3, 0x2, 0x2, 0x2, 0x39e, 0x39f, 0x5, 0x185, 0xc3, 0x2, 0x39f, 0x3a0, + 0x5, 0x197, 0xcc, 0x2, 0x3a0, 0x3a1, 0x5, 0x19d, 0xcf, 0x2, 0x3a1, 0x3a2, + 0x5, 0x193, 0xca, 0x2, 0x3a2, 0x3a3, 0x5, 0x17b, 0xbe, 0x2, 0x3a3, 0x3a4, + 0x5, 0x1a1, 0xd1, 0x2, 0x3a4, 0x7e, 0x3, 0x2, 0x2, 0x2, 0x3a5, 0x3a6, + 0x5, 0x185, 0xc3, 0x2, 0x3a6, 0x3a7, 0x5, 0x19d, 0xcf, 0x2, 0x3a7, 0x3a8, + 0x5, 0x183, 0xc2, 0x2, 0x3a8, 0x3a9, 0x5, 0x183, 0xc2, 0x2, 0x3a9, 0x3aa, + 0x5, 0x1ad, 0xd7, 0x2, 0x3aa, 0x3ab, 0x5, 0x183, 0xc2, 0x2, 0x3ab, 0x80, + 0x3, 0x2, 0x2, 0x2, 0x3ac, 0x3ad, 0x5, 0x185, 0xc3, 0x2, 0x3ad, 0x3ae, + 0x5, 0x19d, 0xcf, 0x2, 0x3ae, 0x3af, 0x5, 0x197, 0xcc, 0x2, 0x3af, 0x3b0, + 0x5, 0x193, 0xca, 0x2, 0x3b0, 0x82, 0x3, 0x2, 0x2, 0x2, 0x3b1, 0x3b2, + 0x5, 0x185, 0xc3, 0x2, 0x3b2, 0x3b3, 0x5, 0x1a3, 0xd2, 0x2, 0x3b3, 0x3b4, + 0x5, 0x191, 0xc9, 0x2, 0x3b4, 0x3b5, 0x5, 0x191, 0xc9, 0x2, 0x3b5, 0x84, + 0x3, 0x2, 0x2, 0x2, 0x3b6, 0x3b7, 0x5, 0x185, 0xc3, 0x2, 0x3b7, 0x3b8, + 0x5, 0x1a3, 0xd2, 0x2, 0x3b8, 0x3b9, 0x5, 0x195, 0xcb, 0x2, 0x3b9, 0x3ba, + 0x5, 0x17f, 0xc0, 0x2, 0x3ba, 0x3bb, 0x5, 0x1a1, 0xd1, 0x2, 0x3bb, 0x3bc, + 0x5, 0x18b, 0xc6, 0x2, 0x3bc, 0x3bd, 0x5, 0x197, 0xcc, 0x2, 0x3bd, 0x3be, + 0x5, 0x195, 0xcb, 0x2, 0x3be, 0x86, 0x3, 0x2, 0x2, 0x2, 0x3bf, 0x3c0, + 0x5, 0x187, 0xc4, 0x2, 0x3c0, 0x3c1, 0x5, 0x191, 0xc9, 0x2, 0x3c1, 0x3c2, + 0x5, 0x197, 0xcc, 0x2, 0x3c2, 0x3c3, 0x5, 0x17d, 0xbf, 0x2, 0x3c3, 0x3c4, + 0x5, 0x17b, 0xbe, 0x2, 0x3c4, 0x3c5, 0x5, 0x191, 0xc9, 0x2, 0x3c5, 0x88, + 0x3, 0x2, 0x2, 0x2, 0x3c6, 0x3c7, 0x5, 0x187, 0xc4, 0x2, 0x3c7, 0x3c8, + 0x5, 0x19d, 0xcf, 0x2, 0x3c8, 0x3c9, 0x5, 0x17b, 0xbe, 0x2, 0x3c9, 0x3ca, + 0x5, 0x195, 0xcb, 0x2, 0x3ca, 0x3cb, 0x5, 0x1a3, 0xd2, 0x2, 0x3cb, 0x3cc, + 0x5, 0x191, 0xc9, 0x2, 0x3cc, 0x3cd, 0x5, 0x17b, 0xbe, 0x2, 0x3cd, 0x3ce, + 0x5, 0x19d, 0xcf, 0x2, 0x3ce, 0x3cf, 0x5, 0x18b, 0xc6, 0x2, 0x3cf, 0x3d0, + 0x5, 0x1a1, 0xd1, 0x2, 0x3d0, 0x3d1, 0x5, 0x1ab, 0xd6, 0x2, 0x3d1, 0x8a, + 0x3, 0x2, 0x2, 0x2, 0x3d2, 0x3d3, 0x5, 0x187, 0xc4, 0x2, 0x3d3, 0x3d4, + 0x5, 0x19d, 0xcf, 0x2, 0x3d4, 0x3d5, 0x5, 0x197, 0xcc, 0x2, 0x3d5, 0x3d6, + 0x5, 0x1a3, 0xd2, 0x2, 0x3d6, 0x3d7, 0x5, 0x199, 0xcd, 0x2, 0x3d7, 0x8c, + 0x3, 0x2, 0x2, 0x2, 0x3d8, 0x3d9, 0x5, 0x189, 0xc5, 0x2, 0x3d9, 0x3da, + 0x5, 0x17b, 0xbe, 0x2, 0x3da, 0x3db, 0x5, 0x1a5, 0xd3, 0x2, 0x3db, 0x3dc, + 0x5, 0x18b, 0xc6, 0x2, 0x3dc, 0x3dd, 0x5, 0x195, 0xcb, 0x2, 0x3dd, 0x3de, + 0x5, 0x187, 0xc4, 0x2, 0x3de, 0x8e, 0x3, 0x2, 0x2, 0x2, 0x3df, 0x3e0, + 0x5, 0x189, 0xc5, 0x2, 0x3e0, 0x3e1, 0x5, 0x18b, 0xc6, 0x2, 0x3e1, 0x3e2, + 0x5, 0x183, 0xc2, 0x2, 0x3e2, 0x3e3, 0x5, 0x19d, 0xcf, 0x2, 0x3e3, 0x3e4, + 0x5, 0x17b, 0xbe, 0x2, 0x3e4, 0x3e5, 0x5, 0x19d, 0xcf, 0x2, 0x3e5, 0x3e6, + 0x5, 0x17f, 0xc0, 0x2, 0x3e6, 0x3e7, 0x5, 0x189, 0xc5, 0x2, 0x3e7, 0x3e8, + 0x5, 0x18b, 0xc6, 0x2, 0x3e8, 0x3e9, 0x5, 0x17f, 0xc0, 0x2, 0x3e9, 0x3ea, + 0x5, 0x17b, 0xbe, 0x2, 0x3ea, 0x3eb, 0x5, 0x191, 0xc9, 0x2, 0x3eb, 0x90, + 0x3, 0x2, 0x2, 0x2, 0x3ec, 0x3ed, 0x5, 0x189, 0xc5, 0x2, 0x3ed, 0x3ee, + 0x5, 0x197, 0xcc, 0x2, 0x3ee, 0x3ef, 0x5, 0x1a3, 0xd2, 0x2, 0x3ef, 0x3f0, + 0x5, 0x19d, 0xcf, 0x2, 0x3f0, 0x92, 0x3, 0x2, 0x2, 0x2, 0x3f1, 0x3f2, + 0x5, 0x18b, 0xc6, 0x2, 0x3f2, 0x3f3, 0x5, 0x181, 0xc1, 0x2, 0x3f3, 0x94, + 0x3, 0x2, 0x2, 0x2, 0x3f4, 0x3f5, 0x5, 0x18b, 0xc6, 0x2, 0x3f5, 0x3f6, + 0x5, 0x185, 0xc3, 0x2, 0x3f6, 0x96, 0x3, 0x2, 0x2, 0x2, 0x3f7, 0x3f8, + 0x5, 0x18b, 0xc6, 0x2, 0x3f8, 0x3f9, 0x5, 0x191, 0xc9, 0x2, 0x3f9, 0x3fa, + 0x5, 0x18b, 0xc6, 0x2, 0x3fa, 0x3fb, 0x5, 0x18f, 0xc8, 0x2, 0x3fb, 0x3fc, + 0x5, 0x183, 0xc2, 0x2, 0x3fc, 0x98, 0x3, 0x2, 0x2, 0x2, 0x3fd, 0x3fe, + 0x5, 0x18b, 0xc6, 0x2, 0x3fe, 0x3ff, 0x5, 0x195, 0xcb, 0x2, 0x3ff, 0x9a, + 0x3, 0x2, 0x2, 0x2, 0x400, 0x401, 0x5, 0x18b, 0xc6, 0x2, 0x401, 0x402, + 0x5, 0x195, 0xcb, 0x2, 0x402, 0x403, 0x5, 0x181, 0xc1, 0x2, 0x403, 0x404, + 0x5, 0x183, 0xc2, 0x2, 0x404, 0x405, 0x5, 0x1a9, 0xd5, 0x2, 0x405, 0x9c, + 0x3, 0x2, 0x2, 0x2, 0x406, 0x407, 0x5, 0x18b, 0xc6, 0x2, 0x407, 0x408, + 0x5, 0x195, 0xcb, 0x2, 0x408, 0x409, 0x5, 0x185, 0xc3, 0x2, 0x409, 0x414, + 0x3, 0x2, 0x2, 0x2, 0x40a, 0x40b, 0x5, 0x18b, 0xc6, 0x2, 0x40b, 0x40c, + 0x5, 0x195, 0xcb, 0x2, 0x40c, 0x40d, 0x5, 0x185, 0xc3, 0x2, 0x40d, 0x40e, + 0x5, 0x18b, 0xc6, 0x2, 0x40e, 0x40f, 0x5, 0x195, 0xcb, 0x2, 0x40f, 0x410, + 0x5, 0x18b, 0xc6, 0x2, 0x410, 0x411, 0x5, 0x1a1, 0xd1, 0x2, 0x411, 0x412, + 0x5, 0x1ab, 0xd6, 0x2, 0x412, 0x414, 0x3, 0x2, 0x2, 0x2, 0x413, 0x406, + 0x3, 0x2, 0x2, 0x2, 0x413, 0x40a, 0x3, 0x2, 0x2, 0x2, 0x414, 0x9e, 0x3, + 0x2, 0x2, 0x2, 0x415, 0x416, 0x5, 0x18b, 0xc6, 0x2, 0x416, 0x417, 0x5, + 0x195, 0xcb, 0x2, 0x417, 0x418, 0x5, 0x18d, 0xc7, 0x2, 0x418, 0x419, + 0x5, 0x183, 0xc2, 0x2, 0x419, 0x41a, 0x5, 0x17f, 0xc0, 0x2, 0x41a, 0x41b, + 0x5, 0x1a1, 0xd1, 0x2, 0x41b, 0x41c, 0x5, 0x18b, 0xc6, 0x2, 0x41c, 0x41d, + 0x5, 0x1a5, 0xd3, 0x2, 0x41d, 0x41e, 0x5, 0x183, 0xc2, 0x2, 0x41e, 0xa0, + 0x3, 0x2, 0x2, 0x2, 0x41f, 0x420, 0x5, 0x18b, 0xc6, 0x2, 0x420, 0x421, + 0x5, 0x195, 0xcb, 0x2, 0x421, 0x422, 0x5, 0x195, 0xcb, 0x2, 0x422, 0x423, + 0x5, 0x183, 0xc2, 0x2, 0x423, 0x424, 0x5, 0x19d, 0xcf, 0x2, 0x424, 0xa2, + 0x3, 0x2, 0x2, 0x2, 0x425, 0x426, 0x5, 0x18b, 0xc6, 0x2, 0x426, 0x427, + 0x5, 0x195, 0xcb, 0x2, 0x427, 0x428, 0x5, 0x19f, 0xd0, 0x2, 0x428, 0x429, + 0x5, 0x183, 0xc2, 0x2, 0x429, 0x42a, 0x5, 0x19d, 0xcf, 0x2, 0x42a, 0x42b, + 0x5, 0x1a1, 0xd1, 0x2, 0x42b, 0xa4, 0x3, 0x2, 0x2, 0x2, 0x42c, 0x42d, + 0x5, 0x18b, 0xc6, 0x2, 0x42d, 0x42e, 0x5, 0x195, 0xcb, 0x2, 0x42e, 0x42f, + 0x5, 0x1a1, 0xd1, 0x2, 0x42f, 0x430, 0x5, 0x183, 0xc2, 0x2, 0x430, 0x431, + 0x5, 0x19d, 0xcf, 0x2, 0x431, 0x432, 0x5, 0x1a5, 0xd3, 0x2, 0x432, 0x433, + 0x5, 0x17b, 0xbe, 0x2, 0x433, 0x434, 0x5, 0x191, 0xc9, 0x2, 0x434, 0xa6, + 0x3, 0x2, 0x2, 0x2, 0x435, 0x436, 0x5, 0x18b, 0xc6, 0x2, 0x436, 0x437, + 0x5, 0x195, 0xcb, 0x2, 0x437, 0x438, 0x5, 0x1a1, 0xd1, 0x2, 0x438, 0x439, + 0x5, 0x197, 0xcc, 0x2, 0x439, 0xa8, 0x3, 0x2, 0x2, 0x2, 0x43a, 0x43b, + 0x5, 0x18b, 0xc6, 0x2, 0x43b, 0x43c, 0x5, 0x19f, 0xd0, 0x2, 0x43c, 0xaa, + 0x3, 0x2, 0x2, 0x2, 0x43d, 0x43e, 0x5, 0x18b, 0xc6, 0x2, 0x43e, 0x43f, + 0x5, 0x19f, 0xd0, 0x2, 0x43f, 0x440, 0x5, 0x1f1, 0xf9, 0x2, 0x440, 0x441, + 0x5, 0x197, 0xcc, 0x2, 0x441, 0x442, 0x5, 0x17d, 0xbf, 0x2, 0x442, 0x443, + 0x5, 0x18d, 0xc7, 0x2, 0x443, 0x444, 0x5, 0x183, 0xc2, 0x2, 0x444, 0x445, + 0x5, 0x17f, 0xc0, 0x2, 0x445, 0x446, 0x5, 0x1a1, 0xd1, 0x2, 0x446, 0x447, + 0x5, 0x1f1, 0xf9, 0x2, 0x447, 0x448, 0x5, 0x18b, 0xc6, 0x2, 0x448, 0x449, + 0x5, 0x181, 0xc1, 0x2, 0x449, 0xac, 0x3, 0x2, 0x2, 0x2, 0x44a, 0x44b, + 0x5, 0x18d, 0xc7, 0x2, 0x44b, 0x44c, 0x5, 0x197, 0xcc, 0x2, 0x44c, 0x44d, + 0x5, 0x18b, 0xc6, 0x2, 0x44d, 0x44e, 0x5, 0x195, 0xcb, 0x2, 0x44e, 0xae, + 0x3, 0x2, 0x2, 0x2, 0x44f, 0x450, 0x5, 0x18f, 0xc8, 0x2, 0x450, 0x451, + 0x5, 0x183, 0xc2, 0x2, 0x451, 0x452, 0x5, 0x1ab, 0xd6, 0x2, 0x452, 0xb0, + 0x3, 0x2, 0x2, 0x2, 0x453, 0x454, 0x5, 0x18f, 0xc8, 0x2, 0x454, 0x455, + 0x5, 0x18b, 0xc6, 0x2, 0x455, 0x456, 0x5, 0x191, 0xc9, 0x2, 0x456, 0x457, + 0x5, 0x191, 0xc9, 0x2, 0x457, 0xb2, 0x3, 0x2, 0x2, 0x2, 0x458, 0x459, + 0x5, 0x191, 0xc9, 0x2, 0x459, 0x45a, 0x5, 0x17b, 0xbe, 0x2, 0x45a, 0x45b, + 0x5, 0x19f, 0xd0, 0x2, 0x45b, 0x45c, 0x5, 0x1a1, 0xd1, 0x2, 0x45c, 0xb4, + 0x3, 0x2, 0x2, 0x2, 0x45d, 0x45e, 0x5, 0x191, 0xc9, 0x2, 0x45e, 0x45f, + 0x5, 0x17b, 0xbe, 0x2, 0x45f, 0x460, 0x5, 0x1ab, 0xd6, 0x2, 0x460, 0x461, + 0x5, 0x197, 0xcc, 0x2, 0x461, 0x462, 0x5, 0x1a3, 0xd2, 0x2, 0x462, 0x463, + 0x5, 0x1a1, 0xd1, 0x2, 0x463, 0xb6, 0x3, 0x2, 0x2, 0x2, 0x464, 0x465, + 0x5, 0x191, 0xc9, 0x2, 0x465, 0x466, 0x5, 0x183, 0xc2, 0x2, 0x466, 0x467, + 0x5, 0x17b, 0xbe, 0x2, 0x467, 0x468, 0x5, 0x181, 0xc1, 0x2, 0x468, 0x469, + 0x5, 0x18b, 0xc6, 0x2, 0x469, 0x46a, 0x5, 0x195, 0xcb, 0x2, 0x46a, 0x46b, + 0x5, 0x187, 0xc4, 0x2, 0x46b, 0xb8, 0x3, 0x2, 0x2, 0x2, 0x46c, 0x46d, + 0x5, 0x191, 0xc9, 0x2, 0x46d, 0x46e, 0x5, 0x183, 0xc2, 0x2, 0x46e, 0x46f, + 0x5, 0x185, 0xc3, 0x2, 0x46f, 0x470, 0x5, 0x1a1, 0xd1, 0x2, 0x470, 0xba, + 0x3, 0x2, 0x2, 0x2, 0x471, 0x472, 0x5, 0x191, 0xc9, 0x2, 0x472, 0x473, + 0x5, 0x18b, 0xc6, 0x2, 0x473, 0x474, 0x5, 0x185, 0xc3, 0x2, 0x474, 0x475, + 0x5, 0x183, 0xc2, 0x2, 0x475, 0x476, 0x5, 0x1a1, 0xd1, 0x2, 0x476, 0x477, + 0x5, 0x18b, 0xc6, 0x2, 0x477, 0x478, 0x5, 0x193, 0xca, 0x2, 0x478, 0x479, + 0x5, 0x183, 0xc2, 0x2, 0x479, 0xbc, 0x3, 0x2, 0x2, 0x2, 0x47a, 0x47b, + 0x5, 0x191, 0xc9, 0x2, 0x47b, 0x47c, 0x5, 0x18b, 0xc6, 0x2, 0x47c, 0x47d, + 0x5, 0x18f, 0xc8, 0x2, 0x47d, 0x47e, 0x5, 0x183, 0xc2, 0x2, 0x47e, 0xbe, + 0x3, 0x2, 0x2, 0x2, 0x47f, 0x480, 0x5, 0x191, 0xc9, 0x2, 0x480, 0x481, + 0x5, 0x18b, 0xc6, 0x2, 0x481, 0x482, 0x5, 0x193, 0xca, 0x2, 0x482, 0x483, + 0x5, 0x18b, 0xc6, 0x2, 0x483, 0x484, 0x5, 0x1a1, 0xd1, 0x2, 0x484, 0xc0, + 0x3, 0x2, 0x2, 0x2, 0x485, 0x486, 0x5, 0x191, 0xc9, 0x2, 0x486, 0x487, + 0x5, 0x18b, 0xc6, 0x2, 0x487, 0x488, 0x5, 0x1a5, 0xd3, 0x2, 0x488, 0x489, + 0x5, 0x183, 0xc2, 0x2, 0x489, 0xc2, 0x3, 0x2, 0x2, 0x2, 0x48a, 0x48b, + 0x5, 0x191, 0xc9, 0x2, 0x48b, 0x48c, 0x5, 0x197, 0xcc, 0x2, 0x48c, 0x48d, + 0x5, 0x17f, 0xc0, 0x2, 0x48d, 0x48e, 0x5, 0x17b, 0xbe, 0x2, 0x48e, 0x48f, + 0x5, 0x191, 0xc9, 0x2, 0x48f, 0xc4, 0x3, 0x2, 0x2, 0x2, 0x490, 0x491, + 0x5, 0x191, 0xc9, 0x2, 0x491, 0x492, 0x5, 0x197, 0xcc, 0x2, 0x492, 0x493, + 0x5, 0x187, 0xc4, 0x2, 0x493, 0x494, 0x5, 0x19f, 0xd0, 0x2, 0x494, 0xc6, + 0x3, 0x2, 0x2, 0x2, 0x495, 0x496, 0x5, 0x193, 0xca, 0x2, 0x496, 0x497, + 0x5, 0x17b, 0xbe, 0x2, 0x497, 0x498, 0x5, 0x1a1, 0xd1, 0x2, 0x498, 0x499, + 0x5, 0x183, 0xc2, 0x2, 0x499, 0x49a, 0x5, 0x19d, 0xcf, 0x2, 0x49a, 0x49b, + 0x5, 0x18b, 0xc6, 0x2, 0x49b, 0x49c, 0x5, 0x17b, 0xbe, 0x2, 0x49c, 0x49d, + 0x5, 0x191, 0xc9, 0x2, 0x49d, 0x49e, 0x5, 0x18b, 0xc6, 0x2, 0x49e, 0x49f, + 0x5, 0x1ad, 0xd7, 0x2, 0x49f, 0x4a0, 0x5, 0x183, 0xc2, 0x2, 0x4a0, 0x4a1, + 0x5, 0x181, 0xc1, 0x2, 0x4a1, 0xc8, 0x3, 0x2, 0x2, 0x2, 0x4a2, 0x4a3, + 0x5, 0x193, 0xca, 0x2, 0x4a3, 0x4a4, 0x5, 0x17b, 0xbe, 0x2, 0x4a4, 0x4a5, + 0x5, 0x1a1, 0xd1, 0x2, 0x4a5, 0x4a6, 0x5, 0x183, 0xc2, 0x2, 0x4a6, 0x4a7, + 0x5, 0x19d, 0xcf, 0x2, 0x4a7, 0x4a8, 0x5, 0x18b, 0xc6, 0x2, 0x4a8, 0x4a9, + 0x5, 0x17b, 0xbe, 0x2, 0x4a9, 0x4aa, 0x5, 0x191, 0xc9, 0x2, 0x4aa, 0x4ab, + 0x5, 0x18b, 0xc6, 0x2, 0x4ab, 0x4ac, 0x5, 0x1ad, 0xd7, 0x2, 0x4ac, 0x4ad, + 0x5, 0x183, 0xc2, 0x2, 0x4ad, 0xca, 0x3, 0x2, 0x2, 0x2, 0x4ae, 0x4af, + 0x5, 0x193, 0xca, 0x2, 0x4af, 0x4b0, 0x5, 0x17b, 0xbe, 0x2, 0x4b0, 0x4b1, + 0x5, 0x1a9, 0xd5, 0x2, 0x4b1, 0xcc, 0x3, 0x2, 0x2, 0x2, 0x4b2, 0x4b3, + 0x5, 0x193, 0xca, 0x2, 0x4b3, 0x4b4, 0x5, 0x183, 0xc2, 0x2, 0x4b4, 0x4b5, + 0x5, 0x19d, 0xcf, 0x2, 0x4b5, 0x4b6, 0x5, 0x187, 0xc4, 0x2, 0x4b6, 0x4b7, + 0x5, 0x183, 0xc2, 0x2, 0x4b7, 0x4b8, 0x5, 0x19f, 0xd0, 0x2, 0x4b8, 0xce, + 0x3, 0x2, 0x2, 0x2, 0x4b9, 0x4ba, 0x5, 0x193, 0xca, 0x2, 0x4ba, 0x4bb, + 0x5, 0x18b, 0xc6, 0x2, 0x4bb, 0x4bc, 0x5, 0x195, 0xcb, 0x2, 0x4bc, 0xd0, + 0x3, 0x2, 0x2, 0x2, 0x4bd, 0x4be, 0x5, 0x193, 0xca, 0x2, 0x4be, 0x4bf, + 0x5, 0x18b, 0xc6, 0x2, 0x4bf, 0x4c0, 0x5, 0x195, 0xcb, 0x2, 0x4c0, 0x4c1, + 0x5, 0x1a3, 0xd2, 0x2, 0x4c1, 0x4c2, 0x5, 0x1a1, 0xd1, 0x2, 0x4c2, 0x4c3, + 0x5, 0x183, 0xc2, 0x2, 0x4c3, 0xd2, 0x3, 0x2, 0x2, 0x2, 0x4c4, 0x4c5, + 0x5, 0x193, 0xca, 0x2, 0x4c5, 0x4c6, 0x5, 0x197, 0xcc, 0x2, 0x4c6, 0x4c7, + 0x5, 0x181, 0xc1, 0x2, 0x4c7, 0x4c8, 0x5, 0x18b, 0xc6, 0x2, 0x4c8, 0x4c9, + 0x5, 0x185, 0xc3, 0x2, 0x4c9, 0x4ca, 0x5, 0x1ab, 0xd6, 0x2, 0x4ca, 0xd4, + 0x3, 0x2, 0x2, 0x2, 0x4cb, 0x4cc, 0x5, 0x193, 0xca, 0x2, 0x4cc, 0x4cd, + 0x5, 0x197, 0xcc, 0x2, 0x4cd, 0x4ce, 0x5, 0x195, 0xcb, 0x2, 0x4ce, 0x4cf, + 0x5, 0x1a1, 0xd1, 0x2, 0x4cf, 0x4d0, 0x5, 0x189, 0xc5, 0x2, 0x4d0, 0xd6, + 0x3, 0x2, 0x2, 0x2, 0x4d1, 0x4d2, 0x5, 0x193, 0xca, 0x2, 0x4d2, 0x4d3, + 0x5, 0x197, 0xcc, 0x2, 0x4d3, 0x4d4, 0x5, 0x1a5, 0xd3, 0x2, 0x4d4, 0x4d5, + 0x5, 0x183, 0xc2, 0x2, 0x4d5, 0xd8, 0x3, 0x2, 0x2, 0x2, 0x4d6, 0x4d7, + 0x5, 0x193, 0xca, 0x2, 0x4d7, 0x4d8, 0x5, 0x1a3, 0xd2, 0x2, 0x4d8, 0x4d9, + 0x5, 0x1a1, 0xd1, 0x2, 0x4d9, 0x4da, 0x5, 0x17b, 0xbe, 0x2, 0x4da, 0x4db, + 0x5, 0x1a1, 0xd1, 0x2, 0x4db, 0x4dc, 0x5, 0x18b, 0xc6, 0x2, 0x4dc, 0x4dd, + 0x5, 0x197, 0xcc, 0x2, 0x4dd, 0x4de, 0x5, 0x195, 0xcb, 0x2, 0x4de, 0xda, + 0x3, 0x2, 0x2, 0x2, 0x4df, 0x4e0, 0x5, 0x195, 0xcb, 0x2, 0x4e0, 0x4e1, + 0x5, 0x17b, 0xbe, 0x2, 0x4e1, 0x4e2, 0x5, 0x195, 0xcb, 0x2, 0x4e2, 0xdc, + 0x3, 0x2, 0x2, 0x2, 0x4e3, 0x4e4, 0x5, 0x195, 0xcb, 0x2, 0x4e4, 0x4e5, + 0x5, 0x197, 0xcc, 0x2, 0x4e5, 0xde, 0x3, 0x2, 0x2, 0x2, 0x4e6, 0x4e7, + 0x5, 0x195, 0xcb, 0x2, 0x4e7, 0x4e8, 0x5, 0x197, 0xcc, 0x2, 0x4e8, 0x4e9, + 0x5, 0x1a1, 0xd1, 0x2, 0x4e9, 0xe0, 0x3, 0x2, 0x2, 0x2, 0x4ea, 0x4eb, + 0x5, 0x195, 0xcb, 0x2, 0x4eb, 0x4ec, 0x5, 0x1a3, 0xd2, 0x2, 0x4ec, 0x4ed, + 0x5, 0x191, 0xc9, 0x2, 0x4ed, 0x4ee, 0x5, 0x191, 0xc9, 0x2, 0x4ee, 0xe2, + 0x3, 0x2, 0x2, 0x2, 0x4ef, 0x4f0, 0x5, 0x195, 0xcb, 0x2, 0x4f0, 0x4f1, + 0x5, 0x1a3, 0xd2, 0x2, 0x4f1, 0x4f2, 0x5, 0x191, 0xc9, 0x2, 0x4f2, 0x4f3, + 0x5, 0x191, 0xc9, 0x2, 0x4f3, 0x4f4, 0x5, 0x19f, 0xd0, 0x2, 0x4f4, 0xe4, + 0x3, 0x2, 0x2, 0x2, 0x4f5, 0x4f6, 0x5, 0x197, 0xcc, 0x2, 0x4f6, 0x4f7, + 0x5, 0x185, 0xc3, 0x2, 0x4f7, 0x4f8, 0x5, 0x185, 0xc3, 0x2, 0x4f8, 0x4f9, + 0x5, 0x19f, 0xd0, 0x2, 0x4f9, 0x4fa, 0x5, 0x183, 0xc2, 0x2, 0x4fa, 0x4fb, + 0x5, 0x1a1, 0xd1, 0x2, 0x4fb, 0xe6, 0x3, 0x2, 0x2, 0x2, 0x4fc, 0x4fd, + 0x5, 0x197, 0xcc, 0x2, 0x4fd, 0x4fe, 0x5, 0x195, 0xcb, 0x2, 0x4fe, 0xe8, + 0x3, 0x2, 0x2, 0x2, 0x4ff, 0x500, 0x5, 0x197, 0xcc, 0x2, 0x500, 0x501, + 0x5, 0x199, 0xcd, 0x2, 0x501, 0x502, 0x5, 0x1a1, 0xd1, 0x2, 0x502, 0x503, + 0x5, 0x18b, 0xc6, 0x2, 0x503, 0x504, 0x5, 0x193, 0xca, 0x2, 0x504, 0x505, + 0x5, 0x18b, 0xc6, 0x2, 0x505, 0x506, 0x5, 0x1ad, 0xd7, 0x2, 0x506, 0x507, + 0x5, 0x183, 0xc2, 0x2, 0x507, 0xea, 0x3, 0x2, 0x2, 0x2, 0x508, 0x509, + 0x5, 0x197, 0xcc, 0x2, 0x509, 0x50a, 0x5, 0x19d, 0xcf, 0x2, 0x50a, 0xec, + 0x3, 0x2, 0x2, 0x2, 0x50b, 0x50c, 0x5, 0x197, 0xcc, 0x2, 0x50c, 0x50d, + 0x5, 0x19d, 0xcf, 0x2, 0x50d, 0x50e, 0x5, 0x181, 0xc1, 0x2, 0x50e, 0x50f, + 0x5, 0x183, 0xc2, 0x2, 0x50f, 0x510, 0x5, 0x19d, 0xcf, 0x2, 0x510, 0xee, + 0x3, 0x2, 0x2, 0x2, 0x511, 0x512, 0x5, 0x197, 0xcc, 0x2, 0x512, 0x513, + 0x5, 0x1a3, 0xd2, 0x2, 0x513, 0x514, 0x5, 0x1a1, 0xd1, 0x2, 0x514, 0x515, + 0x5, 0x183, 0xc2, 0x2, 0x515, 0x516, 0x5, 0x19d, 0xcf, 0x2, 0x516, 0xf0, + 0x3, 0x2, 0x2, 0x2, 0x517, 0x518, 0x5, 0x197, 0xcc, 0x2, 0x518, 0x519, + 0x5, 0x1a3, 0xd2, 0x2, 0x519, 0x51a, 0x5, 0x1a1, 0xd1, 0x2, 0x51a, 0x51b, + 0x5, 0x185, 0xc3, 0x2, 0x51b, 0x51c, 0x5, 0x18b, 0xc6, 0x2, 0x51c, 0x51d, + 0x5, 0x191, 0xc9, 0x2, 0x51d, 0x51e, 0x5, 0x183, 0xc2, 0x2, 0x51e, 0xf2, + 0x3, 0x2, 0x2, 0x2, 0x51f, 0x520, 0x5, 0x199, 0xcd, 0x2, 0x520, 0x521, + 0x5, 0x17b, 0xbe, 0x2, 0x521, 0x522, 0x5, 0x19d, 0xcf, 0x2, 0x522, 0x523, + 0x5, 0x1a1, 0xd1, 0x2, 0x523, 0x524, 0x5, 0x18b, 0xc6, 0x2, 0x524, 0x525, + 0x5, 0x1a1, 0xd1, 0x2, 0x525, 0x526, 0x5, 0x18b, 0xc6, 0x2, 0x526, 0x527, + 0x5, 0x197, 0xcc, 0x2, 0x527, 0x528, 0x5, 0x195, 0xcb, 0x2, 0x528, 0xf4, + 0x3, 0x2, 0x2, 0x2, 0x529, 0x52a, 0x5, 0x199, 0xcd, 0x2, 0x52a, 0x52b, + 0x5, 0x197, 0xcc, 0x2, 0x52b, 0x52c, 0x5, 0x199, 0xcd, 0x2, 0x52c, 0x52d, + 0x5, 0x1a3, 0xd2, 0x2, 0x52d, 0x52e, 0x5, 0x191, 0xc9, 0x2, 0x52e, 0x52f, + 0x5, 0x17b, 0xbe, 0x2, 0x52f, 0x530, 0x5, 0x1a1, 0xd1, 0x2, 0x530, 0x531, + 0x5, 0x183, 0xc2, 0x2, 0x531, 0xf6, 0x3, 0x2, 0x2, 0x2, 0x532, 0x533, + 0x5, 0x199, 0xcd, 0x2, 0x533, 0x534, 0x5, 0x19d, 0xcf, 0x2, 0x534, 0x535, + 0x5, 0x183, 0xc2, 0x2, 0x535, 0x536, 0x5, 0x1a7, 0xd4, 0x2, 0x536, 0x537, + 0x5, 0x189, 0xc5, 0x2, 0x537, 0x538, 0x5, 0x183, 0xc2, 0x2, 0x538, 0x539, + 0x5, 0x19d, 0xcf, 0x2, 0x539, 0x53a, 0x5, 0x183, 0xc2, 0x2, 0x53a, 0xf8, + 0x3, 0x2, 0x2, 0x2, 0x53b, 0x53c, 0x5, 0x199, 0xcd, 0x2, 0x53c, 0x53d, + 0x5, 0x19d, 0xcf, 0x2, 0x53d, 0x53e, 0x5, 0x18b, 0xc6, 0x2, 0x53e, 0x53f, + 0x5, 0x193, 0xca, 0x2, 0x53f, 0x540, 0x5, 0x17b, 0xbe, 0x2, 0x540, 0x541, + 0x5, 0x19d, 0xcf, 0x2, 0x541, 0x542, 0x5, 0x1ab, 0xd6, 0x2, 0x542, 0xfa, + 0x3, 0x2, 0x2, 0x2, 0x543, 0x544, 0x5, 0x199, 0xcd, 0x2, 0x544, 0x545, + 0x5, 0x19d, 0xcf, 0x2, 0x545, 0x546, 0x5, 0x197, 0xcc, 0x2, 0x546, 0x547, + 0x5, 0x18d, 0xc7, 0x2, 0x547, 0x548, 0x5, 0x183, 0xc2, 0x2, 0x548, 0x549, + 0x5, 0x17f, 0xc0, 0x2, 0x549, 0x54a, 0x5, 0x1a1, 0xd1, 0x2, 0x54a, 0x54b, + 0x5, 0x18b, 0xc6, 0x2, 0x54b, 0x54c, 0x5, 0x197, 0xcc, 0x2, 0x54c, 0x54d, + 0x5, 0x195, 0xcb, 0x2, 0x54d, 0xfc, 0x3, 0x2, 0x2, 0x2, 0x54e, 0x54f, + 0x5, 0x19b, 0xce, 0x2, 0x54f, 0x550, 0x5, 0x1a3, 0xd2, 0x2, 0x550, 0x551, + 0x5, 0x17b, 0xbe, 0x2, 0x551, 0x552, 0x5, 0x19d, 0xcf, 0x2, 0x552, 0x553, + 0x5, 0x1a1, 0xd1, 0x2, 0x553, 0x554, 0x5, 0x183, 0xc2, 0x2, 0x554, 0x555, + 0x5, 0x19d, 0xcf, 0x2, 0x555, 0xfe, 0x3, 0x2, 0x2, 0x2, 0x556, 0x557, + 0x5, 0x19d, 0xcf, 0x2, 0x557, 0x558, 0x5, 0x17b, 0xbe, 0x2, 0x558, 0x559, + 0x5, 0x195, 0xcb, 0x2, 0x559, 0x55a, 0x5, 0x187, 0xc4, 0x2, 0x55a, 0x55b, + 0x5, 0x183, 0xc2, 0x2, 0x55b, 0x100, 0x3, 0x2, 0x2, 0x2, 0x55c, 0x55d, + 0x5, 0x19d, 0xcf, 0x2, 0x55d, 0x55e, 0x5, 0x183, 0xc2, 0x2, 0x55e, 0x55f, + 0x5, 0x191, 0xc9, 0x2, 0x55f, 0x560, 0x5, 0x197, 0xcc, 0x2, 0x560, 0x561, + 0x5, 0x17b, 0xbe, 0x2, 0x561, 0x562, 0x5, 0x181, 0xc1, 0x2, 0x562, 0x102, + 0x3, 0x2, 0x2, 0x2, 0x563, 0x564, 0x5, 0x19d, 0xcf, 0x2, 0x564, 0x565, + 0x5, 0x183, 0xc2, 0x2, 0x565, 0x566, 0x5, 0x193, 0xca, 0x2, 0x566, 0x567, + 0x5, 0x197, 0xcc, 0x2, 0x567, 0x568, 0x5, 0x1a5, 0xd3, 0x2, 0x568, 0x569, + 0x5, 0x183, 0xc2, 0x2, 0x569, 0x104, 0x3, 0x2, 0x2, 0x2, 0x56a, 0x56b, + 0x5, 0x19d, 0xcf, 0x2, 0x56b, 0x56c, 0x5, 0x183, 0xc2, 0x2, 0x56c, 0x56d, + 0x5, 0x195, 0xcb, 0x2, 0x56d, 0x56e, 0x5, 0x17b, 0xbe, 0x2, 0x56e, 0x56f, + 0x5, 0x193, 0xca, 0x2, 0x56f, 0x570, 0x5, 0x183, 0xc2, 0x2, 0x570, 0x106, + 0x3, 0x2, 0x2, 0x2, 0x571, 0x572, 0x5, 0x19d, 0xcf, 0x2, 0x572, 0x573, + 0x5, 0x183, 0xc2, 0x2, 0x573, 0x574, 0x5, 0x199, 0xcd, 0x2, 0x574, 0x575, + 0x5, 0x191, 0xc9, 0x2, 0x575, 0x576, 0x5, 0x17b, 0xbe, 0x2, 0x576, 0x577, + 0x5, 0x17f, 0xc0, 0x2, 0x577, 0x578, 0x5, 0x183, 0xc2, 0x2, 0x578, 0x108, + 0x3, 0x2, 0x2, 0x2, 0x579, 0x57a, 0x5, 0x19d, 0xcf, 0x2, 0x57a, 0x57b, + 0x5, 0x183, 0xc2, 0x2, 0x57b, 0x57c, 0x5, 0x199, 0xcd, 0x2, 0x57c, 0x57d, + 0x5, 0x191, 0xc9, 0x2, 0x57d, 0x57e, 0x5, 0x18b, 0xc6, 0x2, 0x57e, 0x57f, + 0x5, 0x17f, 0xc0, 0x2, 0x57f, 0x580, 0x5, 0x17b, 0xbe, 0x2, 0x580, 0x10a, + 0x3, 0x2, 0x2, 0x2, 0x581, 0x582, 0x5, 0x19d, 0xcf, 0x2, 0x582, 0x583, + 0x5, 0x183, 0xc2, 0x2, 0x583, 0x584, 0x5, 0x199, 0xcd, 0x2, 0x584, 0x585, + 0x5, 0x191, 0xc9, 0x2, 0x585, 0x586, 0x5, 0x18b, 0xc6, 0x2, 0x586, 0x587, + 0x5, 0x17f, 0xc0, 0x2, 0x587, 0x588, 0x5, 0x17b, 0xbe, 0x2, 0x588, 0x589, + 0x5, 0x1a1, 0xd1, 0x2, 0x589, 0x58a, 0x5, 0x183, 0xc2, 0x2, 0x58a, 0x58b, + 0x5, 0x181, 0xc1, 0x2, 0x58b, 0x10c, 0x3, 0x2, 0x2, 0x2, 0x58c, 0x58d, + 0x5, 0x19d, 0xcf, 0x2, 0x58d, 0x58e, 0x5, 0x18b, 0xc6, 0x2, 0x58e, 0x58f, + 0x5, 0x187, 0xc4, 0x2, 0x58f, 0x590, 0x5, 0x189, 0xc5, 0x2, 0x590, 0x591, + 0x5, 0x1a1, 0xd1, 0x2, 0x591, 0x10e, 0x3, 0x2, 0x2, 0x2, 0x592, 0x593, + 0x5, 0x19d, 0xcf, 0x2, 0x593, 0x594, 0x5, 0x197, 0xcc, 0x2, 0x594, 0x595, + 0x5, 0x191, 0xc9, 0x2, 0x595, 0x596, 0x5, 0x191, 0xc9, 0x2, 0x596, 0x597, + 0x5, 0x1a3, 0xd2, 0x2, 0x597, 0x598, 0x5, 0x199, 0xcd, 0x2, 0x598, 0x110, + 0x3, 0x2, 0x2, 0x2, 0x599, 0x59a, 0x5, 0x19f, 0xd0, 0x2, 0x59a, 0x59b, + 0x5, 0x17b, 0xbe, 0x2, 0x59b, 0x59c, 0x5, 0x193, 0xca, 0x2, 0x59c, 0x59d, + 0x5, 0x199, 0xcd, 0x2, 0x59d, 0x59e, 0x5, 0x191, 0xc9, 0x2, 0x59e, 0x59f, + 0x5, 0x183, 0xc2, 0x2, 0x59f, 0x112, 0x3, 0x2, 0x2, 0x2, 0x5a0, 0x5a1, + 0x5, 0x19f, 0xd0, 0x2, 0x5a1, 0x5a2, 0x5, 0x183, 0xc2, 0x2, 0x5a2, 0x5a3, + 0x5, 0x17f, 0xc0, 0x2, 0x5a3, 0x5a4, 0x5, 0x197, 0xcc, 0x2, 0x5a4, 0x5a5, + 0x5, 0x195, 0xcb, 0x2, 0x5a5, 0x5a6, 0x5, 0x181, 0xc1, 0x2, 0x5a6, 0x114, + 0x3, 0x2, 0x2, 0x2, 0x5a7, 0x5a8, 0x5, 0x19f, 0xd0, 0x2, 0x5a8, 0x5a9, + 0x5, 0x183, 0xc2, 0x2, 0x5a9, 0x5aa, 0x5, 0x191, 0xc9, 0x2, 0x5aa, 0x5ab, + 0x5, 0x183, 0xc2, 0x2, 0x5ab, 0x5ac, 0x5, 0x17f, 0xc0, 0x2, 0x5ac, 0x5ad, + 0x5, 0x1a1, 0xd1, 0x2, 0x5ad, 0x116, 0x3, 0x2, 0x2, 0x2, 0x5ae, 0x5af, + 0x5, 0x19f, 0xd0, 0x2, 0x5af, 0x5b0, 0x5, 0x183, 0xc2, 0x2, 0x5b0, 0x5b1, + 0x5, 0x193, 0xca, 0x2, 0x5b1, 0x5b2, 0x5, 0x18b, 0xc6, 0x2, 0x5b2, 0x118, + 0x3, 0x2, 0x2, 0x2, 0x5b3, 0x5b4, 0x5, 0x19f, 0xd0, 0x2, 0x5b4, 0x5b5, + 0x5, 0x183, 0xc2, 0x2, 0x5b5, 0x5b6, 0x5, 0x195, 0xcb, 0x2, 0x5b6, 0x5b7, + 0x5, 0x181, 0xc1, 0x2, 0x5b7, 0x5b8, 0x5, 0x19f, 0xd0, 0x2, 0x5b8, 0x11a, + 0x3, 0x2, 0x2, 0x2, 0x5b9, 0x5ba, 0x5, 0x19f, 0xd0, 0x2, 0x5ba, 0x5bb, + 0x5, 0x183, 0xc2, 0x2, 0x5bb, 0x5bc, 0x5, 0x1a1, 0xd1, 0x2, 0x5bc, 0x11c, + 0x3, 0x2, 0x2, 0x2, 0x5bd, 0x5be, 0x5, 0x19f, 0xd0, 0x2, 0x5be, 0x5bf, + 0x5, 0x183, 0xc2, 0x2, 0x5bf, 0x5c0, 0x5, 0x1a1, 0xd1, 0x2, 0x5c0, 0x5c1, + 0x5, 0x1a1, 0xd1, 0x2, 0x5c1, 0x5c2, 0x5, 0x18b, 0xc6, 0x2, 0x5c2, 0x5c3, + 0x5, 0x195, 0xcb, 0x2, 0x5c3, 0x5c4, 0x5, 0x187, 0xc4, 0x2, 0x5c4, 0x5c5, + 0x5, 0x19f, 0xd0, 0x2, 0x5c5, 0x11e, 0x3, 0x2, 0x2, 0x2, 0x5c6, 0x5c7, + 0x5, 0x19f, 0xd0, 0x2, 0x5c7, 0x5c8, 0x5, 0x189, 0xc5, 0x2, 0x5c8, 0x5c9, + 0x5, 0x197, 0xcc, 0x2, 0x5c9, 0x5ca, 0x5, 0x1a7, 0xd4, 0x2, 0x5ca, 0x120, + 0x3, 0x2, 0x2, 0x2, 0x5cb, 0x5cc, 0x5, 0x19f, 0xd0, 0x2, 0x5cc, 0x5cd, + 0x5, 0x197, 0xcc, 0x2, 0x5cd, 0x5ce, 0x5, 0x1a3, 0xd2, 0x2, 0x5ce, 0x5cf, + 0x5, 0x19d, 0xcf, 0x2, 0x5cf, 0x5d0, 0x5, 0x17f, 0xc0, 0x2, 0x5d0, 0x5d1, + 0x5, 0x183, 0xc2, 0x2, 0x5d1, 0x122, 0x3, 0x2, 0x2, 0x2, 0x5d2, 0x5d3, + 0x5, 0x19f, 0xd0, 0x2, 0x5d3, 0x5d4, 0x5, 0x1a1, 0xd1, 0x2, 0x5d4, 0x5d5, + 0x5, 0x17b, 0xbe, 0x2, 0x5d5, 0x5d6, 0x5, 0x19d, 0xcf, 0x2, 0x5d6, 0x5d7, + 0x5, 0x1a1, 0xd1, 0x2, 0x5d7, 0x124, 0x3, 0x2, 0x2, 0x2, 0x5d8, 0x5d9, + 0x5, 0x19f, 0xd0, 0x2, 0x5d9, 0x5da, 0x5, 0x1a1, 0xd1, 0x2, 0x5da, 0x5db, + 0x5, 0x197, 0xcc, 0x2, 0x5db, 0x5dc, 0x5, 0x199, 0xcd, 0x2, 0x5dc, 0x126, + 0x3, 0x2, 0x2, 0x2, 0x5dd, 0x5de, 0x5, 0x19f, 0xd0, 0x2, 0x5de, 0x5df, + 0x5, 0x1a3, 0xd2, 0x2, 0x5df, 0x5e0, 0x5, 0x17d, 0xbf, 0x2, 0x5e0, 0x5e1, + 0x5, 0x19f, 0xd0, 0x2, 0x5e1, 0x5e2, 0x5, 0x1a1, 0xd1, 0x2, 0x5e2, 0x5e3, + 0x5, 0x19d, 0xcf, 0x2, 0x5e3, 0x5e4, 0x5, 0x18b, 0xc6, 0x2, 0x5e4, 0x5e5, + 0x5, 0x195, 0xcb, 0x2, 0x5e5, 0x5e6, 0x5, 0x187, 0xc4, 0x2, 0x5e6, 0x128, + 0x3, 0x2, 0x2, 0x2, 0x5e7, 0x5e8, 0x5, 0x19f, 0xd0, 0x2, 0x5e8, 0x5e9, + 0x5, 0x1ab, 0xd6, 0x2, 0x5e9, 0x5ea, 0x5, 0x195, 0xcb, 0x2, 0x5ea, 0x5eb, + 0x5, 0x17f, 0xc0, 0x2, 0x5eb, 0x12a, 0x3, 0x2, 0x2, 0x2, 0x5ec, 0x5ed, + 0x5, 0x19f, 0xd0, 0x2, 0x5ed, 0x5ee, 0x5, 0x1ab, 0xd6, 0x2, 0x5ee, 0x5ef, + 0x5, 0x195, 0xcb, 0x2, 0x5ef, 0x5f0, 0x5, 0x1a1, 0xd1, 0x2, 0x5f0, 0x5f1, + 0x5, 0x17b, 0xbe, 0x2, 0x5f1, 0x5f2, 0x5, 0x1a9, 0xd5, 0x2, 0x5f2, 0x12c, + 0x3, 0x2, 0x2, 0x2, 0x5f3, 0x5f4, 0x5, 0x19f, 0xd0, 0x2, 0x5f4, 0x5f5, + 0x5, 0x1ab, 0xd6, 0x2, 0x5f5, 0x5f6, 0x5, 0x19f, 0xd0, 0x2, 0x5f6, 0x5f7, + 0x5, 0x1a1, 0xd1, 0x2, 0x5f7, 0x5f8, 0x5, 0x183, 0xc2, 0x2, 0x5f8, 0x5f9, + 0x5, 0x193, 0xca, 0x2, 0x5f9, 0x12e, 0x3, 0x2, 0x2, 0x2, 0x5fa, 0x5fb, + 0x5, 0x1a1, 0xd1, 0x2, 0x5fb, 0x5fc, 0x5, 0x17b, 0xbe, 0x2, 0x5fc, 0x5fd, + 0x5, 0x17d, 0xbf, 0x2, 0x5fd, 0x5fe, 0x5, 0x191, 0xc9, 0x2, 0x5fe, 0x5ff, + 0x5, 0x183, 0xc2, 0x2, 0x5ff, 0x130, 0x3, 0x2, 0x2, 0x2, 0x600, 0x601, + 0x5, 0x1a1, 0xd1, 0x2, 0x601, 0x602, 0x5, 0x17b, 0xbe, 0x2, 0x602, 0x603, + 0x5, 0x17d, 0xbf, 0x2, 0x603, 0x604, 0x5, 0x191, 0xc9, 0x2, 0x604, 0x605, + 0x5, 0x183, 0xc2, 0x2, 0x605, 0x606, 0x5, 0x19f, 0xd0, 0x2, 0x606, 0x132, + 0x3, 0x2, 0x2, 0x2, 0x607, 0x608, 0x5, 0x1a1, 0xd1, 0x2, 0x608, 0x609, + 0x5, 0x183, 0xc2, 0x2, 0x609, 0x60a, 0x5, 0x193, 0xca, 0x2, 0x60a, 0x60b, + 0x5, 0x199, 0xcd, 0x2, 0x60b, 0x60c, 0x5, 0x197, 0xcc, 0x2, 0x60c, 0x60d, + 0x5, 0x19d, 0xcf, 0x2, 0x60d, 0x60e, 0x5, 0x17b, 0xbe, 0x2, 0x60e, 0x60f, + 0x5, 0x19d, 0xcf, 0x2, 0x60f, 0x610, 0x5, 0x1ab, 0xd6, 0x2, 0x610, 0x134, + 0x3, 0x2, 0x2, 0x2, 0x611, 0x612, 0x5, 0x1a1, 0xd1, 0x2, 0x612, 0x613, + 0x5, 0x183, 0xc2, 0x2, 0x613, 0x614, 0x5, 0x19f, 0xd0, 0x2, 0x614, 0x615, + 0x5, 0x1a1, 0xd1, 0x2, 0x615, 0x136, 0x3, 0x2, 0x2, 0x2, 0x616, 0x617, + 0x5, 0x1a1, 0xd1, 0x2, 0x617, 0x618, 0x5, 0x189, 0xc5, 0x2, 0x618, 0x619, + 0x5, 0x183, 0xc2, 0x2, 0x619, 0x61a, 0x5, 0x195, 0xcb, 0x2, 0x61a, 0x138, + 0x3, 0x2, 0x2, 0x2, 0x61b, 0x61c, 0x5, 0x1a1, 0xd1, 0x2, 0x61c, 0x61d, + 0x5, 0x18b, 0xc6, 0x2, 0x61d, 0x61e, 0x5, 0x183, 0xc2, 0x2, 0x61e, 0x61f, + 0x5, 0x19f, 0xd0, 0x2, 0x61f, 0x13a, 0x3, 0x2, 0x2, 0x2, 0x620, 0x621, + 0x5, 0x1a1, 0xd1, 0x2, 0x621, 0x622, 0x5, 0x18b, 0xc6, 0x2, 0x622, 0x623, + 0x5, 0x193, 0xca, 0x2, 0x623, 0x624, 0x5, 0x183, 0xc2, 0x2, 0x624, 0x625, + 0x5, 0x197, 0xcc, 0x2, 0x625, 0x626, 0x5, 0x1a3, 0xd2, 0x2, 0x626, 0x627, + 0x5, 0x1a1, 0xd1, 0x2, 0x627, 0x13c, 0x3, 0x2, 0x2, 0x2, 0x628, 0x629, + 0x5, 0x1a1, 0xd1, 0x2, 0x629, 0x62a, 0x5, 0x18b, 0xc6, 0x2, 0x62a, 0x62b, + 0x5, 0x193, 0xca, 0x2, 0x62b, 0x62c, 0x5, 0x183, 0xc2, 0x2, 0x62c, 0x62d, + 0x5, 0x19f, 0xd0, 0x2, 0x62d, 0x62e, 0x5, 0x1a1, 0xd1, 0x2, 0x62e, 0x62f, + 0x5, 0x17b, 0xbe, 0x2, 0x62f, 0x630, 0x5, 0x193, 0xca, 0x2, 0x630, 0x631, + 0x5, 0x199, 0xcd, 0x2, 0x631, 0x13e, 0x3, 0x2, 0x2, 0x2, 0x632, 0x633, + 0x5, 0x1a1, 0xd1, 0x2, 0x633, 0x634, 0x5, 0x197, 0xcc, 0x2, 0x634, 0x140, + 0x3, 0x2, 0x2, 0x2, 0x635, 0x636, 0x5, 0x1a1, 0xd1, 0x2, 0x636, 0x637, + 0x5, 0x197, 0xcc, 0x2, 0x637, 0x638, 0x5, 0x199, 0xcd, 0x2, 0x638, 0x142, + 0x3, 0x2, 0x2, 0x2, 0x639, 0x63a, 0x5, 0x1a1, 0xd1, 0x2, 0x63a, 0x63b, + 0x5, 0x197, 0xcc, 0x2, 0x63b, 0x63c, 0x5, 0x1a1, 0xd1, 0x2, 0x63c, 0x63d, + 0x5, 0x17b, 0xbe, 0x2, 0x63d, 0x63e, 0x5, 0x191, 0xc9, 0x2, 0x63e, 0x63f, + 0x5, 0x19f, 0xd0, 0x2, 0x63f, 0x144, 0x3, 0x2, 0x2, 0x2, 0x640, 0x641, + 0x5, 0x1a1, 0xd1, 0x2, 0x641, 0x642, 0x5, 0x19d, 0xcf, 0x2, 0x642, 0x643, + 0x5, 0x17b, 0xbe, 0x2, 0x643, 0x644, 0x5, 0x18b, 0xc6, 0x2, 0x644, 0x645, + 0x5, 0x191, 0xc9, 0x2, 0x645, 0x646, 0x5, 0x18b, 0xc6, 0x2, 0x646, 0x647, + 0x5, 0x195, 0xcb, 0x2, 0x647, 0x648, 0x5, 0x187, 0xc4, 0x2, 0x648, 0x146, + 0x3, 0x2, 0x2, 0x2, 0x649, 0x64a, 0x5, 0x1a1, 0xd1, 0x2, 0x64a, 0x64b, + 0x5, 0x19d, 0xcf, 0x2, 0x64b, 0x64c, 0x5, 0x18b, 0xc6, 0x2, 0x64c, 0x64d, + 0x5, 0x193, 0xca, 0x2, 0x64d, 0x148, 0x3, 0x2, 0x2, 0x2, 0x64e, 0x64f, + 0x5, 0x1a1, 0xd1, 0x2, 0x64f, 0x650, 0x5, 0x19d, 0xcf, 0x2, 0x650, 0x651, + 0x5, 0x1a3, 0xd2, 0x2, 0x651, 0x652, 0x5, 0x195, 0xcb, 0x2, 0x652, 0x653, + 0x5, 0x17f, 0xc0, 0x2, 0x653, 0x654, 0x5, 0x17b, 0xbe, 0x2, 0x654, 0x655, + 0x5, 0x1a1, 0xd1, 0x2, 0x655, 0x656, 0x5, 0x183, 0xc2, 0x2, 0x656, 0x14a, + 0x3, 0x2, 0x2, 0x2, 0x657, 0x658, 0x5, 0x1a1, 0xd1, 0x2, 0x658, 0x659, + 0x5, 0x1a1, 0xd1, 0x2, 0x659, 0x65a, 0x5, 0x191, 0xc9, 0x2, 0x65a, 0x14c, + 0x3, 0x2, 0x2, 0x2, 0x65b, 0x65c, 0x5, 0x1a1, 0xd1, 0x2, 0x65c, 0x65d, + 0x5, 0x1ab, 0xd6, 0x2, 0x65d, 0x65e, 0x5, 0x199, 0xcd, 0x2, 0x65e, 0x65f, + 0x5, 0x183, 0xc2, 0x2, 0x65f, 0x14e, 0x3, 0x2, 0x2, 0x2, 0x660, 0x661, + 0x5, 0x1a3, 0xd2, 0x2, 0x661, 0x662, 0x5, 0x195, 0xcb, 0x2, 0x662, 0x663, + 0x5, 0x18b, 0xc6, 0x2, 0x663, 0x664, 0x5, 0x197, 0xcc, 0x2, 0x664, 0x665, + 0x5, 0x195, 0xcb, 0x2, 0x665, 0x150, 0x3, 0x2, 0x2, 0x2, 0x666, 0x667, + 0x5, 0x1a3, 0xd2, 0x2, 0x667, 0x668, 0x5, 0x199, 0xcd, 0x2, 0x668, 0x669, + 0x5, 0x181, 0xc1, 0x2, 0x669, 0x66a, 0x5, 0x17b, 0xbe, 0x2, 0x66a, 0x66b, + 0x5, 0x1a1, 0xd1, 0x2, 0x66b, 0x66c, 0x5, 0x183, 0xc2, 0x2, 0x66c, 0x152, + 0x3, 0x2, 0x2, 0x2, 0x66d, 0x66e, 0x5, 0x1a3, 0xd2, 0x2, 0x66e, 0x66f, + 0x5, 0x19f, 0xd0, 0x2, 0x66f, 0x670, 0x5, 0x183, 0xc2, 0x2, 0x670, 0x154, + 0x3, 0x2, 0x2, 0x2, 0x671, 0x672, 0x5, 0x1a3, 0xd2, 0x2, 0x672, 0x673, + 0x5, 0x19f, 0xd0, 0x2, 0x673, 0x674, 0x5, 0x18b, 0xc6, 0x2, 0x674, 0x675, + 0x5, 0x195, 0xcb, 0x2, 0x675, 0x676, 0x5, 0x187, 0xc4, 0x2, 0x676, 0x156, + 0x3, 0x2, 0x2, 0x2, 0x677, 0x678, 0x5, 0x1a3, 0xd2, 0x2, 0x678, 0x679, + 0x5, 0x1a3, 0xd2, 0x2, 0x679, 0x67a, 0x5, 0x18b, 0xc6, 0x2, 0x67a, 0x67b, + 0x5, 0x181, 0xc1, 0x2, 0x67b, 0x158, 0x3, 0x2, 0x2, 0x2, 0x67c, 0x67d, + 0x5, 0x1a5, 0xd3, 0x2, 0x67d, 0x67e, 0x5, 0x17b, 0xbe, 0x2, 0x67e, 0x67f, + 0x5, 0x191, 0xc9, 0x2, 0x67f, 0x680, 0x5, 0x1a3, 0xd2, 0x2, 0x680, 0x681, + 0x5, 0x183, 0xc2, 0x2, 0x681, 0x682, 0x5, 0x19f, 0xd0, 0x2, 0x682, 0x15a, + 0x3, 0x2, 0x2, 0x2, 0x683, 0x684, 0x5, 0x1a5, 0xd3, 0x2, 0x684, 0x685, + 0x5, 0x18b, 0xc6, 0x2, 0x685, 0x686, 0x5, 0x183, 0xc2, 0x2, 0x686, 0x687, + 0x5, 0x1a7, 0xd4, 0x2, 0x687, 0x15c, 0x3, 0x2, 0x2, 0x2, 0x688, 0x689, + 0x5, 0x1a5, 0xd3, 0x2, 0x689, 0x68a, 0x5, 0x197, 0xcc, 0x2, 0x68a, 0x68b, + 0x5, 0x191, 0xc9, 0x2, 0x68b, 0x68c, 0x5, 0x1a3, 0xd2, 0x2, 0x68c, 0x68d, + 0x5, 0x193, 0xca, 0x2, 0x68d, 0x68e, 0x5, 0x183, 0xc2, 0x2, 0x68e, 0x15e, + 0x3, 0x2, 0x2, 0x2, 0x68f, 0x690, 0x5, 0x1a7, 0xd4, 0x2, 0x690, 0x691, + 0x5, 0x17b, 0xbe, 0x2, 0x691, 0x692, 0x5, 0x1a1, 0xd1, 0x2, 0x692, 0x693, + 0x5, 0x17f, 0xc0, 0x2, 0x693, 0x694, 0x5, 0x189, 0xc5, 0x2, 0x694, 0x160, + 0x3, 0x2, 0x2, 0x2, 0x695, 0x696, 0x5, 0x1a7, 0xd4, 0x2, 0x696, 0x697, + 0x5, 0x183, 0xc2, 0x2, 0x697, 0x698, 0x5, 0x183, 0xc2, 0x2, 0x698, 0x699, + 0x5, 0x18f, 0xc8, 0x2, 0x699, 0x162, 0x3, 0x2, 0x2, 0x2, 0x69a, 0x69b, + 0x5, 0x1a7, 0xd4, 0x2, 0x69b, 0x69c, 0x5, 0x189, 0xc5, 0x2, 0x69c, 0x69d, + 0x5, 0x183, 0xc2, 0x2, 0x69d, 0x69e, 0x5, 0x195, 0xcb, 0x2, 0x69e, 0x164, + 0x3, 0x2, 0x2, 0x2, 0x69f, 0x6a0, 0x5, 0x1a7, 0xd4, 0x2, 0x6a0, 0x6a1, + 0x5, 0x189, 0xc5, 0x2, 0x6a1, 0x6a2, 0x5, 0x183, 0xc2, 0x2, 0x6a2, 0x6a3, + 0x5, 0x19d, 0xcf, 0x2, 0x6a3, 0x6a4, 0x5, 0x183, 0xc2, 0x2, 0x6a4, 0x166, + 0x3, 0x2, 0x2, 0x2, 0x6a5, 0x6a6, 0x5, 0x1a7, 0xd4, 0x2, 0x6a6, 0x6a7, + 0x5, 0x18b, 0xc6, 0x2, 0x6a7, 0x6a8, 0x5, 0x1a1, 0xd1, 0x2, 0x6a8, 0x6a9, + 0x5, 0x189, 0xc5, 0x2, 0x6a9, 0x168, 0x3, 0x2, 0x2, 0x2, 0x6aa, 0x6ab, + 0x5, 0x1ab, 0xd6, 0x2, 0x6ab, 0x6ac, 0x5, 0x183, 0xc2, 0x2, 0x6ac, 0x6ad, + 0x5, 0x17b, 0xbe, 0x2, 0x6ad, 0x6ae, 0x5, 0x19d, 0xcf, 0x2, 0x6ae, 0x6b5, + 0x3, 0x2, 0x2, 0x2, 0x6af, 0x6b0, 0x5, 0x1ab, 0xd6, 0x2, 0x6b0, 0x6b1, + 0x5, 0x1ab, 0xd6, 0x2, 0x6b1, 0x6b2, 0x5, 0x1ab, 0xd6, 0x2, 0x6b2, 0x6b3, + 0x5, 0x1ab, 0xd6, 0x2, 0x6b3, 0x6b5, 0x3, 0x2, 0x2, 0x2, 0x6b4, 0x6aa, + 0x3, 0x2, 0x2, 0x2, 0x6b4, 0x6af, 0x3, 0x2, 0x2, 0x2, 0x6b5, 0x16a, + 0x3, 0x2, 0x2, 0x2, 0x6b6, 0x6b7, 0x7, 0x68, 0x2, 0x2, 0x6b7, 0x6b8, + 0x7, 0x63, 0x2, 0x2, 0x6b8, 0x6b9, 0x7, 0x6e, 0x2, 0x2, 0x6b9, 0x6ba, + 0x7, 0x75, 0x2, 0x2, 0x6ba, 0x6bb, 0x7, 0x67, 0x2, 0x2, 0x6bb, 0x16c, + 0x3, 0x2, 0x2, 0x2, 0x6bc, 0x6bd, 0x7, 0x76, 0x2, 0x2, 0x6bd, 0x6be, + 0x7, 0x74, 0x2, 0x2, 0x6be, 0x6bf, 0x7, 0x77, 0x2, 0x2, 0x6bf, 0x6c0, + 0x7, 0x67, 0x2, 0x2, 0x6c0, 0x16e, 0x3, 0x2, 0x2, 0x2, 0x6c1, 0x6c4, + 0x5, 0x1af, 0xd8, 0x2, 0x6c2, 0x6c4, 0x5, 0x1f1, 0xf9, 0x2, 0x6c3, 0x6c1, + 0x3, 0x2, 0x2, 0x2, 0x6c3, 0x6c2, 0x3, 0x2, 0x2, 0x2, 0x6c4, 0x6ca, + 0x3, 0x2, 0x2, 0x2, 0x6c5, 0x6c9, 0x5, 0x1af, 0xd8, 0x2, 0x6c6, 0x6c9, + 0x5, 0x1f1, 0xf9, 0x2, 0x6c7, 0x6c9, 0x5, 0x1b3, 0xda, 0x2, 0x6c8, 0x6c5, + 0x3, 0x2, 0x2, 0x2, 0x6c8, 0x6c6, 0x3, 0x2, 0x2, 0x2, 0x6c8, 0x6c7, + 0x3, 0x2, 0x2, 0x2, 0x6c9, 0x6cc, 0x3, 0x2, 0x2, 0x2, 0x6ca, 0x6c8, + 0x3, 0x2, 0x2, 0x2, 0x6ca, 0x6cb, 0x3, 0x2, 0x2, 0x2, 0x6cb, 0x6ec, + 0x3, 0x2, 0x2, 0x2, 0x6cc, 0x6ca, 0x3, 0x2, 0x2, 0x2, 0x6cd, 0x6d7, + 0x5, 0x1bb, 0xde, 0x2, 0x6ce, 0x6d6, 0xa, 0x2, 0x2, 0x2, 0x6cf, 0x6d0, + 0x5, 0x1bd, 0xdf, 0x2, 0x6d0, 0x6d1, 0xb, 0x2, 0x2, 0x2, 0x6d1, 0x6d6, + 0x3, 0x2, 0x2, 0x2, 0x6d2, 0x6d3, 0x5, 0x1bb, 0xde, 0x2, 0x6d3, 0x6d4, + 0x5, 0x1bb, 0xde, 0x2, 0x6d4, 0x6d6, 0x3, 0x2, 0x2, 0x2, 0x6d5, 0x6ce, + 0x3, 0x2, 0x2, 0x2, 0x6d5, 0x6cf, 0x3, 0x2, 0x2, 0x2, 0x6d5, 0x6d2, + 0x3, 0x2, 0x2, 0x2, 0x6d6, 0x6d9, 0x3, 0x2, 0x2, 0x2, 0x6d7, 0x6d5, + 0x3, 0x2, 0x2, 0x2, 0x6d7, 0x6d8, 0x3, 0x2, 0x2, 0x2, 0x6d8, 0x6da, + 0x3, 0x2, 0x2, 0x2, 0x6d9, 0x6d7, 0x3, 0x2, 0x2, 0x2, 0x6da, 0x6db, + 0x5, 0x1bb, 0xde, 0x2, 0x6db, 0x6ec, 0x3, 0x2, 0x2, 0x2, 0x6dc, 0x6e6, + 0x5, 0x1e3, 0xf2, 0x2, 0x6dd, 0x6e5, 0xa, 0x3, 0x2, 0x2, 0x6de, 0x6df, + 0x5, 0x1bd, 0xdf, 0x2, 0x6df, 0x6e0, 0xb, 0x2, 0x2, 0x2, 0x6e0, 0x6e5, + 0x3, 0x2, 0x2, 0x2, 0x6e1, 0x6e2, 0x5, 0x1e3, 0xf2, 0x2, 0x6e2, 0x6e3, + 0x5, 0x1e3, 0xf2, 0x2, 0x6e3, 0x6e5, 0x3, 0x2, 0x2, 0x2, 0x6e4, 0x6dd, + 0x3, 0x2, 0x2, 0x2, 0x6e4, 0x6de, 0x3, 0x2, 0x2, 0x2, 0x6e4, 0x6e1, + 0x3, 0x2, 0x2, 0x2, 0x6e5, 0x6e8, 0x3, 0x2, 0x2, 0x2, 0x6e6, 0x6e4, + 0x3, 0x2, 0x2, 0x2, 0x6e6, 0x6e7, 0x3, 0x2, 0x2, 0x2, 0x6e7, 0x6e9, + 0x3, 0x2, 0x2, 0x2, 0x6e8, 0x6e6, 0x3, 0x2, 0x2, 0x2, 0x6e9, 0x6ea, + 0x5, 0x1e3, 0xf2, 0x2, 0x6ea, 0x6ec, 0x3, 0x2, 0x2, 0x2, 0x6eb, 0x6c3, + 0x3, 0x2, 0x2, 0x2, 0x6eb, 0x6cd, 0x3, 0x2, 0x2, 0x2, 0x6eb, 0x6dc, + 0x3, 0x2, 0x2, 0x2, 0x6ec, 0x170, 0x3, 0x2, 0x2, 0x2, 0x6ed, 0x6ee, + 0x5, 0x177, 0xbc, 0x2, 0x6ee, 0x6f2, 0x5, 0x1c7, 0xe4, 0x2, 0x6ef, 0x6f1, + 0x5, 0x1b5, 0xdb, 0x2, 0x6f0, 0x6ef, 0x3, 0x2, 0x2, 0x2, 0x6f1, 0x6f4, + 0x3, 0x2, 0x2, 0x2, 0x6f2, 0x6f0, 0x3, 0x2, 0x2, 0x2, 0x6f2, 0x6f3, + 0x3, 0x2, 0x2, 0x2, 0x6f3, 0x6f7, 0x3, 0x2, 0x2, 0x2, 0x6f4, 0x6f2, + 0x3, 0x2, 0x2, 0x2, 0x6f5, 0x6f8, 0x5, 0x199, 0xcd, 0x2, 0x6f6, 0x6f8, + 0x5, 0x183, 0xc2, 0x2, 0x6f7, 0x6f5, 0x3, 0x2, 0x2, 0x2, 0x6f7, 0x6f6, + 0x3, 0x2, 0x2, 0x2, 0x6f8, 0x6fb, 0x3, 0x2, 0x2, 0x2, 0x6f9, 0x6fc, + 0x5, 0x1df, 0xf0, 0x2, 0x6fa, 0x6fc, 0x5, 0x1c5, 0xe3, 0x2, 0x6fb, 0x6f9, + 0x3, 0x2, 0x2, 0x2, 0x6fb, 0x6fa, 0x3, 0x2, 0x2, 0x2, 0x6fb, 0x6fc, + 0x3, 0x2, 0x2, 0x2, 0x6fc, 0x6fe, 0x3, 0x2, 0x2, 0x2, 0x6fd, 0x6ff, + 0x5, 0x1b3, 0xda, 0x2, 0x6fe, 0x6fd, 0x3, 0x2, 0x2, 0x2, 0x6ff, 0x700, + 0x3, 0x2, 0x2, 0x2, 0x700, 0x6fe, 0x3, 0x2, 0x2, 0x2, 0x700, 0x701, + 0x3, 0x2, 0x2, 0x2, 0x701, 0x73a, 0x3, 0x2, 0x2, 0x2, 0x702, 0x705, + 0x5, 0x177, 0xbc, 0x2, 0x703, 0x706, 0x5, 0x199, 0xcd, 0x2, 0x704, 0x706, + 0x5, 0x183, 0xc2, 0x2, 0x705, 0x703, 0x3, 0x2, 0x2, 0x2, 0x705, 0x704, + 0x3, 0x2, 0x2, 0x2, 0x706, 0x709, 0x3, 0x2, 0x2, 0x2, 0x707, 0x70a, + 0x5, 0x1df, 0xf0, 0x2, 0x708, 0x70a, 0x5, 0x1c5, 0xe3, 0x2, 0x709, 0x707, + 0x3, 0x2, 0x2, 0x2, 0x709, 0x708, 0x3, 0x2, 0x2, 0x2, 0x709, 0x70a, + 0x3, 0x2, 0x2, 0x2, 0x70a, 0x70c, 0x3, 0x2, 0x2, 0x2, 0x70b, 0x70d, + 0x5, 0x1b3, 0xda, 0x2, 0x70c, 0x70b, 0x3, 0x2, 0x2, 0x2, 0x70d, 0x70e, + 0x3, 0x2, 0x2, 0x2, 0x70e, 0x70c, 0x3, 0x2, 0x2, 0x2, 0x70e, 0x70f, + 0x3, 0x2, 0x2, 0x2, 0x70f, 0x73a, 0x3, 0x2, 0x2, 0x2, 0x710, 0x711, + 0x5, 0x175, 0xbb, 0x2, 0x711, 0x715, 0x5, 0x1c7, 0xe4, 0x2, 0x712, 0x714, + 0x5, 0x1b3, 0xda, 0x2, 0x713, 0x712, 0x3, 0x2, 0x2, 0x2, 0x714, 0x717, + 0x3, 0x2, 0x2, 0x2, 0x715, 0x713, 0x3, 0x2, 0x2, 0x2, 0x715, 0x716, + 0x3, 0x2, 0x2, 0x2, 0x716, 0x718, 0x3, 0x2, 0x2, 0x2, 0x717, 0x715, + 0x3, 0x2, 0x2, 0x2, 0x718, 0x71b, 0x5, 0x183, 0xc2, 0x2, 0x719, 0x71c, + 0x5, 0x1df, 0xf0, 0x2, 0x71a, 0x71c, 0x5, 0x1c5, 0xe3, 0x2, 0x71b, 0x719, + 0x3, 0x2, 0x2, 0x2, 0x71b, 0x71a, 0x3, 0x2, 0x2, 0x2, 0x71b, 0x71c, + 0x3, 0x2, 0x2, 0x2, 0x71c, 0x71e, 0x3, 0x2, 0x2, 0x2, 0x71d, 0x71f, + 0x5, 0x1b3, 0xda, 0x2, 0x71e, 0x71d, 0x3, 0x2, 0x2, 0x2, 0x71f, 0x720, + 0x3, 0x2, 0x2, 0x2, 0x720, 0x71e, 0x3, 0x2, 0x2, 0x2, 0x720, 0x721, + 0x3, 0x2, 0x2, 0x2, 0x721, 0x73a, 0x3, 0x2, 0x2, 0x2, 0x722, 0x723, + 0x5, 0x1c7, 0xe4, 0x2, 0x723, 0x724, 0x5, 0x175, 0xbb, 0x2, 0x724, 0x727, + 0x5, 0x183, 0xc2, 0x2, 0x725, 0x728, 0x5, 0x1df, 0xf0, 0x2, 0x726, 0x728, + 0x5, 0x1c5, 0xe3, 0x2, 0x727, 0x725, 0x3, 0x2, 0x2, 0x2, 0x727, 0x726, + 0x3, 0x2, 0x2, 0x2, 0x727, 0x728, 0x3, 0x2, 0x2, 0x2, 0x728, 0x72a, + 0x3, 0x2, 0x2, 0x2, 0x729, 0x72b, 0x5, 0x1b3, 0xda, 0x2, 0x72a, 0x729, + 0x3, 0x2, 0x2, 0x2, 0x72b, 0x72c, 0x3, 0x2, 0x2, 0x2, 0x72c, 0x72a, + 0x3, 0x2, 0x2, 0x2, 0x72c, 0x72d, 0x3, 0x2, 0x2, 0x2, 0x72d, 0x73a, + 0x3, 0x2, 0x2, 0x2, 0x72e, 0x72f, 0x5, 0x175, 0xbb, 0x2, 0x72f, 0x732, + 0x5, 0x183, 0xc2, 0x2, 0x730, 0x733, 0x5, 0x1df, 0xf0, 0x2, 0x731, 0x733, + 0x5, 0x1c5, 0xe3, 0x2, 0x732, 0x730, 0x3, 0x2, 0x2, 0x2, 0x732, 0x731, + 0x3, 0x2, 0x2, 0x2, 0x732, 0x733, 0x3, 0x2, 0x2, 0x2, 0x733, 0x735, + 0x3, 0x2, 0x2, 0x2, 0x734, 0x736, 0x5, 0x1b3, 0xda, 0x2, 0x735, 0x734, + 0x3, 0x2, 0x2, 0x2, 0x736, 0x737, 0x3, 0x2, 0x2, 0x2, 0x737, 0x735, + 0x3, 0x2, 0x2, 0x2, 0x737, 0x738, 0x3, 0x2, 0x2, 0x2, 0x738, 0x73a, + 0x3, 0x2, 0x2, 0x2, 0x739, 0x6ed, 0x3, 0x2, 0x2, 0x2, 0x739, 0x702, + 0x3, 0x2, 0x2, 0x2, 0x739, 0x710, 0x3, 0x2, 0x2, 0x2, 0x739, 0x722, + 0x3, 0x2, 0x2, 0x2, 0x739, 0x72e, 0x3, 0x2, 0x2, 0x2, 0x73a, 0x172, + 0x3, 0x2, 0x2, 0x2, 0x73b, 0x73d, 0x7, 0x32, 0x2, 0x2, 0x73c, 0x73e, + 0x5, 0x1b1, 0xd9, 0x2, 0x73d, 0x73c, 0x3, 0x2, 0x2, 0x2, 0x73e, 0x73f, + 0x3, 0x2, 0x2, 0x2, 0x73f, 0x73d, 0x3, 0x2, 0x2, 0x2, 0x73f, 0x740, + 0x3, 0x2, 0x2, 0x2, 0x740, 0x174, 0x3, 0x2, 0x2, 0x2, 0x741, 0x743, + 0x5, 0x1b3, 0xda, 0x2, 0x742, 0x741, 0x3, 0x2, 0x2, 0x2, 0x743, 0x744, + 0x3, 0x2, 0x2, 0x2, 0x744, 0x742, 0x3, 0x2, 0x2, 0x2, 0x744, 0x745, + 0x3, 0x2, 0x2, 0x2, 0x745, 0x176, 0x3, 0x2, 0x2, 0x2, 0x746, 0x747, + 0x7, 0x32, 0x2, 0x2, 0x747, 0x749, 0x5, 0x1a9, 0xd5, 0x2, 0x748, 0x74a, + 0x5, 0x1b5, 0xdb, 0x2, 0x749, 0x748, 0x3, 0x2, 0x2, 0x2, 0x74a, 0x74b, + 0x3, 0x2, 0x2, 0x2, 0x74b, 0x749, 0x3, 0x2, 0x2, 0x2, 0x74b, 0x74c, + 0x3, 0x2, 0x2, 0x2, 0x74c, 0x178, 0x3, 0x2, 0x2, 0x2, 0x74d, 0x757, + 0x5, 0x1e5, 0xf3, 0x2, 0x74e, 0x756, 0xa, 0x4, 0x2, 0x2, 0x74f, 0x750, + 0x5, 0x1bd, 0xdf, 0x2, 0x750, 0x751, 0xb, 0x2, 0x2, 0x2, 0x751, 0x756, + 0x3, 0x2, 0x2, 0x2, 0x752, 0x753, 0x5, 0x1e5, 0xf3, 0x2, 0x753, 0x754, + 0x5, 0x1e5, 0xf3, 0x2, 0x754, 0x756, 0x3, 0x2, 0x2, 0x2, 0x755, 0x74e, + 0x3, 0x2, 0x2, 0x2, 0x755, 0x74f, 0x3, 0x2, 0x2, 0x2, 0x755, 0x752, + 0x3, 0x2, 0x2, 0x2, 0x756, 0x759, 0x3, 0x2, 0x2, 0x2, 0x757, 0x755, + 0x3, 0x2, 0x2, 0x2, 0x757, 0x758, 0x3, 0x2, 0x2, 0x2, 0x758, 0x75a, + 0x3, 0x2, 0x2, 0x2, 0x759, 0x757, 0x3, 0x2, 0x2, 0x2, 0x75a, 0x75b, + 0x5, 0x1e5, 0xf3, 0x2, 0x75b, 0x17a, 0x3, 0x2, 0x2, 0x2, 0x75c, 0x75d, + 0x9, 0x5, 0x2, 0x2, 0x75d, 0x17c, 0x3, 0x2, 0x2, 0x2, 0x75e, 0x75f, + 0x9, 0x6, 0x2, 0x2, 0x75f, 0x17e, 0x3, 0x2, 0x2, 0x2, 0x760, 0x761, + 0x9, 0x7, 0x2, 0x2, 0x761, 0x180, 0x3, 0x2, 0x2, 0x2, 0x762, 0x763, + 0x9, 0x8, 0x2, 0x2, 0x763, 0x182, 0x3, 0x2, 0x2, 0x2, 0x764, 0x765, + 0x9, 0x9, 0x2, 0x2, 0x765, 0x184, 0x3, 0x2, 0x2, 0x2, 0x766, 0x767, + 0x9, 0xa, 0x2, 0x2, 0x767, 0x186, 0x3, 0x2, 0x2, 0x2, 0x768, 0x769, + 0x9, 0xb, 0x2, 0x2, 0x769, 0x188, 0x3, 0x2, 0x2, 0x2, 0x76a, 0x76b, + 0x9, 0xc, 0x2, 0x2, 0x76b, 0x18a, 0x3, 0x2, 0x2, 0x2, 0x76c, 0x76d, + 0x9, 0xd, 0x2, 0x2, 0x76d, 0x18c, 0x3, 0x2, 0x2, 0x2, 0x76e, 0x76f, + 0x9, 0xe, 0x2, 0x2, 0x76f, 0x18e, 0x3, 0x2, 0x2, 0x2, 0x770, 0x771, + 0x9, 0xf, 0x2, 0x2, 0x771, 0x190, 0x3, 0x2, 0x2, 0x2, 0x772, 0x773, + 0x9, 0x10, 0x2, 0x2, 0x773, 0x192, 0x3, 0x2, 0x2, 0x2, 0x774, 0x775, + 0x9, 0x11, 0x2, 0x2, 0x775, 0x194, 0x3, 0x2, 0x2, 0x2, 0x776, 0x777, + 0x9, 0x12, 0x2, 0x2, 0x777, 0x196, 0x3, 0x2, 0x2, 0x2, 0x778, 0x779, + 0x9, 0x13, 0x2, 0x2, 0x779, 0x198, 0x3, 0x2, 0x2, 0x2, 0x77a, 0x77b, + 0x9, 0x14, 0x2, 0x2, 0x77b, 0x19a, 0x3, 0x2, 0x2, 0x2, 0x77c, 0x77d, + 0x9, 0x15, 0x2, 0x2, 0x77d, 0x19c, 0x3, 0x2, 0x2, 0x2, 0x77e, 0x77f, + 0x9, 0x16, 0x2, 0x2, 0x77f, 0x19e, 0x3, 0x2, 0x2, 0x2, 0x780, 0x781, + 0x9, 0x17, 0x2, 0x2, 0x781, 0x1a0, 0x3, 0x2, 0x2, 0x2, 0x782, 0x783, + 0x9, 0x18, 0x2, 0x2, 0x783, 0x1a2, 0x3, 0x2, 0x2, 0x2, 0x784, 0x785, + 0x9, 0x19, 0x2, 0x2, 0x785, 0x1a4, 0x3, 0x2, 0x2, 0x2, 0x786, 0x787, + 0x9, 0x1a, 0x2, 0x2, 0x787, 0x1a6, 0x3, 0x2, 0x2, 0x2, 0x788, 0x789, + 0x9, 0x1b, 0x2, 0x2, 0x789, 0x1a8, 0x3, 0x2, 0x2, 0x2, 0x78a, 0x78b, + 0x9, 0x1c, 0x2, 0x2, 0x78b, 0x1aa, 0x3, 0x2, 0x2, 0x2, 0x78c, 0x78d, + 0x9, 0x1d, 0x2, 0x2, 0x78d, 0x1ac, 0x3, 0x2, 0x2, 0x2, 0x78e, 0x78f, + 0x9, 0x1e, 0x2, 0x2, 0x78f, 0x1ae, 0x3, 0x2, 0x2, 0x2, 0x790, 0x791, + 0x9, 0x1f, 0x2, 0x2, 0x791, 0x1b0, 0x3, 0x2, 0x2, 0x2, 0x792, 0x793, + 0x9, 0x20, 0x2, 0x2, 0x793, 0x1b2, 0x3, 0x2, 0x2, 0x2, 0x794, 0x795, + 0x9, 0x21, 0x2, 0x2, 0x795, 0x1b4, 0x3, 0x2, 0x2, 0x2, 0x796, 0x797, + 0x9, 0x22, 0x2, 0x2, 0x797, 0x1b6, 0x3, 0x2, 0x2, 0x2, 0x798, 0x799, + 0x7, 0x2f, 0x2, 0x2, 0x799, 0x79a, 0x7, 0x40, 0x2, 0x2, 0x79a, 0x1b8, + 0x3, 0x2, 0x2, 0x2, 0x79b, 0x79c, 0x7, 0x2c, 0x2, 0x2, 0x79c, 0x1ba, + 0x3, 0x2, 0x2, 0x2, 0x79d, 0x79e, 0x7, 0x62, 0x2, 0x2, 0x79e, 0x1bc, + 0x3, 0x2, 0x2, 0x2, 0x79f, 0x7a0, 0x7, 0x5e, 0x2, 0x2, 0x7a0, 0x1be, + 0x3, 0x2, 0x2, 0x2, 0x7a1, 0x7a2, 0x7, 0x3c, 0x2, 0x2, 0x7a2, 0x1c0, + 0x3, 0x2, 0x2, 0x2, 0x7a3, 0x7a4, 0x7, 0x2e, 0x2, 0x2, 0x7a4, 0x1c2, + 0x3, 0x2, 0x2, 0x2, 0x7a5, 0x7a6, 0x7, 0x7e, 0x2, 0x2, 0x7a6, 0x7a7, + 0x7, 0x7e, 0x2, 0x2, 0x7a7, 0x1c4, 0x3, 0x2, 0x2, 0x2, 0x7a8, 0x7a9, + 0x7, 0x2f, 0x2, 0x2, 0x7a9, 0x1c6, 0x3, 0x2, 0x2, 0x2, 0x7aa, 0x7ab, + 0x7, 0x30, 0x2, 0x2, 0x7ab, 0x1c8, 0x3, 0x2, 0x2, 0x2, 0x7ac, 0x7ad, + 0x7, 0x3f, 0x2, 0x2, 0x7ad, 0x7ae, 0x7, 0x3f, 0x2, 0x2, 0x7ae, 0x1ca, + 0x3, 0x2, 0x2, 0x2, 0x7af, 0x7b0, 0x7, 0x3f, 0x2, 0x2, 0x7b0, 0x1cc, + 0x3, 0x2, 0x2, 0x2, 0x7b1, 0x7b2, 0x7, 0x40, 0x2, 0x2, 0x7b2, 0x7b3, + 0x7, 0x3f, 0x2, 0x2, 0x7b3, 0x1ce, 0x3, 0x2, 0x2, 0x2, 0x7b4, 0x7b5, + 0x7, 0x40, 0x2, 0x2, 0x7b5, 0x1d0, 0x3, 0x2, 0x2, 0x2, 0x7b6, 0x7b7, + 0x7, 0x7d, 0x2, 0x2, 0x7b7, 0x1d2, 0x3, 0x2, 0x2, 0x2, 0x7b8, 0x7b9, + 0x7, 0x5d, 0x2, 0x2, 0x7b9, 0x1d4, 0x3, 0x2, 0x2, 0x2, 0x7ba, 0x7bb, + 0x7, 0x3e, 0x2, 0x2, 0x7bb, 0x7bc, 0x7, 0x3f, 0x2, 0x2, 0x7bc, 0x1d6, + 0x3, 0x2, 0x2, 0x2, 0x7bd, 0x7be, 0x7, 0x2a, 0x2, 0x2, 0x7be, 0x1d8, + 0x3, 0x2, 0x2, 0x2, 0x7bf, 0x7c0, 0x7, 0x3e, 0x2, 0x2, 0x7c0, 0x1da, + 0x3, 0x2, 0x2, 0x2, 0x7c1, 0x7c2, 0x7, 0x23, 0x2, 0x2, 0x7c2, 0x7c6, + 0x7, 0x3f, 0x2, 0x2, 0x7c3, 0x7c4, 0x7, 0x3e, 0x2, 0x2, 0x7c4, 0x7c6, + 0x7, 0x40, 0x2, 0x2, 0x7c5, 0x7c1, 0x3, 0x2, 0x2, 0x2, 0x7c5, 0x7c3, + 0x3, 0x2, 0x2, 0x2, 0x7c6, 0x1dc, 0x3, 0x2, 0x2, 0x2, 0x7c7, 0x7c8, + 0x7, 0x27, 0x2, 0x2, 0x7c8, 0x1de, 0x3, 0x2, 0x2, 0x2, 0x7c9, 0x7ca, + 0x7, 0x2d, 0x2, 0x2, 0x7ca, 0x1e0, 0x3, 0x2, 0x2, 0x2, 0x7cb, 0x7cc, + 0x7, 0x41, 0x2, 0x2, 0x7cc, 0x1e2, 0x3, 0x2, 0x2, 0x2, 0x7cd, 0x7ce, + 0x7, 0x24, 0x2, 0x2, 0x7ce, 0x1e4, 0x3, 0x2, 0x2, 0x2, 0x7cf, 0x7d0, + 0x7, 0x29, 0x2, 0x2, 0x7d0, 0x1e6, 0x3, 0x2, 0x2, 0x2, 0x7d1, 0x7d2, + 0x7, 0x7f, 0x2, 0x2, 0x7d2, 0x1e8, 0x3, 0x2, 0x2, 0x2, 0x7d3, 0x7d4, + 0x7, 0x5f, 0x2, 0x2, 0x7d4, 0x1ea, 0x3, 0x2, 0x2, 0x2, 0x7d5, 0x7d6, + 0x7, 0x2b, 0x2, 0x2, 0x7d6, 0x1ec, 0x3, 0x2, 0x2, 0x2, 0x7d7, 0x7d8, + 0x7, 0x3d, 0x2, 0x2, 0x7d8, 0x1ee, 0x3, 0x2, 0x2, 0x2, 0x7d9, 0x7da, + 0x7, 0x31, 0x2, 0x2, 0x7da, 0x1f0, 0x3, 0x2, 0x2, 0x2, 0x7db, 0x7dc, + 0x7, 0x61, 0x2, 0x2, 0x7dc, 0x1f2, 0x3, 0x2, 0x2, 0x2, 0x7dd, 0x7de, + 0x7, 0x31, 0x2, 0x2, 0x7de, 0x7df, 0x7, 0x2c, 0x2, 0x2, 0x7df, 0x7e3, + 0x3, 0x2, 0x2, 0x2, 0x7e0, 0x7e2, 0xb, 0x2, 0x2, 0x2, 0x7e1, 0x7e0, + 0x3, 0x2, 0x2, 0x2, 0x7e2, 0x7e5, 0x3, 0x2, 0x2, 0x2, 0x7e3, 0x7e4, + 0x3, 0x2, 0x2, 0x2, 0x7e3, 0x7e1, 0x3, 0x2, 0x2, 0x2, 0x7e4, 0x7e6, + 0x3, 0x2, 0x2, 0x2, 0x7e5, 0x7e3, 0x3, 0x2, 0x2, 0x2, 0x7e6, 0x7e7, + 0x7, 0x2c, 0x2, 0x2, 0x7e7, 0x7e8, 0x7, 0x31, 0x2, 0x2, 0x7e8, 0x7e9, + 0x3, 0x2, 0x2, 0x2, 0x7e9, 0x7ea, 0x8, 0xfa, 0x2, 0x2, 0x7ea, 0x1f4, + 0x3, 0x2, 0x2, 0x2, 0x7eb, 0x7ec, 0x7, 0x2f, 0x2, 0x2, 0x7ec, 0x7ed, + 0x7, 0x2f, 0x2, 0x2, 0x7ed, 0x7f1, 0x3, 0x2, 0x2, 0x2, 0x7ee, 0x7f0, + 0xa, 0x23, 0x2, 0x2, 0x7ef, 0x7ee, 0x3, 0x2, 0x2, 0x2, 0x7f0, 0x7f3, + 0x3, 0x2, 0x2, 0x2, 0x7f1, 0x7ef, 0x3, 0x2, 0x2, 0x2, 0x7f1, 0x7f2, + 0x3, 0x2, 0x2, 0x2, 0x7f2, 0x7f5, 0x3, 0x2, 0x2, 0x2, 0x7f3, 0x7f1, + 0x3, 0x2, 0x2, 0x2, 0x7f4, 0x7f6, 0x9, 0x24, 0x2, 0x2, 0x7f5, 0x7f4, + 0x3, 0x2, 0x2, 0x2, 0x7f6, 0x7f7, 0x3, 0x2, 0x2, 0x2, 0x7f7, 0x7f8, + 0x8, 0xfb, 0x2, 0x2, 0x7f8, 0x1f6, 0x3, 0x2, 0x2, 0x2, 0x7f9, 0x7fa, + 0x9, 0x25, 0x2, 0x2, 0x7fa, 0x7fb, 0x3, 0x2, 0x2, 0x2, 0x7fb, 0x7fc, + 0x8, 0xfc, 0x2, 0x2, 0x7fc, 0x1f8, 0x3, 0x2, 0x2, 0x2, 0x26, 0x2, 0x237, + 0x413, 0x6b4, 0x6c3, 0x6c8, 0x6ca, 0x6d5, 0x6d7, 0x6e4, 0x6e6, 0x6eb, + 0x6f2, 0x6f7, 0x6fb, 0x700, 0x705, 0x709, 0x70e, 0x715, 0x71b, 0x720, + 0x727, 0x72c, 0x732, 0x737, 0x739, 0x73f, 0x744, 0x74b, 0x755, 0x757, + 0x7c5, 0x7e3, 0x7f1, 0x7f5, 0x3, 0x8, 0x2, 0x2, }; atn::ATNDeserializer deserializer; diff --git a/src/Parsers/New/ClickHouseLexer.g4 b/src/Parsers/New/ClickHouseLexer.g4 index 508bc1d33ff..343b73ff86b 100644 --- a/src/Parsers/New/ClickHouseLexer.g4 +++ b/src/Parsers/New/ClickHouseLexer.g4 @@ -102,6 +102,7 @@ LIMIT: L I M I T; LIVE: L I V E; LOCAL: L O C A L; LOGS: L O G S; +MATERIALIZE: M A T E R I A L I Z E; MATERIALIZED: M A T E R I A L I Z E D; MAX: M A X; MERGES: M E R G E S; @@ -127,6 +128,7 @@ PARTITION: P A R T I T I O N; POPULATE: P O P U L A T E; PREWHERE: P R E W H E R E; PRIMARY: P R I M A R Y; +PROJECTION: P R O J E C T I O N; QUARTER: Q U A R T E R; RANGE: R A N G E; RELOAD: R E L O A D; diff --git a/src/Parsers/New/ClickHouseLexer.h b/src/Parsers/New/ClickHouseLexer.h index 62de0792f3c..4de0a30ba2c 100644 --- a/src/Parsers/New/ClickHouseLexer.h +++ b/src/Parsers/New/ClickHouseLexer.h @@ -13,51 +13,51 @@ namespace DB { class ClickHouseLexer : public antlr4::Lexer { public: enum { - ADD = 1, AFTER = 2, ALIAS = 3, ALL = 4, ALTER = 5, AND = 6, ANTI = 7, - ANY = 8, ARRAY = 9, AS = 10, ASCENDING = 11, ASOF = 12, ASYNC = 13, - ATTACH = 14, BETWEEN = 15, BOTH = 16, BY = 17, CASE = 18, CAST = 19, - CHECK = 20, CLEAR = 21, CLUSTER = 22, CODEC = 23, COLLATE = 24, COLUMN = 25, - COMMENT = 26, CONSTRAINT = 27, CREATE = 28, CROSS = 29, CUBE = 30, DATABASE = 31, - DATABASES = 32, DATE = 33, DAY = 34, DEDUPLICATE = 35, DEFAULT = 36, - DELAY = 37, DELETE = 38, DESC = 39, DESCENDING = 40, DESCRIBE = 41, - DETACH = 42, DICTIONARIES = 43, DICTIONARY = 44, DISK = 45, DISTINCT = 46, - DISTRIBUTED = 47, DROP = 48, ELSE = 49, END = 50, ENGINE = 51, EVENTS = 52, - EXISTS = 53, EXPLAIN = 54, EXPRESSION = 55, EXTRACT = 56, FETCHES = 57, - FINAL = 58, FIRST = 59, FLUSH = 60, FOR = 61, FORMAT = 62, FREEZE = 63, - FROM = 64, FULL = 65, FUNCTION = 66, GLOBAL = 67, GRANULARITY = 68, - GROUP = 69, HAVING = 70, HIERARCHICAL = 71, HOUR = 72, ID = 73, IF = 74, - ILIKE = 75, IN = 76, INDEX = 77, INF = 78, INJECTIVE = 79, INNER = 80, - INSERT = 81, INTERVAL = 82, INTO = 83, IS = 84, IS_OBJECT_ID = 85, JOIN = 86, - KEY = 87, KILL = 88, LAST = 89, LAYOUT = 90, LEADING = 91, LEFT = 92, - LIFETIME = 93, LIKE = 94, LIMIT = 95, LIVE = 96, LOCAL = 97, LOGS = 98, - MATERIALIZED = 99, MAX = 100, MERGES = 101, MIN = 102, MINUTE = 103, - MODIFY = 104, MONTH = 105, MOVE = 106, MUTATION = 107, NAN_SQL = 108, - NO = 109, NOT = 110, NULL_SQL = 111, NULLS = 112, OFFSET = 113, ON = 114, - OPTIMIZE = 115, OR = 116, ORDER = 117, OUTER = 118, OUTFILE = 119, PARTITION = 120, - POPULATE = 121, PREWHERE = 122, PRIMARY = 123, QUARTER = 124, RANGE = 125, - RELOAD = 126, REMOVE = 127, RENAME = 128, REPLACE = 129, REPLICA = 130, - REPLICATED = 131, RIGHT = 132, ROLLUP = 133, SAMPLE = 134, SECOND = 135, - SELECT = 136, SEMI = 137, SENDS = 138, SET = 139, SETTINGS = 140, SHOW = 141, - SOURCE = 142, START = 143, STOP = 144, SUBSTRING = 145, SYNC = 146, - SYNTAX = 147, SYSTEM = 148, TABLE = 149, TABLES = 150, TEMPORARY = 151, - TEST = 152, THEN = 153, TIES = 154, TIMEOUT = 155, TIMESTAMP = 156, - TO = 157, TOP = 158, TOTALS = 159, TRAILING = 160, TRIM = 161, TRUNCATE = 162, - TTL = 163, TYPE = 164, UNION = 165, UPDATE = 166, USE = 167, USING = 168, - UUID = 169, VALUES = 170, VIEW = 171, VOLUME = 172, WATCH = 173, WEEK = 174, - WHEN = 175, WHERE = 176, WITH = 177, YEAR = 178, JSON_FALSE = 179, JSON_TRUE = 180, - IDENTIFIER = 181, FLOATING_LITERAL = 182, OCTAL_LITERAL = 183, DECIMAL_LITERAL = 184, - HEXADECIMAL_LITERAL = 185, STRING_LITERAL = 186, ARROW = 187, ASTERISK = 188, - BACKQUOTE = 189, BACKSLASH = 190, COLON = 191, COMMA = 192, CONCAT = 193, - DASH = 194, DOT = 195, EQ_DOUBLE = 196, EQ_SINGLE = 197, GE = 198, GT = 199, - LBRACE = 200, LBRACKET = 201, LE = 202, LPAREN = 203, LT = 204, NOT_EQ = 205, - PERCENT = 206, PLUS = 207, QUERY = 208, QUOTE_DOUBLE = 209, QUOTE_SINGLE = 210, - RBRACE = 211, RBRACKET = 212, RPAREN = 213, SEMICOLON = 214, SLASH = 215, - UNDERSCORE = 216, MULTI_LINE_COMMENT = 217, SINGLE_LINE_COMMENT = 218, - WHITESPACE = 219 + ADD = 1, AFTER = 2, ALIAS = 3, ALL = 4, ALTER = 5, AND = 6, ANTI = 7, + ANY = 8, ARRAY = 9, AS = 10, ASCENDING = 11, ASOF = 12, ASYNC = 13, + ATTACH = 14, BETWEEN = 15, BOTH = 16, BY = 17, CASE = 18, CAST = 19, + CHECK = 20, CLEAR = 21, CLUSTER = 22, CODEC = 23, COLLATE = 24, COLUMN = 25, + COMMENT = 26, CONSTRAINT = 27, CREATE = 28, CROSS = 29, CUBE = 30, DATABASE = 31, + DATABASES = 32, DATE = 33, DAY = 34, DEDUPLICATE = 35, DEFAULT = 36, + DELAY = 37, DELETE = 38, DESC = 39, DESCENDING = 40, DESCRIBE = 41, + DETACH = 42, DICTIONARIES = 43, DICTIONARY = 44, DISK = 45, DISTINCT = 46, + DISTRIBUTED = 47, DROP = 48, ELSE = 49, END = 50, ENGINE = 51, EVENTS = 52, + EXISTS = 53, EXPLAIN = 54, EXPRESSION = 55, EXTRACT = 56, FETCHES = 57, + FINAL = 58, FIRST = 59, FLUSH = 60, FOR = 61, FORMAT = 62, FREEZE = 63, + FROM = 64, FULL = 65, FUNCTION = 66, GLOBAL = 67, GRANULARITY = 68, + GROUP = 69, HAVING = 70, HIERARCHICAL = 71, HOUR = 72, ID = 73, IF = 74, + ILIKE = 75, IN = 76, INDEX = 77, INF = 78, INJECTIVE = 79, INNER = 80, + INSERT = 81, INTERVAL = 82, INTO = 83, IS = 84, IS_OBJECT_ID = 85, JOIN = 86, + KEY = 87, KILL = 88, LAST = 89, LAYOUT = 90, LEADING = 91, LEFT = 92, + LIFETIME = 93, LIKE = 94, LIMIT = 95, LIVE = 96, LOCAL = 97, LOGS = 98, + MATERIALIZED = 99, MATERIALIZE = 100, MAX = 101, MERGES = 102, MIN = 103, + MINUTE = 104, MODIFY = 105, MONTH = 106, MOVE = 107, MUTATION = 108, + NAN_SQL = 109, NO = 110, NOT = 111, NULL_SQL = 112, NULLS = 113, OFFSET = 114, + ON = 115, OPTIMIZE = 116, OR = 117, ORDER = 118, OUTER = 119, OUTFILE = 120, + PARTITION = 121, POPULATE = 122, PREWHERE = 123, PRIMARY = 124, PROJECTION = 125, + QUARTER = 126, RANGE = 127, RELOAD = 128, REMOVE = 129, RENAME = 130, + REPLACE = 131, REPLICA = 132, REPLICATED = 133, RIGHT = 134, ROLLUP = 135, + SAMPLE = 136, SECOND = 137, SELECT = 138, SEMI = 139, SENDS = 140, SET = 141, + SETTINGS = 142, SHOW = 143, SOURCE = 144, START = 145, STOP = 146, SUBSTRING = 147, + SYNC = 148, SYNTAX = 149, SYSTEM = 150, TABLE = 151, TABLES = 152, TEMPORARY = 153, + TEST = 154, THEN = 155, TIES = 156, TIMEOUT = 157, TIMESTAMP = 158, + TO = 159, TOP = 160, TOTALS = 161, TRAILING = 162, TRIM = 163, TRUNCATE = 164, + TTL = 165, TYPE = 166, UNION = 167, UPDATE = 168, USE = 169, USING = 170, + UUID = 171, VALUES = 172, VIEW = 173, VOLUME = 174, WATCH = 175, WEEK = 176, + WHEN = 177, WHERE = 178, WITH = 179, YEAR = 180, JSON_FALSE = 181, JSON_TRUE = 182, + IDENTIFIER = 183, FLOATING_LITERAL = 184, OCTAL_LITERAL = 185, DECIMAL_LITERAL = 186, + HEXADECIMAL_LITERAL = 187, STRING_LITERAL = 188, ARROW = 189, ASTERISK = 190, + BACKQUOTE = 191, BACKSLASH = 192, COLON = 193, COMMA = 194, CONCAT = 195, + DASH = 196, DOT = 197, EQ_DOUBLE = 198, EQ_SINGLE = 199, GE = 200, GT = 201, + LBRACE = 202, LBRACKET = 203, LE = 204, LPAREN = 205, LT = 206, NOT_EQ = 207, + PERCENT = 208, PLUS = 209, QUERY = 210, QUOTE_DOUBLE = 211, QUOTE_SINGLE = 212, + RBRACE = 213, RBRACKET = 214, RPAREN = 215, SEMICOLON = 216, SLASH = 217, + UNDERSCORE = 218, MULTI_LINE_COMMENT = 219, SINGLE_LINE_COMMENT = 220, + WHITESPACE = 221 }; ClickHouseLexer(antlr4::CharStream *input); - ~ClickHouseLexer() override; + ~ClickHouseLexer(); virtual std::string getGrammarFileName() const override; virtual const std::vector& getRuleNames() const override; diff --git a/src/Parsers/New/ClickHouseParser.cpp b/src/Parsers/New/ClickHouseParser.cpp index 91cf1e47f96..d3ba8563a6f 100644 --- a/src/Parsers/New/ClickHouseParser.cpp +++ b/src/Parsers/New/ClickHouseParser.cpp @@ -92,7 +92,7 @@ ClickHouseParser::QueryStmtContext* ClickHouseParser::queryStmt() { exitRule(); }); try { - setState(226); + setState(232); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::ALTER: @@ -118,36 +118,36 @@ ClickHouseParser::QueryStmtContext* ClickHouseParser::queryStmt() { case ClickHouseParser::WITH: case ClickHouseParser::LPAREN: { enterOuterAlt(_localctx, 1); - setState(212); + setState(218); query(); - setState(216); + setState(222); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::INTO) { - setState(213); + setState(219); match(ClickHouseParser::INTO); - setState(214); + setState(220); match(ClickHouseParser::OUTFILE); - setState(215); + setState(221); match(ClickHouseParser::STRING_LITERAL); } - setState(220); + setState(226); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::FORMAT) { - setState(218); + setState(224); match(ClickHouseParser::FORMAT); - setState(219); + setState(225); identifierOrNull(); } - setState(223); + setState(229); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::SEMICOLON) { - setState(222); + setState(228); match(ClickHouseParser::SEMICOLON); } break; @@ -155,7 +155,7 @@ ClickHouseParser::QueryStmtContext* ClickHouseParser::queryStmt() { case ClickHouseParser::INSERT: { enterOuterAlt(_localctx, 2); - setState(225); + setState(231); insertStmt(); break; } @@ -273,131 +273,131 @@ ClickHouseParser::QueryContext* ClickHouseParser::query() { exitRule(); }); try { - setState(246); + setState(252); _errHandler->sync(this); switch (getInterpreter()->adaptivePredict(_input, 4, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(228); + setState(234); alterStmt(); break; } case 2: { enterOuterAlt(_localctx, 2); - setState(229); + setState(235); attachStmt(); break; } case 3: { enterOuterAlt(_localctx, 3); - setState(230); + setState(236); checkStmt(); break; } case 4: { enterOuterAlt(_localctx, 4); - setState(231); + setState(237); createStmt(); break; } case 5: { enterOuterAlt(_localctx, 5); - setState(232); + setState(238); describeStmt(); break; } case 6: { enterOuterAlt(_localctx, 6); - setState(233); + setState(239); dropStmt(); break; } case 7: { enterOuterAlt(_localctx, 7); - setState(234); + setState(240); existsStmt(); break; } case 8: { enterOuterAlt(_localctx, 8); - setState(235); + setState(241); explainStmt(); break; } case 9: { enterOuterAlt(_localctx, 9); - setState(236); + setState(242); killStmt(); break; } case 10: { enterOuterAlt(_localctx, 10); - setState(237); + setState(243); optimizeStmt(); break; } case 11: { enterOuterAlt(_localctx, 11); - setState(238); + setState(244); renameStmt(); break; } case 12: { enterOuterAlt(_localctx, 12); - setState(239); + setState(245); selectUnionStmt(); break; } case 13: { enterOuterAlt(_localctx, 13); - setState(240); + setState(246); setStmt(); break; } case 14: { enterOuterAlt(_localctx, 14); - setState(241); + setState(247); showStmt(); break; } case 15: { enterOuterAlt(_localctx, 15); - setState(242); + setState(248); systemStmt(); break; } case 16: { enterOuterAlt(_localctx, 16); - setState(243); + setState(249); truncateStmt(); break; } case 17: { enterOuterAlt(_localctx, 17); - setState(244); + setState(250); useStmt(); break; } case 18: { enterOuterAlt(_localctx, 18); - setState(245); + setState(251); watchStmt(); break; } @@ -483,31 +483,31 @@ ClickHouseParser::AlterStmtContext* ClickHouseParser::alterStmt() { try { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(248); + setState(254); match(ClickHouseParser::ALTER); - setState(249); + setState(255); match(ClickHouseParser::TABLE); - setState(250); + setState(256); tableIdentifier(); - setState(252); + setState(258); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(251); + setState(257); clusterClause(); } - setState(254); + setState(260); alterTableClause(); - setState(259); + setState(265); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(255); - match(ClickHouseParser::COMMA); - setState(256); - alterTableClause(); setState(261); + match(ClickHouseParser::COMMA); + setState(262); + alterTableClause(); + setState(267); _errHandler->sync(this); _la = _input->LA(1); } @@ -564,95 +564,6 @@ antlrcpp::Any ClickHouseParser::AlterTableClauseReplaceContext::accept(tree::Par else return visitor->visitChildren(this); } -//----------------- AlterTableClauseRenameContext ------------------------------------------------------------------ - -tree::TerminalNode* ClickHouseParser::AlterTableClauseRenameContext::RENAME() { - return getToken(ClickHouseParser::RENAME, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseRenameContext::COLUMN() { - return getToken(ClickHouseParser::COLUMN, 0); -} - -std::vector ClickHouseParser::AlterTableClauseRenameContext::nestedIdentifier() { - return getRuleContexts(); -} - -ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseRenameContext::nestedIdentifier(size_t i) { - return getRuleContext(i); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseRenameContext::TO() { - return getToken(ClickHouseParser::TO, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseRenameContext::IF() { - return getToken(ClickHouseParser::IF, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseRenameContext::EXISTS() { - return getToken(ClickHouseParser::EXISTS, 0); -} - -ClickHouseParser::AlterTableClauseRenameContext::AlterTableClauseRenameContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } - - -antlrcpp::Any ClickHouseParser::AlterTableClauseRenameContext::accept(tree::ParseTreeVisitor *visitor) { - if (auto parserVisitor = dynamic_cast(visitor)) - return parserVisitor->visitAlterTableClauseRename(this); - else - return visitor->visitChildren(this); -} -//----------------- AlterTableClauseFreezePartitionContext ------------------------------------------------------------------ - -tree::TerminalNode* ClickHouseParser::AlterTableClauseFreezePartitionContext::FREEZE() { - return getToken(ClickHouseParser::FREEZE, 0); -} - -ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseFreezePartitionContext::partitionClause() { - return getRuleContext(0); -} - -ClickHouseParser::AlterTableClauseFreezePartitionContext::AlterTableClauseFreezePartitionContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } - - -antlrcpp::Any ClickHouseParser::AlterTableClauseFreezePartitionContext::accept(tree::ParseTreeVisitor *visitor) { - if (auto parserVisitor = dynamic_cast(visitor)) - return parserVisitor->visitAlterTableClauseFreezePartition(this); - else - return visitor->visitChildren(this); -} -//----------------- AlterTableClauseModifyContext ------------------------------------------------------------------ - -tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyContext::MODIFY() { - return getToken(ClickHouseParser::MODIFY, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyContext::COLUMN() { - return getToken(ClickHouseParser::COLUMN, 0); -} - -ClickHouseParser::TableColumnDfntContext* ClickHouseParser::AlterTableClauseModifyContext::tableColumnDfnt() { - return getRuleContext(0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyContext::IF() { - return getToken(ClickHouseParser::IF, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyContext::EXISTS() { - return getToken(ClickHouseParser::EXISTS, 0); -} - -ClickHouseParser::AlterTableClauseModifyContext::AlterTableClauseModifyContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } - - -antlrcpp::Any ClickHouseParser::AlterTableClauseModifyContext::accept(tree::ParseTreeVisitor *visitor) { - if (auto parserVisitor = dynamic_cast(visitor)) - return parserVisitor->visitAlterTableClauseModify(this); - else - return visitor->visitChildren(this); -} //----------------- AlterTableClauseModifyOrderByContext ------------------------------------------------------------------ tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyOrderByContext::MODIFY() { @@ -680,25 +591,6 @@ antlrcpp::Any ClickHouseParser::AlterTableClauseModifyOrderByContext::accept(tre else return visitor->visitChildren(this); } -//----------------- AlterTableClauseRemoveTTLContext ------------------------------------------------------------------ - -tree::TerminalNode* ClickHouseParser::AlterTableClauseRemoveTTLContext::REMOVE() { - return getToken(ClickHouseParser::REMOVE, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseRemoveTTLContext::TTL() { - return getToken(ClickHouseParser::TTL, 0); -} - -ClickHouseParser::AlterTableClauseRemoveTTLContext::AlterTableClauseRemoveTTLContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } - - -antlrcpp::Any ClickHouseParser::AlterTableClauseRemoveTTLContext::accept(tree::ParseTreeVisitor *visitor) { - if (auto parserVisitor = dynamic_cast(visitor)) - return parserVisitor->visitAlterTableClauseRemoveTTL(this); - else - return visitor->visitChildren(this); -} //----------------- AlterTableClauseUpdateContext ------------------------------------------------------------------ tree::TerminalNode* ClickHouseParser::AlterTableClauseUpdateContext::UPDATE() { @@ -722,6 +614,45 @@ antlrcpp::Any ClickHouseParser::AlterTableClauseUpdateContext::accept(tree::Pars else return visitor->visitChildren(this); } +//----------------- AlterTableClauseClearProjectionContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearProjectionContext::CLEAR() { + return getToken(ClickHouseParser::CLEAR, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearProjectionContext::PROJECTION() { + return getToken(ClickHouseParser::PROJECTION, 0); +} + +ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseClearProjectionContext::nestedIdentifier() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearProjectionContext::IF() { + return getToken(ClickHouseParser::IF, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearProjectionContext::EXISTS() { + return getToken(ClickHouseParser::EXISTS, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearProjectionContext::IN() { + return getToken(ClickHouseParser::IN, 0); +} + +ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseClearProjectionContext::partitionClause() { + return getRuleContext(0); +} + +ClickHouseParser::AlterTableClauseClearProjectionContext::AlterTableClauseClearProjectionContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseClearProjectionContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseClearProjection(this); + else + return visitor->visitChildren(this); +} //----------------- AlterTableClauseModifyRemoveContext ------------------------------------------------------------------ tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyRemoveContext::MODIFY() { @@ -784,41 +715,6 @@ antlrcpp::Any ClickHouseParser::AlterTableClauseDeleteContext::accept(tree::Pars else return visitor->visitChildren(this); } -//----------------- AlterTableClauseModifyCodecContext ------------------------------------------------------------------ - -tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyCodecContext::MODIFY() { - return getToken(ClickHouseParser::MODIFY, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyCodecContext::COLUMN() { - return getToken(ClickHouseParser::COLUMN, 0); -} - -ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseModifyCodecContext::nestedIdentifier() { - return getRuleContext(0); -} - -ClickHouseParser::CodecExprContext* ClickHouseParser::AlterTableClauseModifyCodecContext::codecExpr() { - return getRuleContext(0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyCodecContext::IF() { - return getToken(ClickHouseParser::IF, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyCodecContext::EXISTS() { - return getToken(ClickHouseParser::EXISTS, 0); -} - -ClickHouseParser::AlterTableClauseModifyCodecContext::AlterTableClauseModifyCodecContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } - - -antlrcpp::Any ClickHouseParser::AlterTableClauseModifyCodecContext::accept(tree::ParseTreeVisitor *visitor) { - if (auto parserVisitor = dynamic_cast(visitor)) - return parserVisitor->visitAlterTableClauseModifyCodec(this); - else - return visitor->visitChildren(this); -} //----------------- AlterTableClauseCommentContext ------------------------------------------------------------------ tree::TerminalNode* ClickHouseParser::AlterTableClauseCommentContext::COMMENT() { @@ -854,33 +750,6 @@ antlrcpp::Any ClickHouseParser::AlterTableClauseCommentContext::accept(tree::Par else return visitor->visitChildren(this); } -//----------------- AlterTableClauseAttachContext ------------------------------------------------------------------ - -tree::TerminalNode* ClickHouseParser::AlterTableClauseAttachContext::ATTACH() { - return getToken(ClickHouseParser::ATTACH, 0); -} - -ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseAttachContext::partitionClause() { - return getRuleContext(0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseAttachContext::FROM() { - return getToken(ClickHouseParser::FROM, 0); -} - -ClickHouseParser::TableIdentifierContext* ClickHouseParser::AlterTableClauseAttachContext::tableIdentifier() { - return getRuleContext(0); -} - -ClickHouseParser::AlterTableClauseAttachContext::AlterTableClauseAttachContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } - - -antlrcpp::Any ClickHouseParser::AlterTableClauseAttachContext::accept(tree::ParseTreeVisitor *visitor) { - if (auto parserVisitor = dynamic_cast(visitor)) - return parserVisitor->visitAlterTableClauseAttach(this); - else - return visitor->visitChildren(this); -} //----------------- AlterTableClauseDropColumnContext ------------------------------------------------------------------ tree::TerminalNode* ClickHouseParser::AlterTableClauseDropColumnContext::DROP() { @@ -912,45 +781,6 @@ antlrcpp::Any ClickHouseParser::AlterTableClauseDropColumnContext::accept(tree:: else return visitor->visitChildren(this); } -//----------------- AlterTableClauseClearContext ------------------------------------------------------------------ - -tree::TerminalNode* ClickHouseParser::AlterTableClauseClearContext::CLEAR() { - return getToken(ClickHouseParser::CLEAR, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseClearContext::COLUMN() { - return getToken(ClickHouseParser::COLUMN, 0); -} - -ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseClearContext::nestedIdentifier() { - return getRuleContext(0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseClearContext::IF() { - return getToken(ClickHouseParser::IF, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseClearContext::EXISTS() { - return getToken(ClickHouseParser::EXISTS, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseClearContext::IN() { - return getToken(ClickHouseParser::IN, 0); -} - -ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseClearContext::partitionClause() { - return getRuleContext(0); -} - -ClickHouseParser::AlterTableClauseClearContext::AlterTableClauseClearContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } - - -antlrcpp::Any ClickHouseParser::AlterTableClauseClearContext::accept(tree::ParseTreeVisitor *visitor) { - if (auto parserVisitor = dynamic_cast(visitor)) - return parserVisitor->visitAlterTableClauseClear(this); - else - return visitor->visitChildren(this); -} //----------------- AlterTableClauseDetachContext ------------------------------------------------------------------ tree::TerminalNode* ClickHouseParser::AlterTableClauseDetachContext::DETACH() { @@ -970,37 +800,6 @@ antlrcpp::Any ClickHouseParser::AlterTableClauseDetachContext::accept(tree::Pars else return visitor->visitChildren(this); } -//----------------- AlterTableClauseDropIndexContext ------------------------------------------------------------------ - -tree::TerminalNode* ClickHouseParser::AlterTableClauseDropIndexContext::DROP() { - return getToken(ClickHouseParser::DROP, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseDropIndexContext::INDEX() { - return getToken(ClickHouseParser::INDEX, 0); -} - -ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseDropIndexContext::nestedIdentifier() { - return getRuleContext(0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseDropIndexContext::IF() { - return getToken(ClickHouseParser::IF, 0); -} - -tree::TerminalNode* ClickHouseParser::AlterTableClauseDropIndexContext::EXISTS() { - return getToken(ClickHouseParser::EXISTS, 0); -} - -ClickHouseParser::AlterTableClauseDropIndexContext::AlterTableClauseDropIndexContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } - - -antlrcpp::Any ClickHouseParser::AlterTableClauseDropIndexContext::accept(tree::ParseTreeVisitor *visitor) { - if (auto parserVisitor = dynamic_cast(visitor)) - return parserVisitor->visitAlterTableClauseDropIndex(this); - else - return visitor->visitChildren(this); -} //----------------- AlterTableClauseAddIndexContext ------------------------------------------------------------------ tree::TerminalNode* ClickHouseParser::AlterTableClauseAddIndexContext::ADD() { @@ -1063,6 +862,437 @@ antlrcpp::Any ClickHouseParser::AlterTableClauseDropPartitionContext::accept(tre else return visitor->visitChildren(this); } +//----------------- AlterTableClauseMaterializeIndexContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMaterializeIndexContext::MATERIALIZE() { + return getToken(ClickHouseParser::MATERIALIZE, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMaterializeIndexContext::INDEX() { + return getToken(ClickHouseParser::INDEX, 0); +} + +ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseMaterializeIndexContext::nestedIdentifier() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMaterializeIndexContext::IF() { + return getToken(ClickHouseParser::IF, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMaterializeIndexContext::EXISTS() { + return getToken(ClickHouseParser::EXISTS, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMaterializeIndexContext::IN() { + return getToken(ClickHouseParser::IN, 0); +} + +ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseMaterializeIndexContext::partitionClause() { + return getRuleContext(0); +} + +ClickHouseParser::AlterTableClauseMaterializeIndexContext::AlterTableClauseMaterializeIndexContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseMaterializeIndexContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseMaterializeIndex(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseMaterializeProjectionContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMaterializeProjectionContext::MATERIALIZE() { + return getToken(ClickHouseParser::MATERIALIZE, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMaterializeProjectionContext::PROJECTION() { + return getToken(ClickHouseParser::PROJECTION, 0); +} + +ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseMaterializeProjectionContext::nestedIdentifier() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMaterializeProjectionContext::IF() { + return getToken(ClickHouseParser::IF, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMaterializeProjectionContext::EXISTS() { + return getToken(ClickHouseParser::EXISTS, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMaterializeProjectionContext::IN() { + return getToken(ClickHouseParser::IN, 0); +} + +ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseMaterializeProjectionContext::partitionClause() { + return getRuleContext(0); +} + +ClickHouseParser::AlterTableClauseMaterializeProjectionContext::AlterTableClauseMaterializeProjectionContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseMaterializeProjectionContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseMaterializeProjection(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseMovePartitionContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::MOVE() { + return getToken(ClickHouseParser::MOVE, 0); +} + +ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseMovePartitionContext::partitionClause() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::TO() { + return getToken(ClickHouseParser::TO, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::DISK() { + return getToken(ClickHouseParser::DISK, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::STRING_LITERAL() { + return getToken(ClickHouseParser::STRING_LITERAL, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::VOLUME() { + return getToken(ClickHouseParser::VOLUME, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::TABLE() { + return getToken(ClickHouseParser::TABLE, 0); +} + +ClickHouseParser::TableIdentifierContext* ClickHouseParser::AlterTableClauseMovePartitionContext::tableIdentifier() { + return getRuleContext(0); +} + +ClickHouseParser::AlterTableClauseMovePartitionContext::AlterTableClauseMovePartitionContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseMovePartitionContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseMovePartition(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseRenameContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseRenameContext::RENAME() { + return getToken(ClickHouseParser::RENAME, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseRenameContext::COLUMN() { + return getToken(ClickHouseParser::COLUMN, 0); +} + +std::vector ClickHouseParser::AlterTableClauseRenameContext::nestedIdentifier() { + return getRuleContexts(); +} + +ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseRenameContext::nestedIdentifier(size_t i) { + return getRuleContext(i); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseRenameContext::TO() { + return getToken(ClickHouseParser::TO, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseRenameContext::IF() { + return getToken(ClickHouseParser::IF, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseRenameContext::EXISTS() { + return getToken(ClickHouseParser::EXISTS, 0); +} + +ClickHouseParser::AlterTableClauseRenameContext::AlterTableClauseRenameContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseRenameContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseRename(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseFreezePartitionContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseFreezePartitionContext::FREEZE() { + return getToken(ClickHouseParser::FREEZE, 0); +} + +ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseFreezePartitionContext::partitionClause() { + return getRuleContext(0); +} + +ClickHouseParser::AlterTableClauseFreezePartitionContext::AlterTableClauseFreezePartitionContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseFreezePartitionContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseFreezePartition(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseClearColumnContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearColumnContext::CLEAR() { + return getToken(ClickHouseParser::CLEAR, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearColumnContext::COLUMN() { + return getToken(ClickHouseParser::COLUMN, 0); +} + +ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseClearColumnContext::nestedIdentifier() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearColumnContext::IF() { + return getToken(ClickHouseParser::IF, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearColumnContext::EXISTS() { + return getToken(ClickHouseParser::EXISTS, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearColumnContext::IN() { + return getToken(ClickHouseParser::IN, 0); +} + +ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseClearColumnContext::partitionClause() { + return getRuleContext(0); +} + +ClickHouseParser::AlterTableClauseClearColumnContext::AlterTableClauseClearColumnContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseClearColumnContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseClearColumn(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseModifyContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyContext::MODIFY() { + return getToken(ClickHouseParser::MODIFY, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyContext::COLUMN() { + return getToken(ClickHouseParser::COLUMN, 0); +} + +ClickHouseParser::TableColumnDfntContext* ClickHouseParser::AlterTableClauseModifyContext::tableColumnDfnt() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyContext::IF() { + return getToken(ClickHouseParser::IF, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyContext::EXISTS() { + return getToken(ClickHouseParser::EXISTS, 0); +} + +ClickHouseParser::AlterTableClauseModifyContext::AlterTableClauseModifyContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseModifyContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseModify(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseClearIndexContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearIndexContext::CLEAR() { + return getToken(ClickHouseParser::CLEAR, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearIndexContext::INDEX() { + return getToken(ClickHouseParser::INDEX, 0); +} + +ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseClearIndexContext::nestedIdentifier() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearIndexContext::IF() { + return getToken(ClickHouseParser::IF, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearIndexContext::EXISTS() { + return getToken(ClickHouseParser::EXISTS, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseClearIndexContext::IN() { + return getToken(ClickHouseParser::IN, 0); +} + +ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseClearIndexContext::partitionClause() { + return getRuleContext(0); +} + +ClickHouseParser::AlterTableClauseClearIndexContext::AlterTableClauseClearIndexContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseClearIndexContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseClearIndex(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseRemoveTTLContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseRemoveTTLContext::REMOVE() { + return getToken(ClickHouseParser::REMOVE, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseRemoveTTLContext::TTL() { + return getToken(ClickHouseParser::TTL, 0); +} + +ClickHouseParser::AlterTableClauseRemoveTTLContext::AlterTableClauseRemoveTTLContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseRemoveTTLContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseRemoveTTL(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseModifyCodecContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyCodecContext::MODIFY() { + return getToken(ClickHouseParser::MODIFY, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyCodecContext::COLUMN() { + return getToken(ClickHouseParser::COLUMN, 0); +} + +ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseModifyCodecContext::nestedIdentifier() { + return getRuleContext(0); +} + +ClickHouseParser::CodecExprContext* ClickHouseParser::AlterTableClauseModifyCodecContext::codecExpr() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyCodecContext::IF() { + return getToken(ClickHouseParser::IF, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyCodecContext::EXISTS() { + return getToken(ClickHouseParser::EXISTS, 0); +} + +ClickHouseParser::AlterTableClauseModifyCodecContext::AlterTableClauseModifyCodecContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseModifyCodecContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseModifyCodec(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseAttachContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseAttachContext::ATTACH() { + return getToken(ClickHouseParser::ATTACH, 0); +} + +ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseAttachContext::partitionClause() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseAttachContext::FROM() { + return getToken(ClickHouseParser::FROM, 0); +} + +ClickHouseParser::TableIdentifierContext* ClickHouseParser::AlterTableClauseAttachContext::tableIdentifier() { + return getRuleContext(0); +} + +ClickHouseParser::AlterTableClauseAttachContext::AlterTableClauseAttachContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseAttachContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseAttach(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseDropProjectionContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseDropProjectionContext::DROP() { + return getToken(ClickHouseParser::DROP, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseDropProjectionContext::PROJECTION() { + return getToken(ClickHouseParser::PROJECTION, 0); +} + +ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseDropProjectionContext::nestedIdentifier() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseDropProjectionContext::IF() { + return getToken(ClickHouseParser::IF, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseDropProjectionContext::EXISTS() { + return getToken(ClickHouseParser::EXISTS, 0); +} + +ClickHouseParser::AlterTableClauseDropProjectionContext::AlterTableClauseDropProjectionContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseDropProjectionContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseDropProjection(this); + else + return visitor->visitChildren(this); +} +//----------------- AlterTableClauseDropIndexContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::AlterTableClauseDropIndexContext::DROP() { + return getToken(ClickHouseParser::DROP, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseDropIndexContext::INDEX() { + return getToken(ClickHouseParser::INDEX, 0); +} + +ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseDropIndexContext::nestedIdentifier() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseDropIndexContext::IF() { + return getToken(ClickHouseParser::IF, 0); +} + +tree::TerminalNode* ClickHouseParser::AlterTableClauseDropIndexContext::EXISTS() { + return getToken(ClickHouseParser::EXISTS, 0); +} + +ClickHouseParser::AlterTableClauseDropIndexContext::AlterTableClauseDropIndexContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::AlterTableClauseDropIndexContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitAlterTableClauseDropIndex(this); + else + return visitor->visitChildren(this); +} //----------------- AlterTableClauseModifyCommentContext ------------------------------------------------------------------ tree::TerminalNode* ClickHouseParser::AlterTableClauseModifyCommentContext::MODIFY() { @@ -1121,46 +1351,46 @@ antlrcpp::Any ClickHouseParser::AlterTableClauseModifyTTLContext::accept(tree::P else return visitor->visitChildren(this); } -//----------------- AlterTableClauseMovePartitionContext ------------------------------------------------------------------ +//----------------- AlterTableClauseAddProjectionContext ------------------------------------------------------------------ -tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::MOVE() { - return getToken(ClickHouseParser::MOVE, 0); +tree::TerminalNode* ClickHouseParser::AlterTableClauseAddProjectionContext::ADD() { + return getToken(ClickHouseParser::ADD, 0); } -ClickHouseParser::PartitionClauseContext* ClickHouseParser::AlterTableClauseMovePartitionContext::partitionClause() { - return getRuleContext(0); +tree::TerminalNode* ClickHouseParser::AlterTableClauseAddProjectionContext::PROJECTION() { + return getToken(ClickHouseParser::PROJECTION, 0); } -tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::TO() { - return getToken(ClickHouseParser::TO, 0); +ClickHouseParser::TableProjectionDfntContext* ClickHouseParser::AlterTableClauseAddProjectionContext::tableProjectionDfnt() { + return getRuleContext(0); } -tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::DISK() { - return getToken(ClickHouseParser::DISK, 0); +tree::TerminalNode* ClickHouseParser::AlterTableClauseAddProjectionContext::IF() { + return getToken(ClickHouseParser::IF, 0); } -tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::STRING_LITERAL() { - return getToken(ClickHouseParser::STRING_LITERAL, 0); +tree::TerminalNode* ClickHouseParser::AlterTableClauseAddProjectionContext::NOT() { + return getToken(ClickHouseParser::NOT, 0); } -tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::VOLUME() { - return getToken(ClickHouseParser::VOLUME, 0); +tree::TerminalNode* ClickHouseParser::AlterTableClauseAddProjectionContext::EXISTS() { + return getToken(ClickHouseParser::EXISTS, 0); } -tree::TerminalNode* ClickHouseParser::AlterTableClauseMovePartitionContext::TABLE() { - return getToken(ClickHouseParser::TABLE, 0); +tree::TerminalNode* ClickHouseParser::AlterTableClauseAddProjectionContext::AFTER() { + return getToken(ClickHouseParser::AFTER, 0); } -ClickHouseParser::TableIdentifierContext* ClickHouseParser::AlterTableClauseMovePartitionContext::tableIdentifier() { - return getRuleContext(0); +ClickHouseParser::NestedIdentifierContext* ClickHouseParser::AlterTableClauseAddProjectionContext::nestedIdentifier() { + return getRuleContext(0); } -ClickHouseParser::AlterTableClauseMovePartitionContext::AlterTableClauseMovePartitionContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } +ClickHouseParser::AlterTableClauseAddProjectionContext::AlterTableClauseAddProjectionContext(AlterTableClauseContext *ctx) { copyFrom(ctx); } -antlrcpp::Any ClickHouseParser::AlterTableClauseMovePartitionContext::accept(tree::ParseTreeVisitor *visitor) { +antlrcpp::Any ClickHouseParser::AlterTableClauseAddProjectionContext::accept(tree::ParseTreeVisitor *visitor) { if (auto parserVisitor = dynamic_cast(visitor)) - return parserVisitor->visitAlterTableClauseMovePartition(this); + return parserVisitor->visitAlterTableClauseAddProjection(this); else return visitor->visitChildren(this); } @@ -1216,41 +1446,41 @@ ClickHouseParser::AlterTableClauseContext* ClickHouseParser::alterTableClause() exitRule(); }); try { - setState(413); + setState(482); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 24, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 35, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(262); + setState(268); match(ClickHouseParser::ADD); - setState(263); + setState(269); match(ClickHouseParser::COLUMN); - setState(267); + setState(273); _errHandler->sync(this); switch (getInterpreter()->adaptivePredict(_input, 7, _ctx)) { case 1: { - setState(264); + setState(270); match(ClickHouseParser::IF); - setState(265); + setState(271); match(ClickHouseParser::NOT); - setState(266); + setState(272); match(ClickHouseParser::EXISTS); break; } } - setState(269); + setState(275); tableColumnDfnt(); - setState(272); + setState(278); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::AFTER) { - setState(270); + setState(276); match(ClickHouseParser::AFTER); - setState(271); + setState(277); nestedIdentifier(); } break; @@ -1259,390 +1489,592 @@ ClickHouseParser::AlterTableClauseContext* ClickHouseParser::alterTableClause() case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(274); + setState(280); match(ClickHouseParser::ADD); - setState(275); + setState(281); match(ClickHouseParser::INDEX); - setState(279); + setState(285); _errHandler->sync(this); switch (getInterpreter()->adaptivePredict(_input, 9, _ctx)) { case 1: { - setState(276); + setState(282); match(ClickHouseParser::IF); - setState(277); + setState(283); match(ClickHouseParser::NOT); - setState(278); + setState(284); match(ClickHouseParser::EXISTS); break; } } - setState(281); + setState(287); tableIndexDfnt(); - setState(284); + setState(290); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::AFTER) { - setState(282); + setState(288); match(ClickHouseParser::AFTER); - setState(283); + setState(289); nestedIdentifier(); } break; } case 3: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(286); - match(ClickHouseParser::ATTACH); - setState(287); - partitionClause(); - setState(290); + setState(292); + match(ClickHouseParser::ADD); + setState(293); + match(ClickHouseParser::PROJECTION); + setState(297); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 11, _ctx)) { + case 1: { + setState(294); + match(ClickHouseParser::IF); + setState(295); + match(ClickHouseParser::NOT); + setState(296); + match(ClickHouseParser::EXISTS); + break; + } + + } + setState(299); + tableProjectionDfnt(); + setState(302); _errHandler->sync(this); _la = _input->LA(1); - if (_la == ClickHouseParser::FROM) { - setState(288); - match(ClickHouseParser::FROM); - setState(289); - tableIdentifier(); + if (_la == ClickHouseParser::AFTER) { + setState(300); + match(ClickHouseParser::AFTER); + setState(301); + nestedIdentifier(); } break; } case 4: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 4); - setState(292); - match(ClickHouseParser::CLEAR); - setState(293); - match(ClickHouseParser::COLUMN); - setState(296); - _errHandler->sync(this); - - switch (getInterpreter()->adaptivePredict(_input, 12, _ctx)) { - case 1: { - setState(294); - match(ClickHouseParser::IF); - setState(295); - match(ClickHouseParser::EXISTS); - break; - } - - } - setState(298); - nestedIdentifier(); - setState(301); + setState(304); + match(ClickHouseParser::ATTACH); + setState(305); + partitionClause(); + setState(308); _errHandler->sync(this); _la = _input->LA(1); - if (_la == ClickHouseParser::IN) { - setState(299); - match(ClickHouseParser::IN); - setState(300); - partitionClause(); + if (_la == ClickHouseParser::FROM) { + setState(306); + match(ClickHouseParser::FROM); + setState(307); + tableIdentifier(); } break; } case 5: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 5); - setState(303); - match(ClickHouseParser::COMMENT); - setState(304); + setState(310); + match(ClickHouseParser::CLEAR); + setState(311); match(ClickHouseParser::COLUMN); - setState(307); + setState(314); _errHandler->sync(this); switch (getInterpreter()->adaptivePredict(_input, 14, _ctx)) { case 1: { - setState(305); + setState(312); match(ClickHouseParser::IF); - setState(306); + setState(313); match(ClickHouseParser::EXISTS); break; } } - setState(309); - nestedIdentifier(); - setState(310); - match(ClickHouseParser::STRING_LITERAL); - break; - } - - case 6: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 6); - setState(312); - match(ClickHouseParser::DELETE); - setState(313); - match(ClickHouseParser::WHERE); - setState(314); - columnExpr(0); - break; - } - - case 7: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 7); - setState(315); - match(ClickHouseParser::DETACH); setState(316); - partitionClause(); - break; - } - - case 8: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 8); - setState(317); - match(ClickHouseParser::DROP); - setState(318); - match(ClickHouseParser::COLUMN); - setState(321); - _errHandler->sync(this); - - switch (getInterpreter()->adaptivePredict(_input, 15, _ctx)) { - case 1: { - setState(319); - match(ClickHouseParser::IF); - setState(320); - match(ClickHouseParser::EXISTS); - break; - } - - } - setState(323); nestedIdentifier(); - break; - } - - case 9: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 9); - setState(324); - match(ClickHouseParser::DROP); - setState(325); - match(ClickHouseParser::INDEX); - setState(328); - _errHandler->sync(this); - - switch (getInterpreter()->adaptivePredict(_input, 16, _ctx)) { - case 1: { - setState(326); - match(ClickHouseParser::IF); - setState(327); - match(ClickHouseParser::EXISTS); - break; - } - - } - setState(330); - nestedIdentifier(); - break; - } - - case 10: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 10); - setState(331); - match(ClickHouseParser::DROP); - setState(332); - partitionClause(); - break; - } - - case 11: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 11); - setState(333); - match(ClickHouseParser::FREEZE); - setState(335); + setState(319); _errHandler->sync(this); _la = _input->LA(1); - if (_la == ClickHouseParser::PARTITION) { - setState(334); + if (_la == ClickHouseParser::IN) { + setState(317); + match(ClickHouseParser::IN); + setState(318); partitionClause(); } break; } - case 12: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 12); - setState(337); - match(ClickHouseParser::MODIFY); - setState(338); - match(ClickHouseParser::COLUMN); - setState(341); + case 6: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 6); + setState(321); + match(ClickHouseParser::CLEAR); + setState(322); + match(ClickHouseParser::INDEX); + setState(325); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 16, _ctx)) { + case 1: { + setState(323); + match(ClickHouseParser::IF); + setState(324); + match(ClickHouseParser::EXISTS); + break; + } + + } + setState(327); + nestedIdentifier(); + setState(330); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::IN) { + setState(328); + match(ClickHouseParser::IN); + setState(329); + partitionClause(); + } + break; + } + + case 7: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 7); + setState(332); + match(ClickHouseParser::CLEAR); + setState(333); + match(ClickHouseParser::PROJECTION); + setState(336); _errHandler->sync(this); switch (getInterpreter()->adaptivePredict(_input, 18, _ctx)) { case 1: { - setState(339); + setState(334); match(ClickHouseParser::IF); - setState(340); + setState(335); match(ClickHouseParser::EXISTS); break; } } - setState(343); + setState(338); nestedIdentifier(); - setState(344); - codecExpr(); - break; - } - - case 13: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 13); - setState(346); - match(ClickHouseParser::MODIFY); - setState(347); - match(ClickHouseParser::COLUMN); - setState(350); + setState(341); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 19, _ctx)) { - case 1: { - setState(348); - match(ClickHouseParser::IF); - setState(349); - match(ClickHouseParser::EXISTS); - break; + _la = _input->LA(1); + if (_la == ClickHouseParser::IN) { + setState(339); + match(ClickHouseParser::IN); + setState(340); + partitionClause(); } - - } - setState(352); - nestedIdentifier(); - setState(353); - match(ClickHouseParser::COMMENT); - setState(354); - match(ClickHouseParser::STRING_LITERAL); break; } - case 14: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 14); - setState(356); - match(ClickHouseParser::MODIFY); - setState(357); + case 8: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 8); + setState(343); + match(ClickHouseParser::COMMENT); + setState(344); match(ClickHouseParser::COLUMN); - setState(360); + setState(347); _errHandler->sync(this); switch (getInterpreter()->adaptivePredict(_input, 20, _ctx)) { case 1: { - setState(358); + setState(345); match(ClickHouseParser::IF); - setState(359); + setState(346); match(ClickHouseParser::EXISTS); break; } } - setState(362); + setState(349); nestedIdentifier(); - setState(363); - match(ClickHouseParser::REMOVE); - setState(364); - tableColumnPropertyType(); + setState(350); + match(ClickHouseParser::STRING_LITERAL); break; } - case 15: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 15); - setState(366); - match(ClickHouseParser::MODIFY); - setState(367); - match(ClickHouseParser::COLUMN); - setState(370); - _errHandler->sync(this); - - switch (getInterpreter()->adaptivePredict(_input, 21, _ctx)) { - case 1: { - setState(368); - match(ClickHouseParser::IF); - setState(369); - match(ClickHouseParser::EXISTS); - break; - } - - } - setState(372); - tableColumnDfnt(); - break; - } - - case 16: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 16); - setState(373); - match(ClickHouseParser::MODIFY); - setState(374); - match(ClickHouseParser::ORDER); - setState(375); - match(ClickHouseParser::BY); - setState(376); + case 9: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 9); + setState(352); + match(ClickHouseParser::DELETE); + setState(353); + match(ClickHouseParser::WHERE); + setState(354); columnExpr(0); break; } - case 17: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 17); + case 10: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 10); + setState(355); + match(ClickHouseParser::DETACH); + setState(356); + partitionClause(); + break; + } + + case 11: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 11); + setState(357); + match(ClickHouseParser::DROP); + setState(358); + match(ClickHouseParser::COLUMN); + setState(361); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 21, _ctx)) { + case 1: { + setState(359); + match(ClickHouseParser::IF); + setState(360); + match(ClickHouseParser::EXISTS); + break; + } + + } + setState(363); + nestedIdentifier(); + break; + } + + case 12: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 12); + setState(364); + match(ClickHouseParser::DROP); + setState(365); + match(ClickHouseParser::INDEX); + setState(368); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 22, _ctx)) { + case 1: { + setState(366); + match(ClickHouseParser::IF); + setState(367); + match(ClickHouseParser::EXISTS); + break; + } + + } + setState(370); + nestedIdentifier(); + break; + } + + case 13: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 13); + setState(371); + match(ClickHouseParser::DROP); + setState(372); + match(ClickHouseParser::PROJECTION); + setState(375); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 23, _ctx)) { + case 1: { + setState(373); + match(ClickHouseParser::IF); + setState(374); + match(ClickHouseParser::EXISTS); + break; + } + + } setState(377); - match(ClickHouseParser::MODIFY); + nestedIdentifier(); + break; + } + + case 14: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 14); setState(378); - ttlClause(); + match(ClickHouseParser::DROP); + setState(379); + partitionClause(); + break; + } + + case 15: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 15); + setState(380); + match(ClickHouseParser::FREEZE); + setState(382); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::PARTITION) { + setState(381); + partitionClause(); + } + break; + } + + case 16: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 16); + setState(384); + match(ClickHouseParser::MATERIALIZE); + setState(385); + match(ClickHouseParser::INDEX); + setState(388); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 25, _ctx)) { + case 1: { + setState(386); + match(ClickHouseParser::IF); + setState(387); + match(ClickHouseParser::EXISTS); + break; + } + + } + setState(390); + nestedIdentifier(); + setState(393); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::IN) { + setState(391); + match(ClickHouseParser::IN); + setState(392); + partitionClause(); + } + break; + } + + case 17: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 17); + setState(395); + match(ClickHouseParser::MATERIALIZE); + setState(396); + match(ClickHouseParser::PROJECTION); + setState(399); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 27, _ctx)) { + case 1: { + setState(397); + match(ClickHouseParser::IF); + setState(398); + match(ClickHouseParser::EXISTS); + break; + } + + } + setState(401); + nestedIdentifier(); + setState(404); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::IN) { + setState(402); + match(ClickHouseParser::IN); + setState(403); + partitionClause(); + } break; } case 18: { - _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 18); - setState(379); - match(ClickHouseParser::MOVE); - setState(380); - partitionClause(); - setState(390); + setState(406); + match(ClickHouseParser::MODIFY); + setState(407); + match(ClickHouseParser::COLUMN); + setState(410); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 22, _ctx)) { + + switch (getInterpreter()->adaptivePredict(_input, 29, _ctx)) { case 1: { - setState(381); + setState(408); + match(ClickHouseParser::IF); + setState(409); + match(ClickHouseParser::EXISTS); + break; + } + + } + setState(412); + nestedIdentifier(); + setState(413); + codecExpr(); + break; + } + + case 19: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 19); + setState(415); + match(ClickHouseParser::MODIFY); + setState(416); + match(ClickHouseParser::COLUMN); + setState(419); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 30, _ctx)) { + case 1: { + setState(417); + match(ClickHouseParser::IF); + setState(418); + match(ClickHouseParser::EXISTS); + break; + } + + } + setState(421); + nestedIdentifier(); + setState(422); + match(ClickHouseParser::COMMENT); + setState(423); + match(ClickHouseParser::STRING_LITERAL); + break; + } + + case 20: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 20); + setState(425); + match(ClickHouseParser::MODIFY); + setState(426); + match(ClickHouseParser::COLUMN); + setState(429); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 31, _ctx)) { + case 1: { + setState(427); + match(ClickHouseParser::IF); + setState(428); + match(ClickHouseParser::EXISTS); + break; + } + + } + setState(431); + nestedIdentifier(); + setState(432); + match(ClickHouseParser::REMOVE); + setState(433); + tableColumnPropertyType(); + break; + } + + case 21: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 21); + setState(435); + match(ClickHouseParser::MODIFY); + setState(436); + match(ClickHouseParser::COLUMN); + setState(439); + _errHandler->sync(this); + + switch (getInterpreter()->adaptivePredict(_input, 32, _ctx)) { + case 1: { + setState(437); + match(ClickHouseParser::IF); + setState(438); + match(ClickHouseParser::EXISTS); + break; + } + + } + setState(441); + tableColumnDfnt(); + break; + } + + case 22: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 22); + setState(442); + match(ClickHouseParser::MODIFY); + setState(443); + match(ClickHouseParser::ORDER); + setState(444); + match(ClickHouseParser::BY); + setState(445); + columnExpr(0); + break; + } + + case 23: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 23); + setState(446); + match(ClickHouseParser::MODIFY); + setState(447); + ttlClause(); + break; + } + + case 24: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 24); + setState(448); + match(ClickHouseParser::MOVE); + setState(449); + partitionClause(); + setState(459); + _errHandler->sync(this); + switch (getInterpreter()->adaptivePredict(_input, 33, _ctx)) { + case 1: { + setState(450); match(ClickHouseParser::TO); - setState(382); + setState(451); match(ClickHouseParser::DISK); - setState(383); + setState(452); match(ClickHouseParser::STRING_LITERAL); break; } case 2: { - setState(384); + setState(453); match(ClickHouseParser::TO); - setState(385); + setState(454); match(ClickHouseParser::VOLUME); - setState(386); + setState(455); match(ClickHouseParser::STRING_LITERAL); break; } case 3: { - setState(387); + setState(456); match(ClickHouseParser::TO); - setState(388); + setState(457); match(ClickHouseParser::TABLE); - setState(389); + setState(458); tableIdentifier(); break; } @@ -1651,67 +2083,67 @@ ClickHouseParser::AlterTableClauseContext* ClickHouseParser::alterTableClause() break; } - case 19: { + case 25: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 19); - setState(392); + enterOuterAlt(_localctx, 25); + setState(461); match(ClickHouseParser::REMOVE); - setState(393); + setState(462); match(ClickHouseParser::TTL); break; } - case 20: { + case 26: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 20); - setState(394); + enterOuterAlt(_localctx, 26); + setState(463); match(ClickHouseParser::RENAME); - setState(395); + setState(464); match(ClickHouseParser::COLUMN); - setState(398); + setState(467); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 23, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 34, _ctx)) { case 1: { - setState(396); + setState(465); match(ClickHouseParser::IF); - setState(397); + setState(466); match(ClickHouseParser::EXISTS); break; } } - setState(400); + setState(469); nestedIdentifier(); - setState(401); + setState(470); match(ClickHouseParser::TO); - setState(402); + setState(471); nestedIdentifier(); break; } - case 21: { + case 27: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 21); - setState(404); + enterOuterAlt(_localctx, 27); + setState(473); match(ClickHouseParser::REPLACE); - setState(405); + setState(474); partitionClause(); - setState(406); + setState(475); match(ClickHouseParser::FROM); - setState(407); + setState(476); tableIdentifier(); break; } - case 22: { + case 28: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); - enterOuterAlt(_localctx, 22); - setState(409); + enterOuterAlt(_localctx, 28); + setState(478); match(ClickHouseParser::UPDATE); - setState(410); + setState(479); assignmentExprList(); - setState(411); + setState(480); whereClause(); break; } @@ -1773,17 +2205,17 @@ ClickHouseParser::AssignmentExprListContext* ClickHouseParser::assignmentExprLis }); try { enterOuterAlt(_localctx, 1); - setState(415); + setState(484); assignmentExpr(); - setState(420); + setState(489); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(416); + setState(485); match(ClickHouseParser::COMMA); - setState(417); + setState(486); assignmentExpr(); - setState(422); + setState(491); _errHandler->sync(this); _la = _input->LA(1); } @@ -1838,11 +2270,11 @@ ClickHouseParser::AssignmentExprContext* ClickHouseParser::assignmentExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(423); + setState(492); nestedIdentifier(); - setState(424); + setState(493); match(ClickHouseParser::EQ_SINGLE); - setState(425); + setState(494); columnExpr(0); } @@ -1908,7 +2340,7 @@ ClickHouseParser::TableColumnPropertyTypeContext* ClickHouseParser::tableColumnP }); try { enterOuterAlt(_localctx, 1); - setState(427); + setState(496); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ALIAS) @@ -1975,25 +2407,25 @@ ClickHouseParser::PartitionClauseContext* ClickHouseParser::partitionClause() { exitRule(); }); try { - setState(434); + setState(503); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 26, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 37, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(429); + setState(498); match(ClickHouseParser::PARTITION); - setState(430); + setState(499); columnExpr(0); break; } case 2: { enterOuterAlt(_localctx, 2); - setState(431); + setState(500); match(ClickHouseParser::PARTITION); - setState(432); + setState(501); match(ClickHouseParser::ID); - setState(433); + setState(502); match(ClickHouseParser::STRING_LITERAL); break; } @@ -2063,18 +2495,18 @@ ClickHouseParser::AttachStmtContext* ClickHouseParser::attachStmt() { try { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(436); + setState(505); match(ClickHouseParser::ATTACH); - setState(437); + setState(506); match(ClickHouseParser::DICTIONARY); - setState(438); + setState(507); tableIdentifier(); - setState(440); + setState(509); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(439); + setState(508); clusterClause(); } @@ -2133,18 +2565,18 @@ ClickHouseParser::CheckStmtContext* ClickHouseParser::checkStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(442); + setState(511); match(ClickHouseParser::CHECK); - setState(443); + setState(512); match(ClickHouseParser::TABLE); - setState(444); + setState(513); tableIdentifier(); - setState(446); + setState(515); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::PARTITION) { - setState(445); + setState(514); partitionClause(); } @@ -2556,13 +2988,13 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { exitRule(); }); try { - setState(585); + setState(654); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 60, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 71, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(448); + setState(517); _la = _input->LA(1); if (!(_la == ClickHouseParser::ATTACH @@ -2573,39 +3005,39 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { _errHandler->reportMatch(this); consume(); } - setState(449); + setState(518); match(ClickHouseParser::DATABASE); - setState(453); + setState(522); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 29, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 40, _ctx)) { case 1: { - setState(450); + setState(519); match(ClickHouseParser::IF); - setState(451); + setState(520); match(ClickHouseParser::NOT); - setState(452); + setState(521); match(ClickHouseParser::EXISTS); break; } } - setState(455); + setState(524); databaseIdentifier(); - setState(457); + setState(526); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(456); + setState(525); clusterClause(); } - setState(460); + setState(529); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ENGINE) { - setState(459); + setState(528); engineExpr(); } break; @@ -2614,7 +3046,7 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(462); + setState(531); _la = _input->LA(1); if (!(_la == ClickHouseParser::ATTACH @@ -2625,44 +3057,44 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { _errHandler->reportMatch(this); consume(); } - setState(463); + setState(532); match(ClickHouseParser::DICTIONARY); - setState(467); + setState(536); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 32, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 43, _ctx)) { case 1: { - setState(464); + setState(533); match(ClickHouseParser::IF); - setState(465); + setState(534); match(ClickHouseParser::NOT); - setState(466); + setState(535); match(ClickHouseParser::EXISTS); break; } } - setState(469); + setState(538); tableIdentifier(); - setState(471); + setState(540); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::UUID) { - setState(470); + setState(539); uuidClause(); } - setState(474); + setState(543); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(473); + setState(542); clusterClause(); } - setState(476); + setState(545); dictionarySchemaClause(); - setState(477); + setState(546); dictionaryEngineClause(); break; } @@ -2670,7 +3102,7 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(479); + setState(548); _la = _input->LA(1); if (!(_la == ClickHouseParser::ATTACH @@ -2681,81 +3113,81 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { _errHandler->reportMatch(this); consume(); } - setState(480); + setState(549); match(ClickHouseParser::LIVE); - setState(481); + setState(550); match(ClickHouseParser::VIEW); - setState(485); + setState(554); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 35, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 46, _ctx)) { case 1: { - setState(482); + setState(551); match(ClickHouseParser::IF); - setState(483); + setState(552); match(ClickHouseParser::NOT); - setState(484); + setState(553); match(ClickHouseParser::EXISTS); break; } } - setState(487); + setState(556); tableIdentifier(); - setState(489); + setState(558); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::UUID) { - setState(488); + setState(557); uuidClause(); } - setState(492); + setState(561); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(491); + setState(560); clusterClause(); } - setState(499); + setState(568); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::WITH) { - setState(494); + setState(563); match(ClickHouseParser::WITH); - setState(495); + setState(564); match(ClickHouseParser::TIMEOUT); - setState(497); + setState(566); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::DECIMAL_LITERAL) { - setState(496); + setState(565); match(ClickHouseParser::DECIMAL_LITERAL); } } - setState(502); + setState(571); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TO) { - setState(501); + setState(570); destinationClause(); } - setState(505); + setState(574); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 41, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 52, _ctx)) { case 1: { - setState(504); + setState(573); tableSchemaClause(); break; } } - setState(507); + setState(576); subqueryClause(); break; } @@ -2763,7 +3195,7 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { case 4: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 4); - setState(509); + setState(578); _la = _input->LA(1); if (!(_la == ClickHouseParser::ATTACH @@ -2774,69 +3206,69 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { _errHandler->reportMatch(this); consume(); } - setState(510); + setState(579); match(ClickHouseParser::MATERIALIZED); - setState(511); + setState(580); match(ClickHouseParser::VIEW); - setState(515); + setState(584); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 42, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 53, _ctx)) { case 1: { - setState(512); + setState(581); match(ClickHouseParser::IF); - setState(513); + setState(582); match(ClickHouseParser::NOT); - setState(514); + setState(583); match(ClickHouseParser::EXISTS); break; } } - setState(517); + setState(586); tableIdentifier(); - setState(519); + setState(588); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::UUID) { - setState(518); + setState(587); uuidClause(); } - setState(522); + setState(591); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(521); + setState(590); clusterClause(); } - setState(525); + setState(594); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::AS || _la == ClickHouseParser::LPAREN) { - setState(524); + setState(593); tableSchemaClause(); } - setState(532); + setState(601); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::TO: { - setState(527); + setState(596); destinationClause(); break; } case ClickHouseParser::ENGINE: { - setState(528); + setState(597); engineClause(); - setState(530); + setState(599); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::POPULATE) { - setState(529); + setState(598); match(ClickHouseParser::POPULATE); } break; @@ -2845,7 +3277,7 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { default: throw NoViableAltException(this); } - setState(534); + setState(603); subqueryClause(); break; } @@ -2853,7 +3285,7 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { case 5: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 5); - setState(536); + setState(605); _la = _input->LA(1); if (!(_la == ClickHouseParser::ATTACH @@ -2864,74 +3296,74 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { _errHandler->reportMatch(this); consume(); } - setState(538); + setState(607); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TEMPORARY) { - setState(537); + setState(606); match(ClickHouseParser::TEMPORARY); } - setState(540); + setState(609); match(ClickHouseParser::TABLE); - setState(544); + setState(613); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 49, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 60, _ctx)) { case 1: { - setState(541); + setState(610); match(ClickHouseParser::IF); - setState(542); + setState(611); match(ClickHouseParser::NOT); - setState(543); + setState(612); match(ClickHouseParser::EXISTS); break; } } - setState(546); + setState(615); tableIdentifier(); - setState(548); + setState(617); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::UUID) { - setState(547); + setState(616); uuidClause(); } - setState(551); + setState(620); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(550); + setState(619); clusterClause(); } - setState(554); + setState(623); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 52, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 63, _ctx)) { case 1: { - setState(553); + setState(622); tableSchemaClause(); break; } } - setState(557); + setState(626); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ENGINE) { - setState(556); + setState(625); engineClause(); } - setState(560); + setState(629); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::AS) { - setState(559); + setState(628); subqueryClause(); } break; @@ -2940,7 +3372,7 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { case 6: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 6); - setState(562); + setState(631); _la = _input->LA(1); if (!(_la == ClickHouseParser::ATTACH @@ -2951,63 +3383,63 @@ ClickHouseParser::CreateStmtContext* ClickHouseParser::createStmt() { _errHandler->reportMatch(this); consume(); } - setState(565); + setState(634); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OR) { - setState(563); + setState(632); match(ClickHouseParser::OR); - setState(564); + setState(633); match(ClickHouseParser::REPLACE); } - setState(567); + setState(636); match(ClickHouseParser::VIEW); - setState(571); + setState(640); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 56, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 67, _ctx)) { case 1: { - setState(568); + setState(637); match(ClickHouseParser::IF); - setState(569); + setState(638); match(ClickHouseParser::NOT); - setState(570); + setState(639); match(ClickHouseParser::EXISTS); break; } } - setState(573); + setState(642); tableIdentifier(); - setState(575); + setState(644); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::UUID) { - setState(574); + setState(643); uuidClause(); } - setState(578); + setState(647); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(577); + setState(646); clusterClause(); } - setState(581); + setState(650); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 59, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 70, _ctx)) { case 1: { - setState(580); + setState(649); tableSchemaClause(); break; } } - setState(583); + setState(652); subqueryClause(); break; } @@ -3077,23 +3509,23 @@ ClickHouseParser::DictionarySchemaClauseContext* ClickHouseParser::dictionarySch }); try { enterOuterAlt(_localctx, 1); - setState(587); + setState(656); match(ClickHouseParser::LPAREN); - setState(588); + setState(657); dictionaryAttrDfnt(); - setState(593); + setState(662); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(589); + setState(658); match(ClickHouseParser::COMMA); - setState(590); + setState(659); dictionaryAttrDfnt(); - setState(595); + setState(664); _errHandler->sync(this); _la = _input->LA(1); } - setState(596); + setState(665); match(ClickHouseParser::RPAREN); } @@ -3199,67 +3631,67 @@ ClickHouseParser::DictionaryAttrDfntContext* ClickHouseParser::dictionaryAttrDfn try { size_t alt; enterOuterAlt(_localctx, 1); - setState(598); + setState(667); identifier(); - setState(599); + setState(668); columnTypeExpr(); - setState(621); + setState(690); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 63, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 74, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(619); + setState(688); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 62, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 73, _ctx)) { case 1: { - setState(600); + setState(669); if (!(!_localctx->attrs.count("default"))) throw FailedPredicateException(this, "!$attrs.count(\"default\")"); - setState(601); + setState(670); match(ClickHouseParser::DEFAULT); - setState(602); + setState(671); literal(); _localctx->attrs.insert("default"); break; } case 2: { - setState(605); + setState(674); if (!(!_localctx->attrs.count("expression"))) throw FailedPredicateException(this, "!$attrs.count(\"expression\")"); - setState(606); + setState(675); match(ClickHouseParser::EXPRESSION); - setState(607); + setState(676); columnExpr(0); _localctx->attrs.insert("expression"); break; } case 3: { - setState(610); + setState(679); if (!(!_localctx->attrs.count("hierarchical"))) throw FailedPredicateException(this, "!$attrs.count(\"hierarchical\")"); - setState(611); + setState(680); match(ClickHouseParser::HIERARCHICAL); _localctx->attrs.insert("hierarchical"); break; } case 4: { - setState(613); + setState(682); if (!(!_localctx->attrs.count("injective"))) throw FailedPredicateException(this, "!$attrs.count(\"injective\")"); - setState(614); + setState(683); match(ClickHouseParser::INJECTIVE); _localctx->attrs.insert("injective"); break; } case 5: { - setState(616); + setState(685); if (!(!_localctx->attrs.count("is_object_id"))) throw FailedPredicateException(this, "!$attrs.count(\"is_object_id\")"); - setState(617); + setState(686); match(ClickHouseParser::IS_OBJECT_ID); _localctx->attrs.insert("is_object_id"); break; @@ -3267,9 +3699,9 @@ ClickHouseParser::DictionaryAttrDfntContext* ClickHouseParser::dictionaryAttrDfn } } - setState(623); + setState(692); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 63, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 74, _ctx); } } @@ -3355,70 +3787,70 @@ ClickHouseParser::DictionaryEngineClauseContext* ClickHouseParser::dictionaryEng try { size_t alt; enterOuterAlt(_localctx, 1); - setState(625); + setState(694); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 64, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 75, _ctx)) { case 1: { - setState(624); + setState(693); dictionaryPrimaryKeyClause(); break; } } - setState(649); + setState(718); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 66, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 77, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(647); + setState(716); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 65, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 76, _ctx)) { case 1: { - setState(627); + setState(696); if (!(!_localctx->clauses.count("source"))) throw FailedPredicateException(this, "!$clauses.count(\"source\")"); - setState(628); + setState(697); sourceClause(); _localctx->clauses.insert("source"); break; } case 2: { - setState(631); + setState(700); if (!(!_localctx->clauses.count("lifetime"))) throw FailedPredicateException(this, "!$clauses.count(\"lifetime\")"); - setState(632); + setState(701); lifetimeClause(); _localctx->clauses.insert("lifetime"); break; } case 3: { - setState(635); + setState(704); if (!(!_localctx->clauses.count("layout"))) throw FailedPredicateException(this, "!$clauses.count(\"layout\")"); - setState(636); + setState(705); layoutClause(); _localctx->clauses.insert("layout"); break; } case 4: { - setState(639); + setState(708); if (!(!_localctx->clauses.count("range"))) throw FailedPredicateException(this, "!$clauses.count(\"range\")"); - setState(640); + setState(709); rangeClause(); _localctx->clauses.insert("range"); break; } case 5: { - setState(643); + setState(712); if (!(!_localctx->clauses.count("settings"))) throw FailedPredicateException(this, "!$clauses.count(\"settings\")"); - setState(644); + setState(713); dictionarySettingsClause(); _localctx->clauses.insert("settings"); break; @@ -3426,9 +3858,9 @@ ClickHouseParser::DictionaryEngineClauseContext* ClickHouseParser::dictionaryEng } } - setState(651); + setState(720); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 66, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 77, _ctx); } } @@ -3481,11 +3913,11 @@ ClickHouseParser::DictionaryPrimaryKeyClauseContext* ClickHouseParser::dictionar }); try { enterOuterAlt(_localctx, 1); - setState(652); + setState(721); match(ClickHouseParser::PRIMARY); - setState(653); + setState(722); match(ClickHouseParser::KEY); - setState(654); + setState(723); columnExprList(); } @@ -3547,9 +3979,9 @@ ClickHouseParser::DictionaryArgExprContext* ClickHouseParser::dictionaryArgExpr( }); try { enterOuterAlt(_localctx, 1); - setState(656); + setState(725); identifier(); - setState(663); + setState(732); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::AFTER: @@ -3649,6 +4081,7 @@ ClickHouseParser::DictionaryArgExprContext* ClickHouseParser::dictionaryArgExpr( case ClickHouseParser::LOCAL: case ClickHouseParser::LOGS: case ClickHouseParser::MATERIALIZED: + case ClickHouseParser::MATERIALIZE: case ClickHouseParser::MAX: case ClickHouseParser::MERGES: case ClickHouseParser::MIN: @@ -3729,16 +4162,16 @@ ClickHouseParser::DictionaryArgExprContext* ClickHouseParser::dictionaryArgExpr( case ClickHouseParser::JSON_FALSE: case ClickHouseParser::JSON_TRUE: case ClickHouseParser::IDENTIFIER: { - setState(657); + setState(726); identifier(); - setState(660); + setState(729); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::LPAREN) { - setState(658); + setState(727); match(ClickHouseParser::LPAREN); - setState(659); + setState(728); match(ClickHouseParser::RPAREN); } break; @@ -3755,7 +4188,7 @@ ClickHouseParser::DictionaryArgExprContext* ClickHouseParser::dictionaryArgExpr( case ClickHouseParser::DASH: case ClickHouseParser::DOT: case ClickHouseParser::PLUS: { - setState(662); + setState(731); literal(); break; } @@ -3835,15 +4268,15 @@ ClickHouseParser::SourceClauseContext* ClickHouseParser::sourceClause() { }); try { enterOuterAlt(_localctx, 1); - setState(665); + setState(734); match(ClickHouseParser::SOURCE); - setState(666); + setState(735); match(ClickHouseParser::LPAREN); - setState(667); + setState(736); identifier(); - setState(668); + setState(737); match(ClickHouseParser::LPAREN); - setState(672); + setState(741); _errHandler->sync(this); _la = _input->LA(1); while ((((_la & ~ 0x3fULL) == 0) && @@ -3944,6 +4377,7 @@ ClickHouseParser::SourceClauseContext* ClickHouseParser::sourceClause() { | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -3967,10 +4401,10 @@ ClickHouseParser::SourceClauseContext* ClickHouseParser::sourceClause() { | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) | (1ULL << (ClickHouseParser::QUARTER - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -4024,15 +4458,15 @@ ClickHouseParser::SourceClauseContext* ClickHouseParser::sourceClause() { | (1ULL << (ClickHouseParser::JSON_FALSE - 128)) | (1ULL << (ClickHouseParser::JSON_TRUE - 128)) | (1ULL << (ClickHouseParser::IDENTIFIER - 128)))) != 0)) { - setState(669); + setState(738); dictionaryArgExpr(); - setState(674); + setState(743); _errHandler->sync(this); _la = _input->LA(1); } - setState(675); + setState(744); match(ClickHouseParser::RPAREN); - setState(676); + setState(745); match(ClickHouseParser::RPAREN); } @@ -4101,39 +4535,39 @@ ClickHouseParser::LifetimeClauseContext* ClickHouseParser::lifetimeClause() { }); try { enterOuterAlt(_localctx, 1); - setState(678); + setState(747); match(ClickHouseParser::LIFETIME); - setState(679); + setState(748); match(ClickHouseParser::LPAREN); - setState(689); + setState(758); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::DECIMAL_LITERAL: { - setState(680); + setState(749); match(ClickHouseParser::DECIMAL_LITERAL); break; } case ClickHouseParser::MIN: { - setState(681); + setState(750); match(ClickHouseParser::MIN); - setState(682); + setState(751); match(ClickHouseParser::DECIMAL_LITERAL); - setState(683); + setState(752); match(ClickHouseParser::MAX); - setState(684); + setState(753); match(ClickHouseParser::DECIMAL_LITERAL); break; } case ClickHouseParser::MAX: { - setState(685); + setState(754); match(ClickHouseParser::MAX); - setState(686); + setState(755); match(ClickHouseParser::DECIMAL_LITERAL); - setState(687); + setState(756); match(ClickHouseParser::MIN); - setState(688); + setState(757); match(ClickHouseParser::DECIMAL_LITERAL); break; } @@ -4141,7 +4575,7 @@ ClickHouseParser::LifetimeClauseContext* ClickHouseParser::lifetimeClause() { default: throw NoViableAltException(this); } - setState(691); + setState(760); match(ClickHouseParser::RPAREN); } @@ -4215,15 +4649,15 @@ ClickHouseParser::LayoutClauseContext* ClickHouseParser::layoutClause() { }); try { enterOuterAlt(_localctx, 1); - setState(693); + setState(762); match(ClickHouseParser::LAYOUT); - setState(694); + setState(763); match(ClickHouseParser::LPAREN); - setState(695); + setState(764); identifier(); - setState(696); + setState(765); match(ClickHouseParser::LPAREN); - setState(700); + setState(769); _errHandler->sync(this); _la = _input->LA(1); while ((((_la & ~ 0x3fULL) == 0) && @@ -4324,6 +4758,7 @@ ClickHouseParser::LayoutClauseContext* ClickHouseParser::layoutClause() { | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -4347,10 +4782,10 @@ ClickHouseParser::LayoutClauseContext* ClickHouseParser::layoutClause() { | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) | (1ULL << (ClickHouseParser::QUARTER - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -4404,15 +4839,15 @@ ClickHouseParser::LayoutClauseContext* ClickHouseParser::layoutClause() { | (1ULL << (ClickHouseParser::JSON_FALSE - 128)) | (1ULL << (ClickHouseParser::JSON_TRUE - 128)) | (1ULL << (ClickHouseParser::IDENTIFIER - 128)))) != 0)) { - setState(697); + setState(766); dictionaryArgExpr(); - setState(702); + setState(771); _errHandler->sync(this); _la = _input->LA(1); } - setState(703); + setState(772); match(ClickHouseParser::RPAREN); - setState(704); + setState(773); match(ClickHouseParser::RPAREN); } @@ -4481,33 +4916,33 @@ ClickHouseParser::RangeClauseContext* ClickHouseParser::rangeClause() { }); try { enterOuterAlt(_localctx, 1); - setState(706); + setState(775); match(ClickHouseParser::RANGE); - setState(707); + setState(776); match(ClickHouseParser::LPAREN); - setState(718); + setState(787); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::MIN: { - setState(708); + setState(777); match(ClickHouseParser::MIN); - setState(709); + setState(778); identifier(); - setState(710); + setState(779); match(ClickHouseParser::MAX); - setState(711); + setState(780); identifier(); break; } case ClickHouseParser::MAX: { - setState(713); + setState(782); match(ClickHouseParser::MAX); - setState(714); + setState(783); identifier(); - setState(715); + setState(784); match(ClickHouseParser::MIN); - setState(716); + setState(785); identifier(); break; } @@ -4515,7 +4950,7 @@ ClickHouseParser::RangeClauseContext* ClickHouseParser::rangeClause() { default: throw NoViableAltException(this); } - setState(720); + setState(789); match(ClickHouseParser::RPAREN); } @@ -4572,13 +5007,13 @@ ClickHouseParser::DictionarySettingsClauseContext* ClickHouseParser::dictionaryS }); try { enterOuterAlt(_localctx, 1); - setState(722); + setState(791); match(ClickHouseParser::SETTINGS); - setState(723); + setState(792); match(ClickHouseParser::LPAREN); - setState(724); + setState(793); settingExprList(); - setState(725); + setState(794); match(ClickHouseParser::RPAREN); } @@ -4635,11 +5070,11 @@ ClickHouseParser::ClusterClauseContext* ClickHouseParser::clusterClause() { }); try { enterOuterAlt(_localctx, 1); - setState(727); + setState(796); match(ClickHouseParser::ON); - setState(728); + setState(797); match(ClickHouseParser::CLUSTER); - setState(731); + setState(800); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::AFTER: @@ -4739,6 +5174,7 @@ ClickHouseParser::ClusterClauseContext* ClickHouseParser::clusterClause() { case ClickHouseParser::LOCAL: case ClickHouseParser::LOGS: case ClickHouseParser::MATERIALIZED: + case ClickHouseParser::MATERIALIZE: case ClickHouseParser::MAX: case ClickHouseParser::MERGES: case ClickHouseParser::MIN: @@ -4819,13 +5255,13 @@ ClickHouseParser::ClusterClauseContext* ClickHouseParser::clusterClause() { case ClickHouseParser::JSON_FALSE: case ClickHouseParser::JSON_TRUE: case ClickHouseParser::IDENTIFIER: { - setState(729); + setState(798); identifier(); break; } case ClickHouseParser::STRING_LITERAL: { - setState(730); + setState(799); match(ClickHouseParser::STRING_LITERAL); break; } @@ -4880,9 +5316,9 @@ ClickHouseParser::UuidClauseContext* ClickHouseParser::uuidClause() { }); try { enterOuterAlt(_localctx, 1); - setState(733); + setState(802); match(ClickHouseParser::UUID); - setState(734); + setState(803); match(ClickHouseParser::STRING_LITERAL); } @@ -4931,9 +5367,9 @@ ClickHouseParser::DestinationClauseContext* ClickHouseParser::destinationClause( }); try { enterOuterAlt(_localctx, 1); - setState(736); + setState(805); match(ClickHouseParser::TO); - setState(737); + setState(806); tableIdentifier(); } @@ -4982,9 +5418,9 @@ ClickHouseParser::SubqueryClauseContext* ClickHouseParser::subqueryClause() { }); try { enterOuterAlt(_localctx, 1); - setState(739); + setState(808); match(ClickHouseParser::AS); - setState(740); + setState(809); selectUnionStmt(); } @@ -5094,29 +5530,29 @@ ClickHouseParser::TableSchemaClauseContext* ClickHouseParser::tableSchemaClause( exitRule(); }); try { - setState(757); + setState(826); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 75, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 86, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(742); + setState(811); match(ClickHouseParser::LPAREN); - setState(743); + setState(812); tableElementExpr(); - setState(748); + setState(817); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(744); + setState(813); match(ClickHouseParser::COMMA); - setState(745); + setState(814); tableElementExpr(); - setState(750); + setState(819); _errHandler->sync(this); _la = _input->LA(1); } - setState(751); + setState(820); match(ClickHouseParser::RPAREN); break; } @@ -5124,9 +5560,9 @@ ClickHouseParser::TableSchemaClauseContext* ClickHouseParser::tableSchemaClause( case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(753); + setState(822); match(ClickHouseParser::AS); - setState(754); + setState(823); tableIdentifier(); break; } @@ -5134,9 +5570,9 @@ ClickHouseParser::TableSchemaClauseContext* ClickHouseParser::tableSchemaClause( case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(755); + setState(824); match(ClickHouseParser::AS); - setState(756); + setState(825); tableFunctionExpr(); break; } @@ -5234,71 +5670,71 @@ ClickHouseParser::EngineClauseContext* ClickHouseParser::engineClause() { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(759); + setState(828); engineExpr(); - setState(786); + setState(855); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 77, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 88, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(784); + setState(853); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 76, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 87, _ctx)) { case 1: { - setState(760); + setState(829); if (!(!_localctx->clauses.count("orderByClause"))) throw FailedPredicateException(this, "!$clauses.count(\"orderByClause\")"); - setState(761); + setState(830); orderByClause(); _localctx->clauses.insert("orderByClause"); break; } case 2: { - setState(764); + setState(833); if (!(!_localctx->clauses.count("partitionByClause"))) throw FailedPredicateException(this, "!$clauses.count(\"partitionByClause\")"); - setState(765); + setState(834); partitionByClause(); _localctx->clauses.insert("partitionByClause"); break; } case 3: { - setState(768); + setState(837); if (!(!_localctx->clauses.count("primaryKeyClause"))) throw FailedPredicateException(this, "!$clauses.count(\"primaryKeyClause\")"); - setState(769); + setState(838); primaryKeyClause(); _localctx->clauses.insert("primaryKeyClause"); break; } case 4: { - setState(772); + setState(841); if (!(!_localctx->clauses.count("sampleByClause"))) throw FailedPredicateException(this, "!$clauses.count(\"sampleByClause\")"); - setState(773); + setState(842); sampleByClause(); _localctx->clauses.insert("sampleByClause"); break; } case 5: { - setState(776); + setState(845); if (!(!_localctx->clauses.count("ttlClause"))) throw FailedPredicateException(this, "!$clauses.count(\"ttlClause\")"); - setState(777); + setState(846); ttlClause(); _localctx->clauses.insert("ttlClause"); break; } case 6: { - setState(780); + setState(849); if (!(!_localctx->clauses.count("settingsClause"))) throw FailedPredicateException(this, "!$clauses.count(\"settingsClause\")"); - setState(781); + setState(850); settingsClause(); _localctx->clauses.insert("settingsClause"); break; @@ -5306,9 +5742,9 @@ ClickHouseParser::EngineClauseContext* ClickHouseParser::engineClause() { } } - setState(788); + setState(857); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 77, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 88, _ctx); } } @@ -5361,11 +5797,11 @@ ClickHouseParser::PartitionByClauseContext* ClickHouseParser::partitionByClause( }); try { enterOuterAlt(_localctx, 1); - setState(789); + setState(858); match(ClickHouseParser::PARTITION); - setState(790); + setState(859); match(ClickHouseParser::BY); - setState(791); + setState(860); columnExpr(0); } @@ -5418,11 +5854,11 @@ ClickHouseParser::PrimaryKeyClauseContext* ClickHouseParser::primaryKeyClause() }); try { enterOuterAlt(_localctx, 1); - setState(793); + setState(862); match(ClickHouseParser::PRIMARY); - setState(794); + setState(863); match(ClickHouseParser::KEY); - setState(795); + setState(864); columnExpr(0); } @@ -5475,11 +5911,11 @@ ClickHouseParser::SampleByClauseContext* ClickHouseParser::sampleByClause() { }); try { enterOuterAlt(_localctx, 1); - setState(797); + setState(866); match(ClickHouseParser::SAMPLE); - setState(798); + setState(867); match(ClickHouseParser::BY); - setState(799); + setState(868); columnExpr(0); } @@ -5541,23 +5977,23 @@ ClickHouseParser::TtlClauseContext* ClickHouseParser::ttlClause() { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(801); + setState(870); match(ClickHouseParser::TTL); - setState(802); + setState(871); ttlExpr(); - setState(807); + setState(876); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 78, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 89, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(803); + setState(872); match(ClickHouseParser::COMMA); - setState(804); + setState(873); ttlExpr(); } - setState(809); + setState(878); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 78, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 89, _ctx); } } @@ -5623,26 +6059,26 @@ ClickHouseParser::EngineExprContext* ClickHouseParser::engineExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(810); + setState(879); match(ClickHouseParser::ENGINE); - setState(812); + setState(881); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::EQ_SINGLE) { - setState(811); + setState(880); match(ClickHouseParser::EQ_SINGLE); } - setState(814); + setState(883); identifierOrNull(); - setState(820); + setState(889); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 81, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 92, _ctx)) { case 1: { - setState(815); + setState(884); match(ClickHouseParser::LPAREN); - setState(817); + setState(886); _errHandler->sync(this); _la = _input->LA(1); @@ -5745,6 +6181,7 @@ ClickHouseParser::EngineExprContext* ClickHouseParser::engineExpr() { | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -5770,10 +6207,10 @@ ClickHouseParser::EngineExprContext* ClickHouseParser::engineExpr() { | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) | (1ULL << (ClickHouseParser::QUARTER - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -5832,16 +6269,16 @@ ClickHouseParser::EngineExprContext* ClickHouseParser::engineExpr() { | (1ULL << (ClickHouseParser::DECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::HEXADECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::STRING_LITERAL - 128)) - | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 194) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 194)) & ((1ULL << (ClickHouseParser::DASH - 194)) - | (1ULL << (ClickHouseParser::DOT - 194)) - | (1ULL << (ClickHouseParser::LBRACKET - 194)) - | (1ULL << (ClickHouseParser::LPAREN - 194)) - | (1ULL << (ClickHouseParser::PLUS - 194)))) != 0)) { - setState(816); + | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 196) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 196)) & ((1ULL << (ClickHouseParser::DASH - 196)) + | (1ULL << (ClickHouseParser::DOT - 196)) + | (1ULL << (ClickHouseParser::LBRACKET - 196)) + | (1ULL << (ClickHouseParser::LPAREN - 196)) + | (1ULL << (ClickHouseParser::PLUS - 196)))) != 0)) { + setState(885); columnExprList(); } - setState(819); + setState(888); match(ClickHouseParser::RPAREN); break; } @@ -5873,6 +6310,25 @@ void ClickHouseParser::TableElementExprContext::copyFrom(TableElementExprContext ParserRuleContext::copyFrom(ctx); } +//----------------- TableElementExprProjectionContext ------------------------------------------------------------------ + +tree::TerminalNode* ClickHouseParser::TableElementExprProjectionContext::PROJECTION() { + return getToken(ClickHouseParser::PROJECTION, 0); +} + +ClickHouseParser::TableProjectionDfntContext* ClickHouseParser::TableElementExprProjectionContext::tableProjectionDfnt() { + return getRuleContext(0); +} + +ClickHouseParser::TableElementExprProjectionContext::TableElementExprProjectionContext(TableElementExprContext *ctx) { copyFrom(ctx); } + + +antlrcpp::Any ClickHouseParser::TableElementExprProjectionContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitTableElementExprProjection(this); + else + return visitor->visitChildren(this); +} //----------------- TableElementExprConstraintContext ------------------------------------------------------------------ tree::TerminalNode* ClickHouseParser::TableElementExprConstraintContext::CONSTRAINT() { @@ -5942,13 +6398,13 @@ ClickHouseParser::TableElementExprContext* ClickHouseParser::tableElementExpr() exitRule(); }); try { - setState(830); + setState(901); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 82, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 93, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(822); + setState(891); tableColumnDfnt(); break; } @@ -5956,13 +6412,13 @@ ClickHouseParser::TableElementExprContext* ClickHouseParser::tableElementExpr() case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(823); + setState(892); match(ClickHouseParser::CONSTRAINT); - setState(824); + setState(893); identifier(); - setState(825); + setState(894); match(ClickHouseParser::CHECK); - setState(826); + setState(895); columnExpr(0); break; } @@ -5970,13 +6426,23 @@ ClickHouseParser::TableElementExprContext* ClickHouseParser::tableElementExpr() case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(828); + setState(897); match(ClickHouseParser::INDEX); - setState(829); + setState(898); tableIndexDfnt(); break; } + case 4: { + _localctx = dynamic_cast(_tracker.createInstance(_localctx)); + enterOuterAlt(_localctx, 4); + setState(899); + match(ClickHouseParser::PROJECTION); + setState(900); + tableProjectionDfnt(); + break; + } + } } @@ -6049,51 +6515,51 @@ ClickHouseParser::TableColumnDfntContext* ClickHouseParser::tableColumnDfnt() { exitRule(); }); try { - setState(864); + setState(935); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 91, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 102, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(832); + setState(903); nestedIdentifier(); - setState(833); + setState(904); columnTypeExpr(); - setState(835); + setState(906); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ALIAS || _la == ClickHouseParser::DEFAULT || _la == ClickHouseParser::MATERIALIZED) { - setState(834); + setState(905); tableColumnPropertyExpr(); } - setState(839); + setState(910); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::COMMENT) { - setState(837); + setState(908); match(ClickHouseParser::COMMENT); - setState(838); + setState(909); match(ClickHouseParser::STRING_LITERAL); } - setState(842); + setState(913); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::CODEC) { - setState(841); + setState(912); codecExpr(); } - setState(846); + setState(917); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TTL) { - setState(844); + setState(915); match(ClickHouseParser::TTL); - setState(845); + setState(916); columnExpr(0); } break; @@ -6101,47 +6567,47 @@ ClickHouseParser::TableColumnDfntContext* ClickHouseParser::tableColumnDfnt() { case 2: { enterOuterAlt(_localctx, 2); - setState(848); + setState(919); nestedIdentifier(); - setState(850); + setState(921); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 87, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 98, _ctx)) { case 1: { - setState(849); + setState(920); columnTypeExpr(); break; } } - setState(852); + setState(923); tableColumnPropertyExpr(); - setState(855); + setState(926); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::COMMENT) { - setState(853); + setState(924); match(ClickHouseParser::COMMENT); - setState(854); + setState(925); match(ClickHouseParser::STRING_LITERAL); } - setState(858); + setState(929); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::CODEC) { - setState(857); + setState(928); codecExpr(); } - setState(862); + setState(933); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TTL) { - setState(860); + setState(931); match(ClickHouseParser::TTL); - setState(861); + setState(932); columnExpr(0); } break; @@ -6204,7 +6670,7 @@ ClickHouseParser::TableColumnPropertyExprContext* ClickHouseParser::tableColumnP }); try { enterOuterAlt(_localctx, 1); - setState(866); + setState(937); _la = _input->LA(1); if (!(_la == ClickHouseParser::ALIAS @@ -6215,7 +6681,7 @@ ClickHouseParser::TableColumnPropertyExprContext* ClickHouseParser::tableColumnP _errHandler->reportMatch(this); consume(); } - setState(867); + setState(938); columnExpr(0); } @@ -6280,17 +6746,17 @@ ClickHouseParser::TableIndexDfntContext* ClickHouseParser::tableIndexDfnt() { }); try { enterOuterAlt(_localctx, 1); - setState(869); + setState(940); nestedIdentifier(); - setState(870); + setState(941); columnExpr(0); - setState(871); + setState(942); match(ClickHouseParser::TYPE); - setState(872); + setState(943); columnTypeExpr(); - setState(873); + setState(944); match(ClickHouseParser::GRANULARITY); - setState(874); + setState(945); match(ClickHouseParser::DECIMAL_LITERAL); } @@ -6303,6 +6769,57 @@ ClickHouseParser::TableIndexDfntContext* ClickHouseParser::tableIndexDfnt() { return _localctx; } +//----------------- TableProjectionDfntContext ------------------------------------------------------------------ + +ClickHouseParser::TableProjectionDfntContext::TableProjectionDfntContext(ParserRuleContext *parent, size_t invokingState) + : ParserRuleContext(parent, invokingState) { +} + +ClickHouseParser::NestedIdentifierContext* ClickHouseParser::TableProjectionDfntContext::nestedIdentifier() { + return getRuleContext(0); +} + +ClickHouseParser::ProjectionSelectStmtContext* ClickHouseParser::TableProjectionDfntContext::projectionSelectStmt() { + return getRuleContext(0); +} + + +size_t ClickHouseParser::TableProjectionDfntContext::getRuleIndex() const { + return ClickHouseParser::RuleTableProjectionDfnt; +} + + +antlrcpp::Any ClickHouseParser::TableProjectionDfntContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitTableProjectionDfnt(this); + else + return visitor->visitChildren(this); +} + +ClickHouseParser::TableProjectionDfntContext* ClickHouseParser::tableProjectionDfnt() { + TableProjectionDfntContext *_localctx = _tracker.createInstance(_ctx, getState()); + enterRule(_localctx, 72, ClickHouseParser::RuleTableProjectionDfnt); + + auto onExit = finally([=] { + exitRule(); + }); + try { + enterOuterAlt(_localctx, 1); + setState(947); + nestedIdentifier(); + setState(948); + projectionSelectStmt(); + + } + catch (RecognitionException &e) { + _errHandler->reportError(this, e); + _localctx->exception = std::current_exception(); + _errHandler->recover(this, _localctx->exception); + } + + return _localctx; +} + //----------------- CodecExprContext ------------------------------------------------------------------ ClickHouseParser::CodecExprContext::CodecExprContext(ParserRuleContext *parent, size_t invokingState) @@ -6352,7 +6869,7 @@ antlrcpp::Any ClickHouseParser::CodecExprContext::accept(tree::ParseTreeVisitor ClickHouseParser::CodecExprContext* ClickHouseParser::codecExpr() { CodecExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 72, ClickHouseParser::RuleCodecExpr); + enterRule(_localctx, 74, ClickHouseParser::RuleCodecExpr); size_t _la = 0; auto onExit = finally([=] { @@ -6360,25 +6877,25 @@ ClickHouseParser::CodecExprContext* ClickHouseParser::codecExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(876); + setState(950); match(ClickHouseParser::CODEC); - setState(877); + setState(951); match(ClickHouseParser::LPAREN); - setState(878); + setState(952); codecArgExpr(); - setState(883); + setState(957); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(879); + setState(953); match(ClickHouseParser::COMMA); - setState(880); + setState(954); codecArgExpr(); - setState(885); + setState(959); _errHandler->sync(this); _la = _input->LA(1); } - setState(886); + setState(960); match(ClickHouseParser::RPAREN); } @@ -6428,7 +6945,7 @@ antlrcpp::Any ClickHouseParser::CodecArgExprContext::accept(tree::ParseTreeVisit ClickHouseParser::CodecArgExprContext* ClickHouseParser::codecArgExpr() { CodecArgExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 74, ClickHouseParser::RuleCodecArgExpr); + enterRule(_localctx, 76, ClickHouseParser::RuleCodecArgExpr); size_t _la = 0; auto onExit = finally([=] { @@ -6436,16 +6953,16 @@ ClickHouseParser::CodecArgExprContext* ClickHouseParser::codecArgExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(888); + setState(962); identifier(); - setState(894); + setState(968); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::LPAREN) { - setState(889); + setState(963); match(ClickHouseParser::LPAREN); - setState(891); + setState(965); _errHandler->sync(this); _la = _input->LA(1); @@ -6548,6 +7065,7 @@ ClickHouseParser::CodecArgExprContext* ClickHouseParser::codecArgExpr() { | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -6573,10 +7091,10 @@ ClickHouseParser::CodecArgExprContext* ClickHouseParser::codecArgExpr() { | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) | (1ULL << (ClickHouseParser::QUARTER - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -6635,16 +7153,16 @@ ClickHouseParser::CodecArgExprContext* ClickHouseParser::codecArgExpr() { | (1ULL << (ClickHouseParser::DECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::HEXADECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::STRING_LITERAL - 128)) - | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 194) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 194)) & ((1ULL << (ClickHouseParser::DASH - 194)) - | (1ULL << (ClickHouseParser::DOT - 194)) - | (1ULL << (ClickHouseParser::LBRACKET - 194)) - | (1ULL << (ClickHouseParser::LPAREN - 194)) - | (1ULL << (ClickHouseParser::PLUS - 194)))) != 0)) { - setState(890); + | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 196) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 196)) & ((1ULL << (ClickHouseParser::DASH - 196)) + | (1ULL << (ClickHouseParser::DOT - 196)) + | (1ULL << (ClickHouseParser::LBRACKET - 196)) + | (1ULL << (ClickHouseParser::LPAREN - 196)) + | (1ULL << (ClickHouseParser::PLUS - 196)))) != 0)) { + setState(964); columnExprList(); } - setState(893); + setState(967); match(ClickHouseParser::RPAREN); } @@ -6703,41 +7221,41 @@ antlrcpp::Any ClickHouseParser::TtlExprContext::accept(tree::ParseTreeVisitor *v ClickHouseParser::TtlExprContext* ClickHouseParser::ttlExpr() { TtlExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 76, ClickHouseParser::RuleTtlExpr); + enterRule(_localctx, 78, ClickHouseParser::RuleTtlExpr); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(896); + setState(970); columnExpr(0); - setState(904); + setState(978); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 95, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 106, _ctx)) { case 1: { - setState(897); + setState(971); match(ClickHouseParser::DELETE); break; } case 2: { - setState(898); + setState(972); match(ClickHouseParser::TO); - setState(899); + setState(973); match(ClickHouseParser::DISK); - setState(900); + setState(974); match(ClickHouseParser::STRING_LITERAL); break; } case 3: { - setState(901); + setState(975); match(ClickHouseParser::TO); - setState(902); + setState(976); match(ClickHouseParser::VOLUME); - setState(903); + setState(977); match(ClickHouseParser::STRING_LITERAL); break; } @@ -6791,7 +7309,7 @@ antlrcpp::Any ClickHouseParser::DescribeStmtContext::accept(tree::ParseTreeVisit ClickHouseParser::DescribeStmtContext* ClickHouseParser::describeStmt() { DescribeStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 78, ClickHouseParser::RuleDescribeStmt); + enterRule(_localctx, 80, ClickHouseParser::RuleDescribeStmt); size_t _la = 0; auto onExit = finally([=] { @@ -6799,7 +7317,7 @@ ClickHouseParser::DescribeStmtContext* ClickHouseParser::describeStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(906); + setState(980); _la = _input->LA(1); if (!(_la == ClickHouseParser::DESC @@ -6810,18 +7328,18 @@ ClickHouseParser::DescribeStmtContext* ClickHouseParser::describeStmt() { _errHandler->reportMatch(this); consume(); } - setState(908); + setState(982); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 96, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 107, _ctx)) { case 1: { - setState(907); + setState(981); match(ClickHouseParser::TABLE); break; } } - setState(910); + setState(984); tableExpr(0); } @@ -6949,20 +7467,20 @@ antlrcpp::Any ClickHouseParser::DropTableStmtContext::accept(tree::ParseTreeVisi } ClickHouseParser::DropStmtContext* ClickHouseParser::dropStmt() { DropStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 80, ClickHouseParser::RuleDropStmt); + enterRule(_localctx, 82, ClickHouseParser::RuleDropStmt); size_t _la = 0; auto onExit = finally([=] { exitRule(); }); try { - setState(943); + setState(1017); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 104, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 115, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(912); + setState(986); _la = _input->LA(1); if (!(_la == ClickHouseParser::DETACH @@ -6973,29 +7491,29 @@ ClickHouseParser::DropStmtContext* ClickHouseParser::dropStmt() { _errHandler->reportMatch(this); consume(); } - setState(913); + setState(987); match(ClickHouseParser::DATABASE); - setState(916); + setState(990); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 97, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 108, _ctx)) { case 1: { - setState(914); + setState(988); match(ClickHouseParser::IF); - setState(915); + setState(989); match(ClickHouseParser::EXISTS); break; } } - setState(918); + setState(992); databaseIdentifier(); - setState(920); + setState(994); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(919); + setState(993); clusterClause(); } break; @@ -7004,7 +7522,7 @@ ClickHouseParser::DropStmtContext* ClickHouseParser::dropStmt() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(922); + setState(996); _la = _input->LA(1); if (!(_la == ClickHouseParser::DETACH @@ -7015,32 +7533,32 @@ ClickHouseParser::DropStmtContext* ClickHouseParser::dropStmt() { _errHandler->reportMatch(this); consume(); } - setState(929); + setState(1003); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::DICTIONARY: { - setState(923); + setState(997); match(ClickHouseParser::DICTIONARY); break; } case ClickHouseParser::TABLE: case ClickHouseParser::TEMPORARY: { - setState(925); + setState(999); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TEMPORARY) { - setState(924); + setState(998); match(ClickHouseParser::TEMPORARY); } - setState(927); + setState(1001); match(ClickHouseParser::TABLE); break; } case ClickHouseParser::VIEW: { - setState(928); + setState(1002); match(ClickHouseParser::VIEW); break; } @@ -7048,37 +7566,37 @@ ClickHouseParser::DropStmtContext* ClickHouseParser::dropStmt() { default: throw NoViableAltException(this); } - setState(933); + setState(1007); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 101, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 112, _ctx)) { case 1: { - setState(931); + setState(1005); match(ClickHouseParser::IF); - setState(932); + setState(1006); match(ClickHouseParser::EXISTS); break; } } - setState(935); + setState(1009); tableIdentifier(); - setState(937); + setState(1011); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(936); + setState(1010); clusterClause(); } - setState(941); + setState(1015); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::NO) { - setState(939); + setState(1013); match(ClickHouseParser::NO); - setState(940); + setState(1014); match(ClickHouseParser::DELAY); } break; @@ -7171,24 +7689,24 @@ antlrcpp::Any ClickHouseParser::ExistsDatabaseStmtContext::accept(tree::ParseTre } ClickHouseParser::ExistsStmtContext* ClickHouseParser::existsStmt() { ExistsStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 82, ClickHouseParser::RuleExistsStmt); + enterRule(_localctx, 84, ClickHouseParser::RuleExistsStmt); size_t _la = 0; auto onExit = finally([=] { exitRule(); }); try { - setState(958); + setState(1032); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 107, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 118, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(945); + setState(1019); match(ClickHouseParser::EXISTS); - setState(946); + setState(1020); match(ClickHouseParser::DATABASE); - setState(947); + setState(1021); databaseIdentifier(); break; } @@ -7196,40 +7714,40 @@ ClickHouseParser::ExistsStmtContext* ClickHouseParser::existsStmt() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(948); + setState(1022); match(ClickHouseParser::EXISTS); - setState(955); + setState(1029); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 106, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 117, _ctx)) { case 1: { - setState(949); + setState(1023); match(ClickHouseParser::DICTIONARY); break; } case 2: { - setState(951); + setState(1025); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TEMPORARY) { - setState(950); + setState(1024); match(ClickHouseParser::TEMPORARY); } - setState(953); + setState(1027); match(ClickHouseParser::TABLE); break; } case 3: { - setState(954); + setState(1028); match(ClickHouseParser::VIEW); break; } } - setState(957); + setState(1031); tableIdentifier(); break; } @@ -7279,18 +7797,18 @@ antlrcpp::Any ClickHouseParser::ExplainStmtContext::accept(tree::ParseTreeVisito ClickHouseParser::ExplainStmtContext* ClickHouseParser::explainStmt() { ExplainStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 84, ClickHouseParser::RuleExplainStmt); + enterRule(_localctx, 86, ClickHouseParser::RuleExplainStmt); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(960); + setState(1034); match(ClickHouseParser::EXPLAIN); - setState(961); + setState(1035); match(ClickHouseParser::SYNTAX); - setState(962); + setState(1036); query(); } @@ -7356,58 +7874,58 @@ antlrcpp::Any ClickHouseParser::InsertStmtContext::accept(tree::ParseTreeVisitor ClickHouseParser::InsertStmtContext* ClickHouseParser::insertStmt() { InsertStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 86, ClickHouseParser::RuleInsertStmt); + enterRule(_localctx, 88, ClickHouseParser::RuleInsertStmt); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(964); + setState(1038); match(ClickHouseParser::INSERT); - setState(965); + setState(1039); match(ClickHouseParser::INTO); - setState(967); + setState(1041); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 108, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 119, _ctx)) { case 1: { - setState(966); + setState(1040); match(ClickHouseParser::TABLE); break; } } - setState(972); + setState(1046); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 109, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 120, _ctx)) { case 1: { - setState(969); + setState(1043); tableIdentifier(); break; } case 2: { - setState(970); + setState(1044); match(ClickHouseParser::FUNCTION); - setState(971); + setState(1045); tableFunctionExpr(); break; } } - setState(975); + setState(1049); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 110, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 121, _ctx)) { case 1: { - setState(974); + setState(1048); columnsClause(); break; } } - setState(977); + setState(1051); dataClause(); } @@ -7465,7 +7983,7 @@ antlrcpp::Any ClickHouseParser::ColumnsClauseContext::accept(tree::ParseTreeVisi ClickHouseParser::ColumnsClauseContext* ClickHouseParser::columnsClause() { ColumnsClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 88, ClickHouseParser::RuleColumnsClause); + enterRule(_localctx, 90, ClickHouseParser::RuleColumnsClause); size_t _la = 0; auto onExit = finally([=] { @@ -7473,23 +7991,23 @@ ClickHouseParser::ColumnsClauseContext* ClickHouseParser::columnsClause() { }); try { enterOuterAlt(_localctx, 1); - setState(979); + setState(1053); match(ClickHouseParser::LPAREN); - setState(980); + setState(1054); nestedIdentifier(); - setState(985); + setState(1059); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(981); + setState(1055); match(ClickHouseParser::COMMA); - setState(982); + setState(1056); nestedIdentifier(); - setState(987); + setState(1061); _errHandler->sync(this); _la = _input->LA(1); } - setState(988); + setState(1062); match(ClickHouseParser::RPAREN); } @@ -7576,22 +8094,22 @@ antlrcpp::Any ClickHouseParser::DataClauseSelectContext::accept(tree::ParseTreeV } ClickHouseParser::DataClauseContext* ClickHouseParser::dataClause() { DataClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 90, ClickHouseParser::RuleDataClause); + enterRule(_localctx, 92, ClickHouseParser::RuleDataClause); size_t _la = 0; auto onExit = finally([=] { exitRule(); }); try { - setState(999); + setState(1073); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::FORMAT: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(990); + setState(1064); match(ClickHouseParser::FORMAT); - setState(991); + setState(1065); identifier(); break; } @@ -7599,7 +8117,7 @@ ClickHouseParser::DataClauseContext* ClickHouseParser::dataClause() { case ClickHouseParser::VALUES: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(992); + setState(1066); match(ClickHouseParser::VALUES); break; } @@ -7609,17 +8127,17 @@ ClickHouseParser::DataClauseContext* ClickHouseParser::dataClause() { case ClickHouseParser::LPAREN: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(993); + setState(1067); selectUnionStmt(); - setState(995); + setState(1069); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::SEMICOLON) { - setState(994); + setState(1068); match(ClickHouseParser::SEMICOLON); } - setState(997); + setState(1071); match(ClickHouseParser::EOF); break; } @@ -7694,7 +8212,7 @@ antlrcpp::Any ClickHouseParser::KillMutationStmtContext::accept(tree::ParseTreeV } ClickHouseParser::KillStmtContext* ClickHouseParser::killStmt() { KillStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 92, ClickHouseParser::RuleKillStmt); + enterRule(_localctx, 94, ClickHouseParser::RuleKillStmt); size_t _la = 0; auto onExit = finally([=] { @@ -7703,28 +8221,28 @@ ClickHouseParser::KillStmtContext* ClickHouseParser::killStmt() { try { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1001); + setState(1075); match(ClickHouseParser::KILL); - setState(1002); + setState(1076); match(ClickHouseParser::MUTATION); - setState(1004); + setState(1078); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(1003); + setState(1077); clusterClause(); } - setState(1006); + setState(1080); whereClause(); - setState(1008); + setState(1082); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ASYNC || _la == ClickHouseParser::SYNC || _la == ClickHouseParser::TEST) { - setState(1007); + setState(1081); _la = _input->LA(1); if (!(_la == ClickHouseParser::ASYNC || _la == ClickHouseParser::SYNC @@ -7796,7 +8314,7 @@ antlrcpp::Any ClickHouseParser::OptimizeStmtContext::accept(tree::ParseTreeVisit ClickHouseParser::OptimizeStmtContext* ClickHouseParser::optimizeStmt() { OptimizeStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 94, ClickHouseParser::RuleOptimizeStmt); + enterRule(_localctx, 96, ClickHouseParser::RuleOptimizeStmt); size_t _la = 0; auto onExit = finally([=] { @@ -7804,42 +8322,42 @@ ClickHouseParser::OptimizeStmtContext* ClickHouseParser::optimizeStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1010); + setState(1084); match(ClickHouseParser::OPTIMIZE); - setState(1011); + setState(1085); match(ClickHouseParser::TABLE); - setState(1012); + setState(1086); tableIdentifier(); - setState(1014); + setState(1088); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(1013); + setState(1087); clusterClause(); } - setState(1017); + setState(1091); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::PARTITION) { - setState(1016); + setState(1090); partitionClause(); } - setState(1020); + setState(1094); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::FINAL) { - setState(1019); + setState(1093); match(ClickHouseParser::FINAL); } - setState(1023); + setState(1097); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::DEDUPLICATE) { - setState(1022); + setState(1096); match(ClickHouseParser::DEDUPLICATE); } @@ -7910,7 +8428,7 @@ antlrcpp::Any ClickHouseParser::RenameStmtContext::accept(tree::ParseTreeVisitor ClickHouseParser::RenameStmtContext* ClickHouseParser::renameStmt() { RenameStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 96, ClickHouseParser::RuleRenameStmt); + enterRule(_localctx, 98, ClickHouseParser::RuleRenameStmt); size_t _la = 0; auto onExit = finally([=] { @@ -7918,38 +8436,38 @@ ClickHouseParser::RenameStmtContext* ClickHouseParser::renameStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1025); + setState(1099); match(ClickHouseParser::RENAME); - setState(1026); + setState(1100); match(ClickHouseParser::TABLE); - setState(1027); + setState(1101); tableIdentifier(); - setState(1028); + setState(1102); match(ClickHouseParser::TO); - setState(1029); + setState(1103); tableIdentifier(); - setState(1037); + setState(1111); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1030); + setState(1104); match(ClickHouseParser::COMMA); - setState(1031); + setState(1105); tableIdentifier(); - setState(1032); + setState(1106); match(ClickHouseParser::TO); - setState(1033); + setState(1107); tableIdentifier(); - setState(1039); + setState(1113); _errHandler->sync(this); _la = _input->LA(1); } - setState(1041); + setState(1115); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(1040); + setState(1114); clusterClause(); } @@ -7963,6 +8481,106 @@ ClickHouseParser::RenameStmtContext* ClickHouseParser::renameStmt() { return _localctx; } +//----------------- ProjectionSelectStmtContext ------------------------------------------------------------------ + +ClickHouseParser::ProjectionSelectStmtContext::ProjectionSelectStmtContext(ParserRuleContext *parent, size_t invokingState) + : ParserRuleContext(parent, invokingState) { +} + +tree::TerminalNode* ClickHouseParser::ProjectionSelectStmtContext::LPAREN() { + return getToken(ClickHouseParser::LPAREN, 0); +} + +tree::TerminalNode* ClickHouseParser::ProjectionSelectStmtContext::SELECT() { + return getToken(ClickHouseParser::SELECT, 0); +} + +ClickHouseParser::ColumnExprListContext* ClickHouseParser::ProjectionSelectStmtContext::columnExprList() { + return getRuleContext(0); +} + +tree::TerminalNode* ClickHouseParser::ProjectionSelectStmtContext::RPAREN() { + return getToken(ClickHouseParser::RPAREN, 0); +} + +ClickHouseParser::WithClauseContext* ClickHouseParser::ProjectionSelectStmtContext::withClause() { + return getRuleContext(0); +} + +ClickHouseParser::GroupByClauseContext* ClickHouseParser::ProjectionSelectStmtContext::groupByClause() { + return getRuleContext(0); +} + +ClickHouseParser::ProjectionOrderByClauseContext* ClickHouseParser::ProjectionSelectStmtContext::projectionOrderByClause() { + return getRuleContext(0); +} + + +size_t ClickHouseParser::ProjectionSelectStmtContext::getRuleIndex() const { + return ClickHouseParser::RuleProjectionSelectStmt; +} + + +antlrcpp::Any ClickHouseParser::ProjectionSelectStmtContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitProjectionSelectStmt(this); + else + return visitor->visitChildren(this); +} + +ClickHouseParser::ProjectionSelectStmtContext* ClickHouseParser::projectionSelectStmt() { + ProjectionSelectStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); + enterRule(_localctx, 100, ClickHouseParser::RuleProjectionSelectStmt); + size_t _la = 0; + + auto onExit = finally([=] { + exitRule(); + }); + try { + enterOuterAlt(_localctx, 1); + setState(1117); + match(ClickHouseParser::LPAREN); + setState(1119); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::WITH) { + setState(1118); + withClause(); + } + setState(1121); + match(ClickHouseParser::SELECT); + setState(1122); + columnExprList(); + setState(1124); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::GROUP) { + setState(1123); + groupByClause(); + } + setState(1127); + _errHandler->sync(this); + + _la = _input->LA(1); + if (_la == ClickHouseParser::ORDER) { + setState(1126); + projectionOrderByClause(); + } + setState(1129); + match(ClickHouseParser::RPAREN); + + } + catch (RecognitionException &e) { + _errHandler->reportError(this, e); + _localctx->exception = std::current_exception(); + _errHandler->recover(this, _localctx->exception); + } + + return _localctx; +} + //----------------- SelectUnionStmtContext ------------------------------------------------------------------ ClickHouseParser::SelectUnionStmtContext::SelectUnionStmtContext(ParserRuleContext *parent, size_t invokingState) @@ -8008,7 +8626,7 @@ antlrcpp::Any ClickHouseParser::SelectUnionStmtContext::accept(tree::ParseTreeVi ClickHouseParser::SelectUnionStmtContext* ClickHouseParser::selectUnionStmt() { SelectUnionStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 98, ClickHouseParser::RuleSelectUnionStmt); + enterRule(_localctx, 102, ClickHouseParser::RuleSelectUnionStmt); size_t _la = 0; auto onExit = finally([=] { @@ -8016,19 +8634,19 @@ ClickHouseParser::SelectUnionStmtContext* ClickHouseParser::selectUnionStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1043); + setState(1131); selectStmtWithParens(); - setState(1049); + setState(1137); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::UNION) { - setState(1044); + setState(1132); match(ClickHouseParser::UNION); - setState(1045); + setState(1133); match(ClickHouseParser::ALL); - setState(1046); + setState(1134); selectStmtWithParens(); - setState(1051); + setState(1139); _errHandler->sync(this); _la = _input->LA(1); } @@ -8080,30 +8698,30 @@ antlrcpp::Any ClickHouseParser::SelectStmtWithParensContext::accept(tree::ParseT ClickHouseParser::SelectStmtWithParensContext* ClickHouseParser::selectStmtWithParens() { SelectStmtWithParensContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 100, ClickHouseParser::RuleSelectStmtWithParens); + enterRule(_localctx, 104, ClickHouseParser::RuleSelectStmtWithParens); auto onExit = finally([=] { exitRule(); }); try { - setState(1057); + setState(1145); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::SELECT: case ClickHouseParser::WITH: { enterOuterAlt(_localctx, 1); - setState(1052); + setState(1140); selectStmt(); break; } case ClickHouseParser::LPAREN: { enterOuterAlt(_localctx, 2); - setState(1053); + setState(1141); match(ClickHouseParser::LPAREN); - setState(1054); + setState(1142); selectUnionStmt(); - setState(1055); + setState(1143); match(ClickHouseParser::RPAREN); break; } @@ -8223,7 +8841,7 @@ antlrcpp::Any ClickHouseParser::SelectStmtContext::accept(tree::ParseTreeVisitor ClickHouseParser::SelectStmtContext* ClickHouseParser::selectStmt() { SelectStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 102, ClickHouseParser::RuleSelectStmt); + enterRule(_localctx, 106, ClickHouseParser::RuleSelectStmt); size_t _la = 0; auto onExit = finally([=] { @@ -8231,90 +8849,90 @@ ClickHouseParser::SelectStmtContext* ClickHouseParser::selectStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1060); + setState(1148); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::WITH) { - setState(1059); + setState(1147); withClause(); } - setState(1062); + setState(1150); match(ClickHouseParser::SELECT); - setState(1064); + setState(1152); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 125, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 139, _ctx)) { case 1: { - setState(1063); + setState(1151); match(ClickHouseParser::DISTINCT); break; } } - setState(1067); + setState(1155); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 126, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 140, _ctx)) { case 1: { - setState(1066); + setState(1154); topClause(); break; } } - setState(1069); + setState(1157); columnExprList(); - setState(1071); + setState(1159); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::FROM) { - setState(1070); + setState(1158); fromClause(); } - setState(1074); + setState(1162); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ARRAY || _la == ClickHouseParser::INNER || _la == ClickHouseParser::LEFT) { - setState(1073); + setState(1161); arrayJoinClause(); } - setState(1077); + setState(1165); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::PREWHERE) { - setState(1076); + setState(1164); prewhereClause(); } - setState(1080); + setState(1168); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::WHERE) { - setState(1079); + setState(1167); whereClause(); } - setState(1083); + setState(1171); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::GROUP) { - setState(1082); + setState(1170); groupByClause(); } - setState(1087); + setState(1175); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 132, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 146, _ctx)) { case 1: { - setState(1085); + setState(1173); match(ClickHouseParser::WITH); - setState(1086); + setState(1174); _la = _input->LA(1); if (!(_la == ClickHouseParser::CUBE || _la == ClickHouseParser::ROLLUP)) { _errHandler->recoverInline(this); @@ -8327,57 +8945,57 @@ ClickHouseParser::SelectStmtContext* ClickHouseParser::selectStmt() { } } - setState(1091); + setState(1179); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::WITH) { - setState(1089); + setState(1177); match(ClickHouseParser::WITH); - setState(1090); + setState(1178); match(ClickHouseParser::TOTALS); } - setState(1094); + setState(1182); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::HAVING) { - setState(1093); + setState(1181); havingClause(); } - setState(1097); + setState(1185); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ORDER) { - setState(1096); + setState(1184); orderByClause(); } - setState(1100); + setState(1188); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 136, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 150, _ctx)) { case 1: { - setState(1099); + setState(1187); limitByClause(); break; } } - setState(1103); + setState(1191); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::LIMIT) { - setState(1102); + setState(1190); limitClause(); } - setState(1106); + setState(1194); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::SETTINGS) { - setState(1105); + setState(1193); settingsClause(); } @@ -8420,16 +9038,16 @@ antlrcpp::Any ClickHouseParser::WithClauseContext::accept(tree::ParseTreeVisitor ClickHouseParser::WithClauseContext* ClickHouseParser::withClause() { WithClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 104, ClickHouseParser::RuleWithClause); + enterRule(_localctx, 108, ClickHouseParser::RuleWithClause); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1108); + setState(1196); match(ClickHouseParser::WITH); - setState(1109); + setState(1197); columnExprList(); } @@ -8479,25 +9097,25 @@ antlrcpp::Any ClickHouseParser::TopClauseContext::accept(tree::ParseTreeVisitor ClickHouseParser::TopClauseContext* ClickHouseParser::topClause() { TopClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 106, ClickHouseParser::RuleTopClause); + enterRule(_localctx, 110, ClickHouseParser::RuleTopClause); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1111); + setState(1199); match(ClickHouseParser::TOP); - setState(1112); + setState(1200); match(ClickHouseParser::DECIMAL_LITERAL); - setState(1115); + setState(1203); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 139, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 153, _ctx)) { case 1: { - setState(1113); + setState(1201); match(ClickHouseParser::WITH); - setState(1114); + setState(1202); match(ClickHouseParser::TIES); break; } @@ -8543,16 +9161,16 @@ antlrcpp::Any ClickHouseParser::FromClauseContext::accept(tree::ParseTreeVisitor ClickHouseParser::FromClauseContext* ClickHouseParser::fromClause() { FromClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 108, ClickHouseParser::RuleFromClause); + enterRule(_localctx, 112, ClickHouseParser::RuleFromClause); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1117); + setState(1205); match(ClickHouseParser::FROM); - setState(1118); + setState(1206); joinExpr(0); } @@ -8606,7 +9224,7 @@ antlrcpp::Any ClickHouseParser::ArrayJoinClauseContext::accept(tree::ParseTreeVi ClickHouseParser::ArrayJoinClauseContext* ClickHouseParser::arrayJoinClause() { ArrayJoinClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 110, ClickHouseParser::RuleArrayJoinClause); + enterRule(_localctx, 114, ClickHouseParser::RuleArrayJoinClause); size_t _la = 0; auto onExit = finally([=] { @@ -8614,14 +9232,14 @@ ClickHouseParser::ArrayJoinClauseContext* ClickHouseParser::arrayJoinClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1121); + setState(1209); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::INNER || _la == ClickHouseParser::LEFT) { - setState(1120); + setState(1208); _la = _input->LA(1); if (!(_la == ClickHouseParser::INNER @@ -8633,11 +9251,11 @@ ClickHouseParser::ArrayJoinClauseContext* ClickHouseParser::arrayJoinClause() { consume(); } } - setState(1123); + setState(1211); match(ClickHouseParser::ARRAY); - setState(1124); + setState(1212); match(ClickHouseParser::JOIN); - setState(1125); + setState(1213); columnExprList(); } @@ -8679,16 +9297,16 @@ antlrcpp::Any ClickHouseParser::PrewhereClauseContext::accept(tree::ParseTreeVis ClickHouseParser::PrewhereClauseContext* ClickHouseParser::prewhereClause() { PrewhereClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 112, ClickHouseParser::RulePrewhereClause); + enterRule(_localctx, 116, ClickHouseParser::RulePrewhereClause); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1127); + setState(1215); match(ClickHouseParser::PREWHERE); - setState(1128); + setState(1216); columnExpr(0); } @@ -8730,16 +9348,16 @@ antlrcpp::Any ClickHouseParser::WhereClauseContext::accept(tree::ParseTreeVisito ClickHouseParser::WhereClauseContext* ClickHouseParser::whereClause() { WhereClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 114, ClickHouseParser::RuleWhereClause); + enterRule(_localctx, 118, ClickHouseParser::RuleWhereClause); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1130); + setState(1218); match(ClickHouseParser::WHERE); - setState(1131); + setState(1219); columnExpr(0); } @@ -8801,7 +9419,7 @@ antlrcpp::Any ClickHouseParser::GroupByClauseContext::accept(tree::ParseTreeVisi ClickHouseParser::GroupByClauseContext* ClickHouseParser::groupByClause() { GroupByClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 116, ClickHouseParser::RuleGroupByClause); + enterRule(_localctx, 120, ClickHouseParser::RuleGroupByClause); size_t _la = 0; auto onExit = finally([=] { @@ -8809,15 +9427,15 @@ ClickHouseParser::GroupByClauseContext* ClickHouseParser::groupByClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1133); + setState(1221); match(ClickHouseParser::GROUP); - setState(1134); + setState(1222); match(ClickHouseParser::BY); - setState(1141); + setState(1229); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 141, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 155, _ctx)) { case 1: { - setState(1135); + setState(1223); _la = _input->LA(1); if (!(_la == ClickHouseParser::CUBE || _la == ClickHouseParser::ROLLUP)) { _errHandler->recoverInline(this); @@ -8826,17 +9444,17 @@ ClickHouseParser::GroupByClauseContext* ClickHouseParser::groupByClause() { _errHandler->reportMatch(this); consume(); } - setState(1136); + setState(1224); match(ClickHouseParser::LPAREN); - setState(1137); + setState(1225); columnExprList(); - setState(1138); + setState(1226); match(ClickHouseParser::RPAREN); break; } case 2: { - setState(1140); + setState(1228); columnExprList(); break; } @@ -8882,16 +9500,16 @@ antlrcpp::Any ClickHouseParser::HavingClauseContext::accept(tree::ParseTreeVisit ClickHouseParser::HavingClauseContext* ClickHouseParser::havingClause() { HavingClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 118, ClickHouseParser::RuleHavingClause); + enterRule(_localctx, 122, ClickHouseParser::RuleHavingClause); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1143); + setState(1231); match(ClickHouseParser::HAVING); - setState(1144); + setState(1232); columnExpr(0); } @@ -8937,18 +9555,18 @@ antlrcpp::Any ClickHouseParser::OrderByClauseContext::accept(tree::ParseTreeVisi ClickHouseParser::OrderByClauseContext* ClickHouseParser::orderByClause() { OrderByClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 120, ClickHouseParser::RuleOrderByClause); + enterRule(_localctx, 124, ClickHouseParser::RuleOrderByClause); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1146); + setState(1234); match(ClickHouseParser::ORDER); - setState(1147); + setState(1235); match(ClickHouseParser::BY); - setState(1148); + setState(1236); orderExprList(); } @@ -8961,6 +9579,63 @@ ClickHouseParser::OrderByClauseContext* ClickHouseParser::orderByClause() { return _localctx; } +//----------------- ProjectionOrderByClauseContext ------------------------------------------------------------------ + +ClickHouseParser::ProjectionOrderByClauseContext::ProjectionOrderByClauseContext(ParserRuleContext *parent, size_t invokingState) + : ParserRuleContext(parent, invokingState) { +} + +tree::TerminalNode* ClickHouseParser::ProjectionOrderByClauseContext::ORDER() { + return getToken(ClickHouseParser::ORDER, 0); +} + +tree::TerminalNode* ClickHouseParser::ProjectionOrderByClauseContext::BY() { + return getToken(ClickHouseParser::BY, 0); +} + +ClickHouseParser::ColumnExprListContext* ClickHouseParser::ProjectionOrderByClauseContext::columnExprList() { + return getRuleContext(0); +} + + +size_t ClickHouseParser::ProjectionOrderByClauseContext::getRuleIndex() const { + return ClickHouseParser::RuleProjectionOrderByClause; +} + + +antlrcpp::Any ClickHouseParser::ProjectionOrderByClauseContext::accept(tree::ParseTreeVisitor *visitor) { + if (auto parserVisitor = dynamic_cast(visitor)) + return parserVisitor->visitProjectionOrderByClause(this); + else + return visitor->visitChildren(this); +} + +ClickHouseParser::ProjectionOrderByClauseContext* ClickHouseParser::projectionOrderByClause() { + ProjectionOrderByClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); + enterRule(_localctx, 126, ClickHouseParser::RuleProjectionOrderByClause); + + auto onExit = finally([=] { + exitRule(); + }); + try { + enterOuterAlt(_localctx, 1); + setState(1238); + match(ClickHouseParser::ORDER); + setState(1239); + match(ClickHouseParser::BY); + setState(1240); + columnExprList(); + + } + catch (RecognitionException &e) { + _errHandler->reportError(this, e); + _localctx->exception = std::current_exception(); + _errHandler->recover(this, _localctx->exception); + } + + return _localctx; +} + //----------------- LimitByClauseContext ------------------------------------------------------------------ ClickHouseParser::LimitByClauseContext::LimitByClauseContext(ParserRuleContext *parent, size_t invokingState) @@ -8998,20 +9673,20 @@ antlrcpp::Any ClickHouseParser::LimitByClauseContext::accept(tree::ParseTreeVisi ClickHouseParser::LimitByClauseContext* ClickHouseParser::limitByClause() { LimitByClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 122, ClickHouseParser::RuleLimitByClause); + enterRule(_localctx, 128, ClickHouseParser::RuleLimitByClause); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1150); + setState(1242); match(ClickHouseParser::LIMIT); - setState(1151); + setState(1243); limitExpr(); - setState(1152); + setState(1244); match(ClickHouseParser::BY); - setState(1153); + setState(1245); columnExprList(); } @@ -9061,7 +9736,7 @@ antlrcpp::Any ClickHouseParser::LimitClauseContext::accept(tree::ParseTreeVisito ClickHouseParser::LimitClauseContext* ClickHouseParser::limitClause() { LimitClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 124, ClickHouseParser::RuleLimitClause); + enterRule(_localctx, 130, ClickHouseParser::RuleLimitClause); size_t _la = 0; auto onExit = finally([=] { @@ -9069,18 +9744,18 @@ ClickHouseParser::LimitClauseContext* ClickHouseParser::limitClause() { }); try { enterOuterAlt(_localctx, 1); - setState(1155); + setState(1247); match(ClickHouseParser::LIMIT); - setState(1156); + setState(1248); limitExpr(); - setState(1159); + setState(1251); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::WITH) { - setState(1157); + setState(1249); match(ClickHouseParser::WITH); - setState(1158); + setState(1250); match(ClickHouseParser::TIES); } @@ -9123,16 +9798,16 @@ antlrcpp::Any ClickHouseParser::SettingsClauseContext::accept(tree::ParseTreeVis ClickHouseParser::SettingsClauseContext* ClickHouseParser::settingsClause() { SettingsClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 126, ClickHouseParser::RuleSettingsClause); + enterRule(_localctx, 132, ClickHouseParser::RuleSettingsClause); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1161); + setState(1253); match(ClickHouseParser::SETTINGS); - setState(1162); + setState(1254); settingExprList(); } @@ -9279,8 +9954,8 @@ ClickHouseParser::JoinExprContext* ClickHouseParser::joinExpr(int precedence) { ClickHouseParser::JoinExprContext *_localctx = _tracker.createInstance(_ctx, parentState); ClickHouseParser::JoinExprContext *previousContext = _localctx; (void)previousContext; // Silence compiler, in case the context is not used by generated code. - size_t startState = 128; - enterRecursionRule(_localctx, 128, ClickHouseParser::RuleJoinExpr, precedence); + size_t startState = 134; + enterRecursionRule(_localctx, 134, ClickHouseParser::RuleJoinExpr, precedence); size_t _la = 0; @@ -9290,33 +9965,33 @@ ClickHouseParser::JoinExprContext* ClickHouseParser::joinExpr(int precedence) { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1176); + setState(1268); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 145, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 159, _ctx)) { case 1: { _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1165); + setState(1257); tableExpr(0); - setState(1167); + setState(1259); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 143, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 157, _ctx)) { case 1: { - setState(1166); + setState(1258); match(ClickHouseParser::FINAL); break; } } - setState(1170); + setState(1262); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 144, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 158, _ctx)) { case 1: { - setState(1169); + setState(1261); sampleClause(); break; } @@ -9329,38 +10004,38 @@ ClickHouseParser::JoinExprContext* ClickHouseParser::joinExpr(int precedence) { _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1172); + setState(1264); match(ClickHouseParser::LPAREN); - setState(1173); + setState(1265); joinExpr(0); - setState(1174); + setState(1266); match(ClickHouseParser::RPAREN); break; } } _ctx->stop = _input->LT(-1); - setState(1195); + setState(1287); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 149, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 163, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { if (!_parseListeners.empty()) triggerExitRuleEvent(); previousContext = _localctx; - setState(1193); + setState(1285); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 148, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 162, _ctx)) { case 1: { auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleJoinExpr); - setState(1178); + setState(1270); if (!(precpred(_ctx, 3))) throw FailedPredicateException(this, "precpred(_ctx, 3)"); - setState(1179); + setState(1271); joinOpCross(); - setState(1180); + setState(1272); joinExpr(4); break; } @@ -9369,17 +10044,17 @@ ClickHouseParser::JoinExprContext* ClickHouseParser::joinExpr(int precedence) { auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleJoinExpr); - setState(1182); + setState(1274); if (!(precpred(_ctx, 4))) throw FailedPredicateException(this, "precpred(_ctx, 4)"); - setState(1184); + setState(1276); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::GLOBAL || _la == ClickHouseParser::LOCAL) { - setState(1183); + setState(1275); _la = _input->LA(1); if (!(_la == ClickHouseParser::GLOBAL @@ -9391,7 +10066,7 @@ ClickHouseParser::JoinExprContext* ClickHouseParser::joinExpr(int precedence) { consume(); } } - setState(1187); + setState(1279); _errHandler->sync(this); _la = _input->LA(1); @@ -9405,23 +10080,23 @@ ClickHouseParser::JoinExprContext* ClickHouseParser::joinExpr(int precedence) { | (1ULL << (ClickHouseParser::LEFT - 80)) | (1ULL << (ClickHouseParser::RIGHT - 80)) | (1ULL << (ClickHouseParser::SEMI - 80)))) != 0)) { - setState(1186); + setState(1278); joinOp(); } - setState(1189); + setState(1281); match(ClickHouseParser::JOIN); - setState(1190); + setState(1282); joinExpr(0); - setState(1191); + setState(1283); joinConstraintClause(); break; } } } - setState(1197); + setState(1289); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 149, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 163, _ctx); } } catch (RecognitionException &e) { @@ -9546,24 +10221,24 @@ antlrcpp::Any ClickHouseParser::JoinOpLeftRightContext::accept(tree::ParseTreeVi } ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { JoinOpContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 130, ClickHouseParser::RuleJoinOp); + enterRule(_localctx, 136, ClickHouseParser::RuleJoinOp); size_t _la = 0; auto onExit = finally([=] { exitRule(); }); try { - setState(1241); + setState(1333); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 163, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 177, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1207); + setState(1299); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 152, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 166, _ctx)) { case 1: { - setState(1199); + setState(1291); _errHandler->sync(this); _la = _input->LA(1); @@ -9571,7 +10246,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) | (1ULL << ClickHouseParser::ANY) | (1ULL << ClickHouseParser::ASOF))) != 0)) { - setState(1198); + setState(1290); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) @@ -9584,15 +10259,15 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { consume(); } } - setState(1201); + setState(1293); match(ClickHouseParser::INNER); break; } case 2: { - setState(1202); + setState(1294); match(ClickHouseParser::INNER); - setState(1204); + setState(1296); _errHandler->sync(this); _la = _input->LA(1); @@ -9600,7 +10275,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) | (1ULL << ClickHouseParser::ANY) | (1ULL << ClickHouseParser::ASOF))) != 0)) { - setState(1203); + setState(1295); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) @@ -9617,7 +10292,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { } case 3: { - setState(1206); + setState(1298); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) @@ -9639,11 +10314,11 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(1223); + setState(1315); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 157, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 171, _ctx)) { case 1: { - setState(1210); + setState(1302); _errHandler->sync(this); _la = _input->LA(1); @@ -9652,7 +10327,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { | (1ULL << ClickHouseParser::ANTI) | (1ULL << ClickHouseParser::ANY) | (1ULL << ClickHouseParser::ASOF))) != 0) || _la == ClickHouseParser::SEMI) { - setState(1209); + setState(1301); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) @@ -9666,7 +10341,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { consume(); } } - setState(1212); + setState(1304); _la = _input->LA(1); if (!(_la == ClickHouseParser::LEFT @@ -9677,19 +10352,19 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { _errHandler->reportMatch(this); consume(); } - setState(1214); + setState(1306); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OUTER) { - setState(1213); + setState(1305); match(ClickHouseParser::OUTER); } break; } case 2: { - setState(1216); + setState(1308); _la = _input->LA(1); if (!(_la == ClickHouseParser::LEFT @@ -9700,15 +10375,15 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { _errHandler->reportMatch(this); consume(); } - setState(1218); + setState(1310); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OUTER) { - setState(1217); + setState(1309); match(ClickHouseParser::OUTER); } - setState(1221); + setState(1313); _errHandler->sync(this); _la = _input->LA(1); @@ -9717,7 +10392,7 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { | (1ULL << ClickHouseParser::ANTI) | (1ULL << ClickHouseParser::ANY) | (1ULL << ClickHouseParser::ASOF))) != 0) || _la == ClickHouseParser::SEMI) { - setState(1220); + setState(1312); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ALL) @@ -9741,18 +10416,18 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(1239); + setState(1331); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 162, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 176, _ctx)) { case 1: { - setState(1226); + setState(1318); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ALL || _la == ClickHouseParser::ANY) { - setState(1225); + setState(1317); _la = _input->LA(1); if (!(_la == ClickHouseParser::ALL @@ -9764,38 +10439,38 @@ ClickHouseParser::JoinOpContext* ClickHouseParser::joinOp() { consume(); } } - setState(1228); + setState(1320); match(ClickHouseParser::FULL); - setState(1230); + setState(1322); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OUTER) { - setState(1229); + setState(1321); match(ClickHouseParser::OUTER); } break; } case 2: { - setState(1232); + setState(1324); match(ClickHouseParser::FULL); - setState(1234); + setState(1326); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OUTER) { - setState(1233); + setState(1325); match(ClickHouseParser::OUTER); } - setState(1237); + setState(1329); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ALL || _la == ClickHouseParser::ANY) { - setState(1236); + setState(1328); _la = _input->LA(1); if (!(_la == ClickHouseParser::ALL @@ -9867,28 +10542,28 @@ antlrcpp::Any ClickHouseParser::JoinOpCrossContext::accept(tree::ParseTreeVisito ClickHouseParser::JoinOpCrossContext* ClickHouseParser::joinOpCross() { JoinOpCrossContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 132, ClickHouseParser::RuleJoinOpCross); + enterRule(_localctx, 138, ClickHouseParser::RuleJoinOpCross); size_t _la = 0; auto onExit = finally([=] { exitRule(); }); try { - setState(1249); + setState(1341); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::CROSS: case ClickHouseParser::GLOBAL: case ClickHouseParser::LOCAL: { enterOuterAlt(_localctx, 1); - setState(1244); + setState(1336); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::GLOBAL || _la == ClickHouseParser::LOCAL) { - setState(1243); + setState(1335); _la = _input->LA(1); if (!(_la == ClickHouseParser::GLOBAL @@ -9900,16 +10575,16 @@ ClickHouseParser::JoinOpCrossContext* ClickHouseParser::joinOpCross() { consume(); } } - setState(1246); + setState(1338); match(ClickHouseParser::CROSS); - setState(1247); + setState(1339); match(ClickHouseParser::JOIN); break; } case ClickHouseParser::COMMA: { enterOuterAlt(_localctx, 2); - setState(1248); + setState(1340); match(ClickHouseParser::COMMA); break; } @@ -9969,42 +10644,42 @@ antlrcpp::Any ClickHouseParser::JoinConstraintClauseContext::accept(tree::ParseT ClickHouseParser::JoinConstraintClauseContext* ClickHouseParser::joinConstraintClause() { JoinConstraintClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 134, ClickHouseParser::RuleJoinConstraintClause); + enterRule(_localctx, 140, ClickHouseParser::RuleJoinConstraintClause); auto onExit = finally([=] { exitRule(); }); try { - setState(1260); + setState(1352); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 166, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 180, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(1251); + setState(1343); match(ClickHouseParser::ON); - setState(1252); + setState(1344); columnExprList(); break; } case 2: { enterOuterAlt(_localctx, 2); - setState(1253); + setState(1345); match(ClickHouseParser::USING); - setState(1254); + setState(1346); match(ClickHouseParser::LPAREN); - setState(1255); + setState(1347); columnExprList(); - setState(1256); + setState(1348); match(ClickHouseParser::RPAREN); break; } case 3: { enterOuterAlt(_localctx, 3); - setState(1258); + setState(1350); match(ClickHouseParser::USING); - setState(1259); + setState(1351); columnExprList(); break; } @@ -10058,25 +10733,25 @@ antlrcpp::Any ClickHouseParser::SampleClauseContext::accept(tree::ParseTreeVisit ClickHouseParser::SampleClauseContext* ClickHouseParser::sampleClause() { SampleClauseContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 136, ClickHouseParser::RuleSampleClause); + enterRule(_localctx, 142, ClickHouseParser::RuleSampleClause); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1262); + setState(1354); match(ClickHouseParser::SAMPLE); - setState(1263); + setState(1355); ratioExpr(); - setState(1266); + setState(1358); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 167, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 181, _ctx)) { case 1: { - setState(1264); + setState(1356); match(ClickHouseParser::OFFSET); - setState(1265); + setState(1357); ratioExpr(); break; } @@ -10130,7 +10805,7 @@ antlrcpp::Any ClickHouseParser::LimitExprContext::accept(tree::ParseTreeVisitor ClickHouseParser::LimitExprContext* ClickHouseParser::limitExpr() { LimitExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 138, ClickHouseParser::RuleLimitExpr); + enterRule(_localctx, 144, ClickHouseParser::RuleLimitExpr); size_t _la = 0; auto onExit = finally([=] { @@ -10138,14 +10813,14 @@ ClickHouseParser::LimitExprContext* ClickHouseParser::limitExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(1268); + setState(1360); columnExpr(0); - setState(1271); + setState(1363); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::OFFSET || _la == ClickHouseParser::COMMA) { - setState(1269); + setState(1361); _la = _input->LA(1); if (!(_la == ClickHouseParser::OFFSET || _la == ClickHouseParser::COMMA)) { _errHandler->recoverInline(this); @@ -10154,7 +10829,7 @@ ClickHouseParser::LimitExprContext* ClickHouseParser::limitExpr() { _errHandler->reportMatch(this); consume(); } - setState(1270); + setState(1362); columnExpr(0); } @@ -10205,7 +10880,7 @@ antlrcpp::Any ClickHouseParser::OrderExprListContext::accept(tree::ParseTreeVisi ClickHouseParser::OrderExprListContext* ClickHouseParser::orderExprList() { OrderExprListContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 140, ClickHouseParser::RuleOrderExprList); + enterRule(_localctx, 146, ClickHouseParser::RuleOrderExprList); auto onExit = finally([=] { exitRule(); @@ -10213,21 +10888,21 @@ ClickHouseParser::OrderExprListContext* ClickHouseParser::orderExprList() { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1273); + setState(1365); orderExpr(); - setState(1278); + setState(1370); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 169, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 183, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(1274); + setState(1366); match(ClickHouseParser::COMMA); - setState(1275); + setState(1367); orderExpr(); } - setState(1280); + setState(1372); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 169, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 183, _ctx); } } @@ -10297,7 +10972,7 @@ antlrcpp::Any ClickHouseParser::OrderExprContext::accept(tree::ParseTreeVisitor ClickHouseParser::OrderExprContext* ClickHouseParser::orderExpr() { OrderExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 142, ClickHouseParser::RuleOrderExpr); + enterRule(_localctx, 148, ClickHouseParser::RuleOrderExpr); size_t _la = 0; auto onExit = finally([=] { @@ -10305,14 +10980,14 @@ ClickHouseParser::OrderExprContext* ClickHouseParser::orderExpr() { }); try { enterOuterAlt(_localctx, 1); - setState(1281); + setState(1373); columnExpr(0); - setState(1283); + setState(1375); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 170, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 184, _ctx)) { case 1: { - setState(1282); + setState(1374); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::ASCENDING) @@ -10328,14 +11003,14 @@ ClickHouseParser::OrderExprContext* ClickHouseParser::orderExpr() { } } - setState(1287); + setState(1379); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 171, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 185, _ctx)) { case 1: { - setState(1285); + setState(1377); match(ClickHouseParser::NULLS); - setState(1286); + setState(1378); _la = _input->LA(1); if (!(_la == ClickHouseParser::FIRST @@ -10350,14 +11025,14 @@ ClickHouseParser::OrderExprContext* ClickHouseParser::orderExpr() { } } - setState(1291); + setState(1383); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 172, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 186, _ctx)) { case 1: { - setState(1289); + setState(1381); match(ClickHouseParser::COLLATE); - setState(1290); + setState(1382); match(ClickHouseParser::STRING_LITERAL); break; } @@ -10407,23 +11082,23 @@ antlrcpp::Any ClickHouseParser::RatioExprContext::accept(tree::ParseTreeVisitor ClickHouseParser::RatioExprContext* ClickHouseParser::ratioExpr() { RatioExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 144, ClickHouseParser::RuleRatioExpr); + enterRule(_localctx, 150, ClickHouseParser::RuleRatioExpr); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1293); + setState(1385); numberLiteral(); - setState(1296); + setState(1388); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 173, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 187, _ctx)) { case 1: { - setState(1294); + setState(1386); match(ClickHouseParser::SLASH); - setState(1295); + setState(1387); numberLiteral(); break; } @@ -10477,7 +11152,7 @@ antlrcpp::Any ClickHouseParser::SettingExprListContext::accept(tree::ParseTreeVi ClickHouseParser::SettingExprListContext* ClickHouseParser::settingExprList() { SettingExprListContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 146, ClickHouseParser::RuleSettingExprList); + enterRule(_localctx, 152, ClickHouseParser::RuleSettingExprList); auto onExit = finally([=] { exitRule(); @@ -10485,21 +11160,21 @@ ClickHouseParser::SettingExprListContext* ClickHouseParser::settingExprList() { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1298); + setState(1390); settingExpr(); - setState(1303); + setState(1395); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 174, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 188, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(1299); + setState(1391); match(ClickHouseParser::COMMA); - setState(1300); + setState(1392); settingExpr(); } - setState(1305); + setState(1397); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 174, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 188, _ctx); } } @@ -10545,18 +11220,18 @@ antlrcpp::Any ClickHouseParser::SettingExprContext::accept(tree::ParseTreeVisito ClickHouseParser::SettingExprContext* ClickHouseParser::settingExpr() { SettingExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 148, ClickHouseParser::RuleSettingExpr); + enterRule(_localctx, 154, ClickHouseParser::RuleSettingExpr); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1306); + setState(1398); identifier(); - setState(1307); + setState(1399); match(ClickHouseParser::EQ_SINGLE); - setState(1308); + setState(1400); literal(); } @@ -10598,16 +11273,16 @@ antlrcpp::Any ClickHouseParser::SetStmtContext::accept(tree::ParseTreeVisitor *v ClickHouseParser::SetStmtContext* ClickHouseParser::setStmt() { SetStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 150, ClickHouseParser::RuleSetStmt); + enterRule(_localctx, 156, ClickHouseParser::RuleSetStmt); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1310); + setState(1402); match(ClickHouseParser::SET); - setState(1311); + setState(1403); settingExprList(); } @@ -10819,26 +11494,26 @@ antlrcpp::Any ClickHouseParser::ShowCreateDictionaryStmtContext::accept(tree::Pa } ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { ShowStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 152, ClickHouseParser::RuleShowStmt); + enterRule(_localctx, 158, ClickHouseParser::RuleShowStmt); size_t _la = 0; auto onExit = finally([=] { exitRule(); }); try { - setState(1355); + setState(1447); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 182, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 196, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1313); + setState(1405); match(ClickHouseParser::SHOW); - setState(1314); + setState(1406); match(ClickHouseParser::CREATE); - setState(1315); + setState(1407); match(ClickHouseParser::DATABASE); - setState(1316); + setState(1408); databaseIdentifier(); break; } @@ -10846,13 +11521,13 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(1317); + setState(1409); match(ClickHouseParser::SHOW); - setState(1318); + setState(1410); match(ClickHouseParser::CREATE); - setState(1319); + setState(1411); match(ClickHouseParser::DICTIONARY); - setState(1320); + setState(1412); tableIdentifier(); break; } @@ -10860,33 +11535,33 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(1321); + setState(1413); match(ClickHouseParser::SHOW); - setState(1322); + setState(1414); match(ClickHouseParser::CREATE); - setState(1324); + setState(1416); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 175, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 189, _ctx)) { case 1: { - setState(1323); + setState(1415); match(ClickHouseParser::TEMPORARY); break; } } - setState(1327); + setState(1419); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 176, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 190, _ctx)) { case 1: { - setState(1326); + setState(1418); match(ClickHouseParser::TABLE); break; } } - setState(1329); + setState(1421); tableIdentifier(); break; } @@ -10894,9 +11569,9 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { case 4: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 4); - setState(1330); + setState(1422); match(ClickHouseParser::SHOW); - setState(1331); + setState(1423); match(ClickHouseParser::DATABASES); break; } @@ -10904,18 +11579,18 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { case 5: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 5); - setState(1332); + setState(1424); match(ClickHouseParser::SHOW); - setState(1333); + setState(1425); match(ClickHouseParser::DICTIONARIES); - setState(1336); + setState(1428); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::FROM) { - setState(1334); + setState(1426); match(ClickHouseParser::FROM); - setState(1335); + setState(1427); databaseIdentifier(); } break; @@ -10924,26 +11599,26 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { case 6: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 6); - setState(1338); + setState(1430); match(ClickHouseParser::SHOW); - setState(1340); + setState(1432); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TEMPORARY) { - setState(1339); + setState(1431); match(ClickHouseParser::TEMPORARY); } - setState(1342); + setState(1434); match(ClickHouseParser::TABLES); - setState(1345); + setState(1437); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::FROM || _la == ClickHouseParser::IN) { - setState(1343); + setState(1435); _la = _input->LA(1); if (!(_la == ClickHouseParser::FROM @@ -10954,22 +11629,22 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { _errHandler->reportMatch(this); consume(); } - setState(1344); + setState(1436); databaseIdentifier(); } - setState(1350); + setState(1442); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::LIKE: { - setState(1347); + setState(1439); match(ClickHouseParser::LIKE); - setState(1348); + setState(1440); match(ClickHouseParser::STRING_LITERAL); break; } case ClickHouseParser::WHERE: { - setState(1349); + setState(1441); whereClause(); break; } @@ -10985,12 +11660,12 @@ ClickHouseParser::ShowStmtContext* ClickHouseParser::showStmt() { default: break; } - setState(1353); + setState(1445); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::LIMIT) { - setState(1352); + setState(1444); limitClause(); } break; @@ -11097,69 +11772,69 @@ antlrcpp::Any ClickHouseParser::SystemStmtContext::accept(tree::ParseTreeVisitor ClickHouseParser::SystemStmtContext* ClickHouseParser::systemStmt() { SystemStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 154, ClickHouseParser::RuleSystemStmt); + enterRule(_localctx, 160, ClickHouseParser::RuleSystemStmt); size_t _la = 0; auto onExit = finally([=] { exitRule(); }); try { - setState(1391); + setState(1483); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 185, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 199, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(1357); + setState(1449); match(ClickHouseParser::SYSTEM); - setState(1358); + setState(1450); match(ClickHouseParser::FLUSH); - setState(1359); + setState(1451); match(ClickHouseParser::DISTRIBUTED); - setState(1360); + setState(1452); tableIdentifier(); break; } case 2: { enterOuterAlt(_localctx, 2); - setState(1361); + setState(1453); match(ClickHouseParser::SYSTEM); - setState(1362); + setState(1454); match(ClickHouseParser::FLUSH); - setState(1363); + setState(1455); match(ClickHouseParser::LOGS); break; } case 3: { enterOuterAlt(_localctx, 3); - setState(1364); + setState(1456); match(ClickHouseParser::SYSTEM); - setState(1365); + setState(1457); match(ClickHouseParser::RELOAD); - setState(1366); + setState(1458); match(ClickHouseParser::DICTIONARIES); break; } case 4: { enterOuterAlt(_localctx, 4); - setState(1367); + setState(1459); match(ClickHouseParser::SYSTEM); - setState(1368); + setState(1460); match(ClickHouseParser::RELOAD); - setState(1369); + setState(1461); match(ClickHouseParser::DICTIONARY); - setState(1370); + setState(1462); tableIdentifier(); break; } case 5: { enterOuterAlt(_localctx, 5); - setState(1371); + setState(1463); match(ClickHouseParser::SYSTEM); - setState(1372); + setState(1464); _la = _input->LA(1); if (!(_la == ClickHouseParser::START @@ -11170,34 +11845,34 @@ ClickHouseParser::SystemStmtContext* ClickHouseParser::systemStmt() { _errHandler->reportMatch(this); consume(); } - setState(1380); + setState(1472); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::DISTRIBUTED: { - setState(1373); + setState(1465); match(ClickHouseParser::DISTRIBUTED); - setState(1374); + setState(1466); match(ClickHouseParser::SENDS); break; } case ClickHouseParser::FETCHES: { - setState(1375); + setState(1467); match(ClickHouseParser::FETCHES); break; } case ClickHouseParser::MERGES: case ClickHouseParser::TTL: { - setState(1377); + setState(1469); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::TTL) { - setState(1376); + setState(1468); match(ClickHouseParser::TTL); } - setState(1379); + setState(1471); match(ClickHouseParser::MERGES); break; } @@ -11205,16 +11880,16 @@ ClickHouseParser::SystemStmtContext* ClickHouseParser::systemStmt() { default: throw NoViableAltException(this); } - setState(1382); + setState(1474); tableIdentifier(); break; } case 6: { enterOuterAlt(_localctx, 6); - setState(1383); + setState(1475); match(ClickHouseParser::SYSTEM); - setState(1384); + setState(1476); _la = _input->LA(1); if (!(_la == ClickHouseParser::START @@ -11225,22 +11900,22 @@ ClickHouseParser::SystemStmtContext* ClickHouseParser::systemStmt() { _errHandler->reportMatch(this); consume(); } - setState(1385); + setState(1477); match(ClickHouseParser::REPLICATED); - setState(1386); + setState(1478); match(ClickHouseParser::SENDS); break; } case 7: { enterOuterAlt(_localctx, 7); - setState(1387); + setState(1479); match(ClickHouseParser::SYSTEM); - setState(1388); + setState(1480); match(ClickHouseParser::SYNC); - setState(1389); + setState(1481); match(ClickHouseParser::REPLICA); - setState(1390); + setState(1482); tableIdentifier(); break; } @@ -11306,7 +11981,7 @@ antlrcpp::Any ClickHouseParser::TruncateStmtContext::accept(tree::ParseTreeVisit ClickHouseParser::TruncateStmtContext* ClickHouseParser::truncateStmt() { TruncateStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 156, ClickHouseParser::RuleTruncateStmt); + enterRule(_localctx, 162, ClickHouseParser::RuleTruncateStmt); size_t _la = 0; auto onExit = finally([=] { @@ -11314,51 +11989,51 @@ ClickHouseParser::TruncateStmtContext* ClickHouseParser::truncateStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1393); + setState(1485); match(ClickHouseParser::TRUNCATE); - setState(1395); + setState(1487); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 186, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 200, _ctx)) { case 1: { - setState(1394); + setState(1486); match(ClickHouseParser::TEMPORARY); break; } } - setState(1398); + setState(1490); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 187, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 201, _ctx)) { case 1: { - setState(1397); + setState(1489); match(ClickHouseParser::TABLE); break; } } - setState(1402); + setState(1494); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 188, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 202, _ctx)) { case 1: { - setState(1400); + setState(1492); match(ClickHouseParser::IF); - setState(1401); + setState(1493); match(ClickHouseParser::EXISTS); break; } } - setState(1404); + setState(1496); tableIdentifier(); - setState(1406); + setState(1498); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ON) { - setState(1405); + setState(1497); clusterClause(); } @@ -11401,16 +12076,16 @@ antlrcpp::Any ClickHouseParser::UseStmtContext::accept(tree::ParseTreeVisitor *v ClickHouseParser::UseStmtContext* ClickHouseParser::useStmt() { UseStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 158, ClickHouseParser::RuleUseStmt); + enterRule(_localctx, 164, ClickHouseParser::RuleUseStmt); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1408); + setState(1500); match(ClickHouseParser::USE); - setState(1409); + setState(1501); databaseIdentifier(); } @@ -11464,7 +12139,7 @@ antlrcpp::Any ClickHouseParser::WatchStmtContext::accept(tree::ParseTreeVisitor ClickHouseParser::WatchStmtContext* ClickHouseParser::watchStmt() { WatchStmtContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 160, ClickHouseParser::RuleWatchStmt); + enterRule(_localctx, 166, ClickHouseParser::RuleWatchStmt); size_t _la = 0; auto onExit = finally([=] { @@ -11472,26 +12147,26 @@ ClickHouseParser::WatchStmtContext* ClickHouseParser::watchStmt() { }); try { enterOuterAlt(_localctx, 1); - setState(1411); + setState(1503); match(ClickHouseParser::WATCH); - setState(1412); + setState(1504); tableIdentifier(); - setState(1414); + setState(1506); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::EVENTS) { - setState(1413); + setState(1505); match(ClickHouseParser::EVENTS); } - setState(1418); + setState(1510); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::LIMIT) { - setState(1416); + setState(1508); match(ClickHouseParser::LIMIT); - setState(1417); + setState(1509); match(ClickHouseParser::DECIMAL_LITERAL); } @@ -11685,20 +12360,20 @@ antlrcpp::Any ClickHouseParser::ColumnTypeExprEnumContext::accept(tree::ParseTre } ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { ColumnTypeExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 162, ClickHouseParser::RuleColumnTypeExpr); + enterRule(_localctx, 168, ClickHouseParser::RuleColumnTypeExpr); size_t _la = 0; auto onExit = finally([=] { exitRule(); }); try { - setState(1467); + setState(1559); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 196, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 210, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1420); + setState(1512); identifier(); break; } @@ -11706,29 +12381,29 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(1421); + setState(1513); identifier(); - setState(1422); + setState(1514); match(ClickHouseParser::LPAREN); - setState(1423); + setState(1515); identifier(); - setState(1424); + setState(1516); columnTypeExpr(); - setState(1431); + setState(1523); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1425); + setState(1517); match(ClickHouseParser::COMMA); - setState(1426); + setState(1518); identifier(); - setState(1427); + setState(1519); columnTypeExpr(); - setState(1433); + setState(1525); _errHandler->sync(this); _la = _input->LA(1); } - setState(1434); + setState(1526); match(ClickHouseParser::RPAREN); break; } @@ -11736,25 +12411,25 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(1436); + setState(1528); identifier(); - setState(1437); + setState(1529); match(ClickHouseParser::LPAREN); - setState(1438); + setState(1530); enumValue(); - setState(1443); + setState(1535); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1439); + setState(1531); match(ClickHouseParser::COMMA); - setState(1440); + setState(1532); enumValue(); - setState(1445); + setState(1537); _errHandler->sync(this); _la = _input->LA(1); } - setState(1446); + setState(1538); match(ClickHouseParser::RPAREN); break; } @@ -11762,25 +12437,25 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { case 4: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 4); - setState(1448); + setState(1540); identifier(); - setState(1449); + setState(1541); match(ClickHouseParser::LPAREN); - setState(1450); + setState(1542); columnTypeExpr(); - setState(1455); + setState(1547); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1451); + setState(1543); match(ClickHouseParser::COMMA); - setState(1452); + setState(1544); columnTypeExpr(); - setState(1457); + setState(1549); _errHandler->sync(this); _la = _input->LA(1); } - setState(1458); + setState(1550); match(ClickHouseParser::RPAREN); break; } @@ -11788,11 +12463,11 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { case 5: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 5); - setState(1460); + setState(1552); identifier(); - setState(1461); + setState(1553); match(ClickHouseParser::LPAREN); - setState(1463); + setState(1555); _errHandler->sync(this); _la = _input->LA(1); @@ -11895,6 +12570,7 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -11920,10 +12596,10 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) | (1ULL << (ClickHouseParser::QUARTER - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -11982,16 +12658,16 @@ ClickHouseParser::ColumnTypeExprContext* ClickHouseParser::columnTypeExpr() { | (1ULL << (ClickHouseParser::DECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::HEXADECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::STRING_LITERAL - 128)) - | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 194) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 194)) & ((1ULL << (ClickHouseParser::DASH - 194)) - | (1ULL << (ClickHouseParser::DOT - 194)) - | (1ULL << (ClickHouseParser::LBRACKET - 194)) - | (1ULL << (ClickHouseParser::LPAREN - 194)) - | (1ULL << (ClickHouseParser::PLUS - 194)))) != 0)) { - setState(1462); + | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 196) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 196)) & ((1ULL << (ClickHouseParser::DASH - 196)) + | (1ULL << (ClickHouseParser::DOT - 196)) + | (1ULL << (ClickHouseParser::LBRACKET - 196)) + | (1ULL << (ClickHouseParser::LPAREN - 196)) + | (1ULL << (ClickHouseParser::PLUS - 196)))) != 0)) { + setState(1554); columnExprList(); } - setState(1465); + setState(1557); match(ClickHouseParser::RPAREN); break; } @@ -12045,7 +12721,7 @@ antlrcpp::Any ClickHouseParser::ColumnExprListContext::accept(tree::ParseTreeVis ClickHouseParser::ColumnExprListContext* ClickHouseParser::columnExprList() { ColumnExprListContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 164, ClickHouseParser::RuleColumnExprList); + enterRule(_localctx, 170, ClickHouseParser::RuleColumnExprList); auto onExit = finally([=] { exitRule(); @@ -12053,21 +12729,21 @@ ClickHouseParser::ColumnExprListContext* ClickHouseParser::columnExprList() { try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1469); + setState(1561); columnsExpr(); - setState(1474); + setState(1566); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 197, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 211, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { - setState(1470); + setState(1562); match(ClickHouseParser::COMMA); - setState(1471); + setState(1563); columnsExpr(); } - setState(1476); + setState(1568); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 197, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 211, _ctx); } } @@ -12158,20 +12834,20 @@ antlrcpp::Any ClickHouseParser::ColumnsExprSubqueryContext::accept(tree::ParseTr } ClickHouseParser::ColumnsExprContext* ClickHouseParser::columnsExpr() { ColumnsExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 166, ClickHouseParser::RuleColumnsExpr); + enterRule(_localctx, 172, ClickHouseParser::RuleColumnsExpr); size_t _la = 0; auto onExit = finally([=] { exitRule(); }); try { - setState(1488); + setState(1580); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 199, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 213, _ctx)) { case 1: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 1); - setState(1480); + setState(1572); _errHandler->sync(this); _la = _input->LA(1); @@ -12273,6 +12949,7 @@ ClickHouseParser::ColumnsExprContext* ClickHouseParser::columnsExpr() { | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -12296,10 +12973,10 @@ ClickHouseParser::ColumnsExprContext* ClickHouseParser::columnsExpr() { | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) | (1ULL << (ClickHouseParser::QUARTER - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -12353,12 +13030,12 @@ ClickHouseParser::ColumnsExprContext* ClickHouseParser::columnsExpr() { | (1ULL << (ClickHouseParser::JSON_FALSE - 128)) | (1ULL << (ClickHouseParser::JSON_TRUE - 128)) | (1ULL << (ClickHouseParser::IDENTIFIER - 128)))) != 0)) { - setState(1477); + setState(1569); tableIdentifier(); - setState(1478); + setState(1570); match(ClickHouseParser::DOT); } - setState(1482); + setState(1574); match(ClickHouseParser::ASTERISK); break; } @@ -12366,11 +13043,11 @@ ClickHouseParser::ColumnsExprContext* ClickHouseParser::columnsExpr() { case 2: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 2); - setState(1483); + setState(1575); match(ClickHouseParser::LPAREN); - setState(1484); + setState(1576); selectUnionStmt(); - setState(1485); + setState(1577); match(ClickHouseParser::RPAREN); break; } @@ -12378,7 +13055,7 @@ ClickHouseParser::ColumnsExprContext* ClickHouseParser::columnsExpr() { case 3: { _localctx = dynamic_cast(_tracker.createInstance(_localctx)); enterOuterAlt(_localctx, 3); - setState(1487); + setState(1579); columnExpr(0); break; } @@ -13248,8 +13925,8 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence ClickHouseParser::ColumnExprContext *_localctx = _tracker.createInstance(_ctx, parentState); ClickHouseParser::ColumnExprContext *previousContext = _localctx; (void)previousContext; // Silence compiler, in case the context is not used by generated code. - size_t startState = 168; - enterRecursionRule(_localctx, 168, ClickHouseParser::RuleColumnExpr, precedence); + size_t startState = 174; + enterRecursionRule(_localctx, 174, ClickHouseParser::RuleColumnExpr, precedence); size_t _la = 0; @@ -13259,54 +13936,54 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1597); + setState(1689); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 210, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 224, _ctx)) { case 1: { _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1491); + setState(1583); match(ClickHouseParser::CASE); - setState(1493); + setState(1585); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 200, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 214, _ctx)) { case 1: { - setState(1492); + setState(1584); columnExpr(0); break; } } - setState(1500); + setState(1592); _errHandler->sync(this); _la = _input->LA(1); do { - setState(1495); + setState(1587); match(ClickHouseParser::WHEN); - setState(1496); + setState(1588); columnExpr(0); - setState(1497); + setState(1589); match(ClickHouseParser::THEN); - setState(1498); + setState(1590); columnExpr(0); - setState(1502); + setState(1594); _errHandler->sync(this); _la = _input->LA(1); } while (_la == ClickHouseParser::WHEN); - setState(1506); + setState(1598); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::ELSE) { - setState(1504); + setState(1596); match(ClickHouseParser::ELSE); - setState(1505); + setState(1597); columnExpr(0); } - setState(1508); + setState(1600); match(ClickHouseParser::END); break; } @@ -13315,17 +13992,17 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1510); + setState(1602); match(ClickHouseParser::CAST); - setState(1511); + setState(1603); match(ClickHouseParser::LPAREN); - setState(1512); + setState(1604); columnExpr(0); - setState(1513); + setState(1605); match(ClickHouseParser::AS); - setState(1514); + setState(1606); columnTypeExpr(); - setState(1515); + setState(1607); match(ClickHouseParser::RPAREN); break; } @@ -13334,9 +14011,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1517); + setState(1609); match(ClickHouseParser::DATE); - setState(1518); + setState(1610); match(ClickHouseParser::STRING_LITERAL); break; } @@ -13345,17 +14022,17 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1519); + setState(1611); match(ClickHouseParser::EXTRACT); - setState(1520); + setState(1612); match(ClickHouseParser::LPAREN); - setState(1521); + setState(1613); interval(); - setState(1522); + setState(1614); match(ClickHouseParser::FROM); - setState(1523); + setState(1615); columnExpr(0); - setState(1524); + setState(1616); match(ClickHouseParser::RPAREN); break; } @@ -13364,11 +14041,11 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1526); + setState(1618); match(ClickHouseParser::INTERVAL); - setState(1527); + setState(1619); columnExpr(0); - setState(1528); + setState(1620); interval(); break; } @@ -13377,27 +14054,27 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1530); + setState(1622); match(ClickHouseParser::SUBSTRING); - setState(1531); + setState(1623); match(ClickHouseParser::LPAREN); - setState(1532); + setState(1624); columnExpr(0); - setState(1533); + setState(1625); match(ClickHouseParser::FROM); - setState(1534); + setState(1626); columnExpr(0); - setState(1537); + setState(1629); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::FOR) { - setState(1535); + setState(1627); match(ClickHouseParser::FOR); - setState(1536); + setState(1628); columnExpr(0); } - setState(1539); + setState(1631); match(ClickHouseParser::RPAREN); break; } @@ -13406,9 +14083,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1541); + setState(1633); match(ClickHouseParser::TIMESTAMP); - setState(1542); + setState(1634); match(ClickHouseParser::STRING_LITERAL); break; } @@ -13417,11 +14094,11 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1543); + setState(1635); match(ClickHouseParser::TRIM); - setState(1544); + setState(1636); match(ClickHouseParser::LPAREN); - setState(1545); + setState(1637); _la = _input->LA(1); if (!(_la == ClickHouseParser::BOTH || _la == ClickHouseParser::LEADING || _la == ClickHouseParser::TRAILING)) { _errHandler->recoverInline(this); @@ -13430,13 +14107,13 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _errHandler->reportMatch(this); consume(); } - setState(1546); + setState(1638); match(ClickHouseParser::STRING_LITERAL); - setState(1547); + setState(1639); match(ClickHouseParser::FROM); - setState(1548); + setState(1640); columnExpr(0); - setState(1549); + setState(1641); match(ClickHouseParser::RPAREN); break; } @@ -13445,16 +14122,16 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1551); + setState(1643); identifier(); - setState(1557); + setState(1649); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 205, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 219, _ctx)) { case 1: { - setState(1552); + setState(1644); match(ClickHouseParser::LPAREN); - setState(1554); + setState(1646); _errHandler->sync(this); _la = _input->LA(1); @@ -13557,6 +14234,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -13582,10 +14260,10 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) | (1ULL << (ClickHouseParser::QUARTER - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -13644,35 +14322,35 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::DECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::HEXADECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::STRING_LITERAL - 128)) - | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 194) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 194)) & ((1ULL << (ClickHouseParser::DASH - 194)) - | (1ULL << (ClickHouseParser::DOT - 194)) - | (1ULL << (ClickHouseParser::LBRACKET - 194)) - | (1ULL << (ClickHouseParser::LPAREN - 194)) - | (1ULL << (ClickHouseParser::PLUS - 194)))) != 0)) { - setState(1553); + | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 196) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 196)) & ((1ULL << (ClickHouseParser::DASH - 196)) + | (1ULL << (ClickHouseParser::DOT - 196)) + | (1ULL << (ClickHouseParser::LBRACKET - 196)) + | (1ULL << (ClickHouseParser::LPAREN - 196)) + | (1ULL << (ClickHouseParser::PLUS - 196)))) != 0)) { + setState(1645); columnExprList(); } - setState(1556); + setState(1648); match(ClickHouseParser::RPAREN); break; } } - setState(1559); + setState(1651); match(ClickHouseParser::LPAREN); - setState(1561); + setState(1653); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 206, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 220, _ctx)) { case 1: { - setState(1560); + setState(1652); match(ClickHouseParser::DISTINCT); break; } } - setState(1564); + setState(1656); _errHandler->sync(this); _la = _input->LA(1); @@ -13775,6 +14453,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -13800,10 +14479,10 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) | (1ULL << (ClickHouseParser::QUARTER - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -13862,16 +14541,16 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::DECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::HEXADECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::STRING_LITERAL - 128)) - | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 194) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 194)) & ((1ULL << (ClickHouseParser::DASH - 194)) - | (1ULL << (ClickHouseParser::DOT - 194)) - | (1ULL << (ClickHouseParser::LBRACKET - 194)) - | (1ULL << (ClickHouseParser::LPAREN - 194)) - | (1ULL << (ClickHouseParser::PLUS - 194)))) != 0)) { - setState(1563); + | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 196) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 196)) & ((1ULL << (ClickHouseParser::DASH - 196)) + | (1ULL << (ClickHouseParser::DOT - 196)) + | (1ULL << (ClickHouseParser::LBRACKET - 196)) + | (1ULL << (ClickHouseParser::LPAREN - 196)) + | (1ULL << (ClickHouseParser::PLUS - 196)))) != 0)) { + setState(1655); columnArgList(); } - setState(1566); + setState(1658); match(ClickHouseParser::RPAREN); break; } @@ -13880,7 +14559,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1568); + setState(1660); literal(); break; } @@ -13889,9 +14568,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1569); + setState(1661); match(ClickHouseParser::DASH); - setState(1570); + setState(1662); columnExpr(17); break; } @@ -13900,9 +14579,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1571); + setState(1663); match(ClickHouseParser::NOT); - setState(1572); + setState(1664); columnExpr(12); break; } @@ -13911,7 +14590,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1576); + setState(1668); _errHandler->sync(this); _la = _input->LA(1); @@ -14013,6 +14692,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -14036,10 +14716,10 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) | (1ULL << (ClickHouseParser::QUARTER - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -14093,12 +14773,12 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::JSON_FALSE - 128)) | (1ULL << (ClickHouseParser::JSON_TRUE - 128)) | (1ULL << (ClickHouseParser::IDENTIFIER - 128)))) != 0)) { - setState(1573); + setState(1665); tableIdentifier(); - setState(1574); + setState(1666); match(ClickHouseParser::DOT); } - setState(1578); + setState(1670); match(ClickHouseParser::ASTERISK); break; } @@ -14107,11 +14787,11 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1579); + setState(1671); match(ClickHouseParser::LPAREN); - setState(1580); + setState(1672); selectUnionStmt(); - setState(1581); + setState(1673); match(ClickHouseParser::RPAREN); break; } @@ -14120,11 +14800,11 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1583); + setState(1675); match(ClickHouseParser::LPAREN); - setState(1584); + setState(1676); columnExpr(0); - setState(1585); + setState(1677); match(ClickHouseParser::RPAREN); break; } @@ -14133,11 +14813,11 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1587); + setState(1679); match(ClickHouseParser::LPAREN); - setState(1588); + setState(1680); columnExprList(); - setState(1589); + setState(1681); match(ClickHouseParser::RPAREN); break; } @@ -14146,9 +14826,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1591); + setState(1683); match(ClickHouseParser::LBRACKET); - setState(1593); + setState(1685); _errHandler->sync(this); _la = _input->LA(1); @@ -14251,6 +14931,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -14276,10 +14957,10 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) | (1ULL << (ClickHouseParser::QUARTER - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -14338,16 +15019,16 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence | (1ULL << (ClickHouseParser::DECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::HEXADECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::STRING_LITERAL - 128)) - | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 194) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 194)) & ((1ULL << (ClickHouseParser::DASH - 194)) - | (1ULL << (ClickHouseParser::DOT - 194)) - | (1ULL << (ClickHouseParser::LBRACKET - 194)) - | (1ULL << (ClickHouseParser::LPAREN - 194)) - | (1ULL << (ClickHouseParser::PLUS - 194)))) != 0)) { - setState(1592); + | (1ULL << (ClickHouseParser::ASTERISK - 128)))) != 0) || ((((_la - 196) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 196)) & ((1ULL << (ClickHouseParser::DASH - 196)) + | (1ULL << (ClickHouseParser::DOT - 196)) + | (1ULL << (ClickHouseParser::LBRACKET - 196)) + | (1ULL << (ClickHouseParser::LPAREN - 196)) + | (1ULL << (ClickHouseParser::PLUS - 196)))) != 0)) { + setState(1684); columnExprList(); } - setState(1595); + setState(1687); match(ClickHouseParser::RBRACKET); break; } @@ -14356,44 +15037,44 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1596); + setState(1688); columnIdentifier(); break; } } _ctx->stop = _input->LT(-1); - setState(1670); + setState(1762); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 219, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 233, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { if (!_parseListeners.empty()) triggerExitRuleEvent(); previousContext = _localctx; - setState(1668); + setState(1760); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 218, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 232, _ctx)) { case 1: { auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1599); + setState(1691); if (!(precpred(_ctx, 16))) throw FailedPredicateException(this, "precpred(_ctx, 16)"); - setState(1600); + setState(1692); _la = _input->LA(1); - if (!(((((_la - 188) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 188)) & ((1ULL << (ClickHouseParser::ASTERISK - 188)) - | (1ULL << (ClickHouseParser::PERCENT - 188)) - | (1ULL << (ClickHouseParser::SLASH - 188)))) != 0))) { + if (!(((((_la - 190) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 190)) & ((1ULL << (ClickHouseParser::ASTERISK - 190)) + | (1ULL << (ClickHouseParser::PERCENT - 190)) + | (1ULL << (ClickHouseParser::SLASH - 190)))) != 0))) { _errHandler->recoverInline(this); } else { _errHandler->reportMatch(this); consume(); } - setState(1601); + setState(1693); columnExpr(17); break; } @@ -14402,22 +15083,22 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1602); + setState(1694); if (!(precpred(_ctx, 15))) throw FailedPredicateException(this, "precpred(_ctx, 15)"); - setState(1603); + setState(1695); _la = _input->LA(1); - if (!(((((_la - 193) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 193)) & ((1ULL << (ClickHouseParser::CONCAT - 193)) - | (1ULL << (ClickHouseParser::DASH - 193)) - | (1ULL << (ClickHouseParser::PLUS - 193)))) != 0))) { + if (!(((((_la - 195) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 195)) & ((1ULL << (ClickHouseParser::CONCAT - 195)) + | (1ULL << (ClickHouseParser::DASH - 195)) + | (1ULL << (ClickHouseParser::PLUS - 195)))) != 0))) { _errHandler->recoverInline(this); } else { _errHandler->reportMatch(this); consume(); } - setState(1604); + setState(1696); columnExpr(16); break; } @@ -14426,86 +15107,86 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1605); + setState(1697); if (!(precpred(_ctx, 14))) throw FailedPredicateException(this, "precpred(_ctx, 14)"); - setState(1624); + setState(1716); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 214, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 228, _ctx)) { case 1: { - setState(1606); + setState(1698); match(ClickHouseParser::EQ_DOUBLE); break; } case 2: { - setState(1607); + setState(1699); match(ClickHouseParser::EQ_SINGLE); break; } case 3: { - setState(1608); + setState(1700); match(ClickHouseParser::NOT_EQ); break; } case 4: { - setState(1609); + setState(1701); match(ClickHouseParser::LE); break; } case 5: { - setState(1610); + setState(1702); match(ClickHouseParser::GE); break; } case 6: { - setState(1611); + setState(1703); match(ClickHouseParser::LT); break; } case 7: { - setState(1612); + setState(1704); match(ClickHouseParser::GT); break; } case 8: { - setState(1614); + setState(1706); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::GLOBAL) { - setState(1613); + setState(1705); match(ClickHouseParser::GLOBAL); } - setState(1617); + setState(1709); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::NOT) { - setState(1616); + setState(1708); match(ClickHouseParser::NOT); } - setState(1619); + setState(1711); match(ClickHouseParser::IN); break; } case 9: { - setState(1621); + setState(1713); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::NOT) { - setState(1620); + setState(1712); match(ClickHouseParser::NOT); } - setState(1623); + setState(1715); _la = _input->LA(1); if (!(_la == ClickHouseParser::ILIKE @@ -14520,7 +15201,7 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence } } - setState(1626); + setState(1718); columnExpr(15); break; } @@ -14529,12 +15210,12 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1627); + setState(1719); if (!(precpred(_ctx, 11))) throw FailedPredicateException(this, "precpred(_ctx, 11)"); - setState(1628); + setState(1720); match(ClickHouseParser::AND); - setState(1629); + setState(1721); columnExpr(12); break; } @@ -14543,12 +15224,12 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1630); + setState(1722); if (!(precpred(_ctx, 10))) throw FailedPredicateException(this, "precpred(_ctx, 10)"); - setState(1631); + setState(1723); match(ClickHouseParser::OR); - setState(1632); + setState(1724); columnExpr(11); break; } @@ -14557,24 +15238,24 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1633); + setState(1725); if (!(precpred(_ctx, 9))) throw FailedPredicateException(this, "precpred(_ctx, 9)"); - setState(1635); + setState(1727); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::NOT) { - setState(1634); + setState(1726); match(ClickHouseParser::NOT); } - setState(1637); + setState(1729); match(ClickHouseParser::BETWEEN); - setState(1638); + setState(1730); columnExpr(0); - setState(1639); + setState(1731); match(ClickHouseParser::AND); - setState(1640); + setState(1732); columnExpr(10); break; } @@ -14583,16 +15264,16 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1642); + setState(1734); if (!(precpred(_ctx, 8))) throw FailedPredicateException(this, "precpred(_ctx, 8)"); - setState(1643); + setState(1735); match(ClickHouseParser::QUERY); - setState(1644); + setState(1736); columnExpr(0); - setState(1645); + setState(1737); match(ClickHouseParser::COLON); - setState(1646); + setState(1738); columnExpr(8); break; } @@ -14601,14 +15282,14 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1648); + setState(1740); if (!(precpred(_ctx, 19))) throw FailedPredicateException(this, "precpred(_ctx, 19)"); - setState(1649); + setState(1741); match(ClickHouseParser::LBRACKET); - setState(1650); + setState(1742); columnExpr(0); - setState(1651); + setState(1743); match(ClickHouseParser::RBRACKET); break; } @@ -14617,12 +15298,12 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1653); + setState(1745); if (!(precpred(_ctx, 18))) throw FailedPredicateException(this, "precpred(_ctx, 18)"); - setState(1654); + setState(1746); match(ClickHouseParser::DOT); - setState(1655); + setState(1747); match(ClickHouseParser::DECIMAL_LITERAL); break; } @@ -14631,20 +15312,20 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1656); + setState(1748); if (!(precpred(_ctx, 13))) throw FailedPredicateException(this, "precpred(_ctx, 13)"); - setState(1657); + setState(1749); match(ClickHouseParser::IS); - setState(1659); + setState(1751); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::NOT) { - setState(1658); + setState(1750); match(ClickHouseParser::NOT); } - setState(1661); + setState(1753); match(ClickHouseParser::NULL_SQL); break; } @@ -14653,10 +15334,10 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleColumnExpr); - setState(1662); + setState(1754); if (!(precpred(_ctx, 7))) throw FailedPredicateException(this, "precpred(_ctx, 7)"); - setState(1666); + setState(1758); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::DATE: @@ -14664,15 +15345,15 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence case ClickHouseParser::ID: case ClickHouseParser::KEY: case ClickHouseParser::IDENTIFIER: { - setState(1663); + setState(1755); alias(); break; } case ClickHouseParser::AS: { - setState(1664); + setState(1756); match(ClickHouseParser::AS); - setState(1665); + setState(1757); identifier(); break; } @@ -14685,9 +15366,9 @@ ClickHouseParser::ColumnExprContext* ClickHouseParser::columnExpr(int precedence } } - setState(1672); + setState(1764); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 219, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 233, _ctx); } } catch (RecognitionException &e) { @@ -14735,7 +15416,7 @@ antlrcpp::Any ClickHouseParser::ColumnArgListContext::accept(tree::ParseTreeVisi ClickHouseParser::ColumnArgListContext* ClickHouseParser::columnArgList() { ColumnArgListContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 170, ClickHouseParser::RuleColumnArgList); + enterRule(_localctx, 176, ClickHouseParser::RuleColumnArgList); size_t _la = 0; auto onExit = finally([=] { @@ -14743,17 +15424,17 @@ ClickHouseParser::ColumnArgListContext* ClickHouseParser::columnArgList() { }); try { enterOuterAlt(_localctx, 1); - setState(1673); + setState(1765); columnArgExpr(); - setState(1678); + setState(1770); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1674); + setState(1766); match(ClickHouseParser::COMMA); - setState(1675); + setState(1767); columnArgExpr(); - setState(1680); + setState(1772); _errHandler->sync(this); _la = _input->LA(1); } @@ -14797,25 +15478,25 @@ antlrcpp::Any ClickHouseParser::ColumnArgExprContext::accept(tree::ParseTreeVisi ClickHouseParser::ColumnArgExprContext* ClickHouseParser::columnArgExpr() { ColumnArgExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 172, ClickHouseParser::RuleColumnArgExpr); + enterRule(_localctx, 178, ClickHouseParser::RuleColumnArgExpr); auto onExit = finally([=] { exitRule(); }); try { - setState(1683); + setState(1775); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 221, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 235, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(1681); + setState(1773); columnLambdaExpr(); break; } case 2: { enterOuterAlt(_localctx, 2); - setState(1682); + setState(1774); columnExpr(0); break; } @@ -14885,7 +15566,7 @@ antlrcpp::Any ClickHouseParser::ColumnLambdaExprContext::accept(tree::ParseTreeV ClickHouseParser::ColumnLambdaExprContext* ClickHouseParser::columnLambdaExpr() { ColumnLambdaExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 174, ClickHouseParser::RuleColumnLambdaExpr); + enterRule(_localctx, 180, ClickHouseParser::RuleColumnLambdaExpr); size_t _la = 0; auto onExit = finally([=] { @@ -14893,27 +15574,27 @@ ClickHouseParser::ColumnLambdaExprContext* ClickHouseParser::columnLambdaExpr() }); try { enterOuterAlt(_localctx, 1); - setState(1704); + setState(1796); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::LPAREN: { - setState(1685); + setState(1777); match(ClickHouseParser::LPAREN); - setState(1686); + setState(1778); identifier(); - setState(1691); + setState(1783); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1687); + setState(1779); match(ClickHouseParser::COMMA); - setState(1688); + setState(1780); identifier(); - setState(1693); + setState(1785); _errHandler->sync(this); _la = _input->LA(1); } - setState(1694); + setState(1786); match(ClickHouseParser::RPAREN); break; } @@ -15015,6 +15696,7 @@ ClickHouseParser::ColumnLambdaExprContext* ClickHouseParser::columnLambdaExpr() case ClickHouseParser::LOCAL: case ClickHouseParser::LOGS: case ClickHouseParser::MATERIALIZED: + case ClickHouseParser::MATERIALIZE: case ClickHouseParser::MAX: case ClickHouseParser::MERGES: case ClickHouseParser::MIN: @@ -15095,17 +15777,17 @@ ClickHouseParser::ColumnLambdaExprContext* ClickHouseParser::columnLambdaExpr() case ClickHouseParser::JSON_FALSE: case ClickHouseParser::JSON_TRUE: case ClickHouseParser::IDENTIFIER: { - setState(1696); + setState(1788); identifier(); - setState(1701); + setState(1793); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1697); + setState(1789); match(ClickHouseParser::COMMA); - setState(1698); + setState(1790); identifier(); - setState(1703); + setState(1795); _errHandler->sync(this); _la = _input->LA(1); } @@ -15115,9 +15797,9 @@ ClickHouseParser::ColumnLambdaExprContext* ClickHouseParser::columnLambdaExpr() default: throw NoViableAltException(this); } - setState(1706); + setState(1798); match(ClickHouseParser::ARROW); - setState(1707); + setState(1799); columnExpr(0); } @@ -15163,27 +15845,27 @@ antlrcpp::Any ClickHouseParser::ColumnIdentifierContext::accept(tree::ParseTreeV ClickHouseParser::ColumnIdentifierContext* ClickHouseParser::columnIdentifier() { ColumnIdentifierContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 176, ClickHouseParser::RuleColumnIdentifier); + enterRule(_localctx, 182, ClickHouseParser::RuleColumnIdentifier); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1712); + setState(1804); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 225, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 239, _ctx)) { case 1: { - setState(1709); + setState(1801); tableIdentifier(); - setState(1710); + setState(1802); match(ClickHouseParser::DOT); break; } } - setState(1714); + setState(1806); nestedIdentifier(); } @@ -15229,23 +15911,23 @@ antlrcpp::Any ClickHouseParser::NestedIdentifierContext::accept(tree::ParseTreeV ClickHouseParser::NestedIdentifierContext* ClickHouseParser::nestedIdentifier() { NestedIdentifierContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 178, ClickHouseParser::RuleNestedIdentifier); + enterRule(_localctx, 184, ClickHouseParser::RuleNestedIdentifier); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1716); + setState(1808); identifier(); - setState(1719); + setState(1811); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 226, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 240, _ctx)) { case 1: { - setState(1717); + setState(1809); match(ClickHouseParser::DOT); - setState(1718); + setState(1810); identifier(); break; } @@ -15368,8 +16050,8 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) ClickHouseParser::TableExprContext *_localctx = _tracker.createInstance(_ctx, parentState); ClickHouseParser::TableExprContext *previousContext = _localctx; (void)previousContext; // Silence compiler, in case the context is not used by generated code. - size_t startState = 180; - enterRecursionRule(_localctx, 180, ClickHouseParser::RuleTableExpr, precedence); + size_t startState = 186; + enterRecursionRule(_localctx, 186, ClickHouseParser::RuleTableExpr, precedence); @@ -15379,15 +16061,15 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) try { size_t alt; enterOuterAlt(_localctx, 1); - setState(1728); + setState(1820); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 227, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 241, _ctx)) { case 1: { _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1722); + setState(1814); tableIdentifier(); break; } @@ -15396,7 +16078,7 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1723); + setState(1815); tableFunctionExpr(); break; } @@ -15405,20 +16087,20 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) _localctx = _tracker.createInstance(_localctx); _ctx = _localctx; previousContext = _localctx; - setState(1724); + setState(1816); match(ClickHouseParser::LPAREN); - setState(1725); + setState(1817); selectUnionStmt(); - setState(1726); + setState(1818); match(ClickHouseParser::RPAREN); break; } } _ctx->stop = _input->LT(-1); - setState(1738); + setState(1830); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 229, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 243, _ctx); while (alt != 2 && alt != atn::ATN::INVALID_ALT_NUMBER) { if (alt == 1) { if (!_parseListeners.empty()) @@ -15427,10 +16109,10 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) auto newContext = _tracker.createInstance(_tracker.createInstance(parentContext, parentState)); _localctx = newContext; pushNewRecursionContext(newContext, startState, RuleTableExpr); - setState(1730); + setState(1822); if (!(precpred(_ctx, 1))) throw FailedPredicateException(this, "precpred(_ctx, 1)"); - setState(1734); + setState(1826); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::DATE: @@ -15438,15 +16120,15 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) case ClickHouseParser::ID: case ClickHouseParser::KEY: case ClickHouseParser::IDENTIFIER: { - setState(1731); + setState(1823); alias(); break; } case ClickHouseParser::AS: { - setState(1732); + setState(1824); match(ClickHouseParser::AS); - setState(1733); + setState(1825); identifier(); break; } @@ -15455,9 +16137,9 @@ ClickHouseParser::TableExprContext* ClickHouseParser::tableExpr(int precedence) throw NoViableAltException(this); } } - setState(1740); + setState(1832); _errHandler->sync(this); - alt = getInterpreter()->adaptivePredict(_input, 229, _ctx); + alt = getInterpreter()->adaptivePredict(_input, 243, _ctx); } } catch (RecognitionException &e) { @@ -15505,7 +16187,7 @@ antlrcpp::Any ClickHouseParser::TableFunctionExprContext::accept(tree::ParseTree ClickHouseParser::TableFunctionExprContext* ClickHouseParser::tableFunctionExpr() { TableFunctionExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 182, ClickHouseParser::RuleTableFunctionExpr); + enterRule(_localctx, 188, ClickHouseParser::RuleTableFunctionExpr); size_t _la = 0; auto onExit = finally([=] { @@ -15513,11 +16195,11 @@ ClickHouseParser::TableFunctionExprContext* ClickHouseParser::tableFunctionExpr( }); try { enterOuterAlt(_localctx, 1); - setState(1741); + setState(1833); identifier(); - setState(1742); + setState(1834); match(ClickHouseParser::LPAREN); - setState(1744); + setState(1836); _errHandler->sync(this); _la = _input->LA(1); @@ -15620,6 +16302,7 @@ ClickHouseParser::TableFunctionExprContext* ClickHouseParser::tableFunctionExpr( | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -15645,10 +16328,10 @@ ClickHouseParser::TableFunctionExprContext* ClickHouseParser::tableFunctionExpr( | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) | (1ULL << (ClickHouseParser::QUARTER - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -15706,14 +16389,14 @@ ClickHouseParser::TableFunctionExprContext* ClickHouseParser::tableFunctionExpr( | (1ULL << (ClickHouseParser::OCTAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::DECIMAL_LITERAL - 128)) | (1ULL << (ClickHouseParser::HEXADECIMAL_LITERAL - 128)) - | (1ULL << (ClickHouseParser::STRING_LITERAL - 128)))) != 0) || ((((_la - 194) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 194)) & ((1ULL << (ClickHouseParser::DASH - 194)) - | (1ULL << (ClickHouseParser::DOT - 194)) - | (1ULL << (ClickHouseParser::PLUS - 194)))) != 0)) { - setState(1743); + | (1ULL << (ClickHouseParser::STRING_LITERAL - 128)))) != 0) || ((((_la - 196) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 196)) & ((1ULL << (ClickHouseParser::DASH - 196)) + | (1ULL << (ClickHouseParser::DOT - 196)) + | (1ULL << (ClickHouseParser::PLUS - 196)))) != 0)) { + setState(1835); tableArgList(); } - setState(1746); + setState(1838); match(ClickHouseParser::RPAREN); } @@ -15759,27 +16442,27 @@ antlrcpp::Any ClickHouseParser::TableIdentifierContext::accept(tree::ParseTreeVi ClickHouseParser::TableIdentifierContext* ClickHouseParser::tableIdentifier() { TableIdentifierContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 184, ClickHouseParser::RuleTableIdentifier); + enterRule(_localctx, 190, ClickHouseParser::RuleTableIdentifier); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1751); + setState(1843); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 231, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 245, _ctx)) { case 1: { - setState(1748); + setState(1840); databaseIdentifier(); - setState(1749); + setState(1841); match(ClickHouseParser::DOT); break; } } - setState(1753); + setState(1845); identifier(); } @@ -15829,7 +16512,7 @@ antlrcpp::Any ClickHouseParser::TableArgListContext::accept(tree::ParseTreeVisit ClickHouseParser::TableArgListContext* ClickHouseParser::tableArgList() { TableArgListContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 186, ClickHouseParser::RuleTableArgList); + enterRule(_localctx, 192, ClickHouseParser::RuleTableArgList); size_t _la = 0; auto onExit = finally([=] { @@ -15837,17 +16520,17 @@ ClickHouseParser::TableArgListContext* ClickHouseParser::tableArgList() { }); try { enterOuterAlt(_localctx, 1); - setState(1755); + setState(1847); tableArgExpr(); - setState(1760); + setState(1852); _errHandler->sync(this); _la = _input->LA(1); while (_la == ClickHouseParser::COMMA) { - setState(1756); + setState(1848); match(ClickHouseParser::COMMA); - setState(1757); + setState(1849); tableArgExpr(); - setState(1762); + setState(1854); _errHandler->sync(this); _la = _input->LA(1); } @@ -15895,32 +16578,32 @@ antlrcpp::Any ClickHouseParser::TableArgExprContext::accept(tree::ParseTreeVisit ClickHouseParser::TableArgExprContext* ClickHouseParser::tableArgExpr() { TableArgExprContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 188, ClickHouseParser::RuleTableArgExpr); + enterRule(_localctx, 194, ClickHouseParser::RuleTableArgExpr); auto onExit = finally([=] { exitRule(); }); try { - setState(1766); + setState(1858); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 233, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 247, _ctx)) { case 1: { enterOuterAlt(_localctx, 1); - setState(1763); + setState(1855); tableIdentifier(); break; } case 2: { enterOuterAlt(_localctx, 2); - setState(1764); + setState(1856); tableFunctionExpr(); break; } case 3: { enterOuterAlt(_localctx, 3); - setState(1765); + setState(1857); literal(); break; } @@ -15962,14 +16645,14 @@ antlrcpp::Any ClickHouseParser::DatabaseIdentifierContext::accept(tree::ParseTre ClickHouseParser::DatabaseIdentifierContext* ClickHouseParser::databaseIdentifier() { DatabaseIdentifierContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 190, ClickHouseParser::RuleDatabaseIdentifier); + enterRule(_localctx, 196, ClickHouseParser::RuleDatabaseIdentifier); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1768); + setState(1860); identifier(); } @@ -16023,28 +16706,28 @@ antlrcpp::Any ClickHouseParser::FloatingLiteralContext::accept(tree::ParseTreeVi ClickHouseParser::FloatingLiteralContext* ClickHouseParser::floatingLiteral() { FloatingLiteralContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 192, ClickHouseParser::RuleFloatingLiteral); + enterRule(_localctx, 198, ClickHouseParser::RuleFloatingLiteral); size_t _la = 0; auto onExit = finally([=] { exitRule(); }); try { - setState(1778); + setState(1870); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::FLOATING_LITERAL: { enterOuterAlt(_localctx, 1); - setState(1770); + setState(1862); match(ClickHouseParser::FLOATING_LITERAL); break; } case ClickHouseParser::DOT: { enterOuterAlt(_localctx, 2); - setState(1771); + setState(1863); match(ClickHouseParser::DOT); - setState(1772); + setState(1864); _la = _input->LA(1); if (!(_la == ClickHouseParser::OCTAL_LITERAL @@ -16060,16 +16743,16 @@ ClickHouseParser::FloatingLiteralContext* ClickHouseParser::floatingLiteral() { case ClickHouseParser::DECIMAL_LITERAL: { enterOuterAlt(_localctx, 3); - setState(1773); + setState(1865); match(ClickHouseParser::DECIMAL_LITERAL); - setState(1774); + setState(1866); match(ClickHouseParser::DOT); - setState(1776); + setState(1868); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 234, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 248, _ctx)) { case 1: { - setState(1775); + setState(1867); _la = _input->LA(1); if (!(_la == ClickHouseParser::OCTAL_LITERAL @@ -16154,7 +16837,7 @@ antlrcpp::Any ClickHouseParser::NumberLiteralContext::accept(tree::ParseTreeVisi ClickHouseParser::NumberLiteralContext* ClickHouseParser::numberLiteral() { NumberLiteralContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 194, ClickHouseParser::RuleNumberLiteral); + enterRule(_localctx, 200, ClickHouseParser::RuleNumberLiteral); size_t _la = 0; auto onExit = finally([=] { @@ -16162,14 +16845,14 @@ ClickHouseParser::NumberLiteralContext* ClickHouseParser::numberLiteral() { }); try { enterOuterAlt(_localctx, 1); - setState(1781); + setState(1873); _errHandler->sync(this); _la = _input->LA(1); if (_la == ClickHouseParser::DASH || _la == ClickHouseParser::PLUS) { - setState(1780); + setState(1872); _la = _input->LA(1); if (!(_la == ClickHouseParser::DASH @@ -16181,41 +16864,41 @@ ClickHouseParser::NumberLiteralContext* ClickHouseParser::numberLiteral() { consume(); } } - setState(1789); + setState(1881); _errHandler->sync(this); - switch (getInterpreter()->adaptivePredict(_input, 237, _ctx)) { + switch (getInterpreter()->adaptivePredict(_input, 251, _ctx)) { case 1: { - setState(1783); + setState(1875); floatingLiteral(); break; } case 2: { - setState(1784); + setState(1876); match(ClickHouseParser::OCTAL_LITERAL); break; } case 3: { - setState(1785); + setState(1877); match(ClickHouseParser::DECIMAL_LITERAL); break; } case 4: { - setState(1786); + setState(1878); match(ClickHouseParser::HEXADECIMAL_LITERAL); break; } case 5: { - setState(1787); + setState(1879); match(ClickHouseParser::INF); break; } case 6: { - setState(1788); + setState(1880); match(ClickHouseParser::NAN_SQL); break; } @@ -16265,13 +16948,13 @@ antlrcpp::Any ClickHouseParser::LiteralContext::accept(tree::ParseTreeVisitor *v ClickHouseParser::LiteralContext* ClickHouseParser::literal() { LiteralContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 196, ClickHouseParser::RuleLiteral); + enterRule(_localctx, 202, ClickHouseParser::RuleLiteral); auto onExit = finally([=] { exitRule(); }); try { - setState(1794); + setState(1886); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::INF: @@ -16284,21 +16967,21 @@ ClickHouseParser::LiteralContext* ClickHouseParser::literal() { case ClickHouseParser::DOT: case ClickHouseParser::PLUS: { enterOuterAlt(_localctx, 1); - setState(1791); + setState(1883); numberLiteral(); break; } case ClickHouseParser::STRING_LITERAL: { enterOuterAlt(_localctx, 2); - setState(1792); + setState(1884); match(ClickHouseParser::STRING_LITERAL); break; } case ClickHouseParser::NULL_SQL: { enterOuterAlt(_localctx, 3); - setState(1793); + setState(1885); match(ClickHouseParser::NULL_SQL); break; } @@ -16370,7 +17053,7 @@ antlrcpp::Any ClickHouseParser::IntervalContext::accept(tree::ParseTreeVisitor * ClickHouseParser::IntervalContext* ClickHouseParser::interval() { IntervalContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 198, ClickHouseParser::RuleInterval); + enterRule(_localctx, 204, ClickHouseParser::RuleInterval); size_t _la = 0; auto onExit = finally([=] { @@ -16378,16 +17061,16 @@ ClickHouseParser::IntervalContext* ClickHouseParser::interval() { }); try { enterOuterAlt(_localctx, 1); - setState(1796); + setState(1888); _la = _input->LA(1); if (!(_la == ClickHouseParser::DAY || ((((_la - 72) & ~ 0x3fULL) == 0) && ((1ULL << (_la - 72)) & ((1ULL << (ClickHouseParser::HOUR - 72)) | (1ULL << (ClickHouseParser::MINUTE - 72)) | (1ULL << (ClickHouseParser::MONTH - 72)) - | (1ULL << (ClickHouseParser::QUARTER - 72)) - | (1ULL << (ClickHouseParser::SECOND - 72)))) != 0) || _la == ClickHouseParser::WEEK - - || _la == ClickHouseParser::YEAR)) { + | (1ULL << (ClickHouseParser::QUARTER - 72)))) != 0) || ((((_la - 137) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 137)) & ((1ULL << (ClickHouseParser::SECOND - 137)) + | (1ULL << (ClickHouseParser::WEEK - 137)) + | (1ULL << (ClickHouseParser::YEAR - 137)))) != 0))) { _errHandler->recoverInline(this); } else { @@ -16795,6 +17478,10 @@ tree::TerminalNode* ClickHouseParser::KeywordContext::LOGS() { return getToken(ClickHouseParser::LOGS, 0); } +tree::TerminalNode* ClickHouseParser::KeywordContext::MATERIALIZE() { + return getToken(ClickHouseParser::MATERIALIZE, 0); +} + tree::TerminalNode* ClickHouseParser::KeywordContext::MATERIALIZED() { return getToken(ClickHouseParser::MATERIALIZED, 0); } @@ -17098,7 +17785,7 @@ antlrcpp::Any ClickHouseParser::KeywordContext::accept(tree::ParseTreeVisitor *v ClickHouseParser::KeywordContext* ClickHouseParser::keyword() { KeywordContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 200, ClickHouseParser::RuleKeyword); + enterRule(_localctx, 206, ClickHouseParser::RuleKeyword); size_t _la = 0; auto onExit = finally([=] { @@ -17106,7 +17793,7 @@ ClickHouseParser::KeywordContext* ClickHouseParser::keyword() { }); try { enterOuterAlt(_localctx, 1); - setState(1798); + setState(1890); _la = _input->LA(1); if (!((((_la & ~ 0x3fULL) == 0) && ((1ULL << _la) & ((1ULL << ClickHouseParser::AFTER) @@ -17204,6 +17891,7 @@ ClickHouseParser::KeywordContext* ClickHouseParser::keyword() { | (1ULL << (ClickHouseParser::LOCAL - 64)) | (1ULL << (ClickHouseParser::LOGS - 64)) | (1ULL << (ClickHouseParser::MATERIALIZED - 64)) + | (1ULL << (ClickHouseParser::MATERIALIZE - 64)) | (1ULL << (ClickHouseParser::MAX - 64)) | (1ULL << (ClickHouseParser::MERGES - 64)) | (1ULL << (ClickHouseParser::MIN - 64)) @@ -17224,10 +17912,10 @@ ClickHouseParser::KeywordContext* ClickHouseParser::keyword() { | (1ULL << (ClickHouseParser::POPULATE - 64)) | (1ULL << (ClickHouseParser::PREWHERE - 64)) | (1ULL << (ClickHouseParser::PRIMARY - 64)) - | (1ULL << (ClickHouseParser::RANGE - 64)) - | (1ULL << (ClickHouseParser::RELOAD - 64)) - | (1ULL << (ClickHouseParser::REMOVE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && - ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RENAME - 128)) + | (1ULL << (ClickHouseParser::RANGE - 64)))) != 0) || ((((_la - 128) & ~ 0x3fULL) == 0) && + ((1ULL << (_la - 128)) & ((1ULL << (ClickHouseParser::RELOAD - 128)) + | (1ULL << (ClickHouseParser::REMOVE - 128)) + | (1ULL << (ClickHouseParser::RENAME - 128)) | (1ULL << (ClickHouseParser::REPLACE - 128)) | (1ULL << (ClickHouseParser::REPLICA - 128)) | (1ULL << (ClickHouseParser::REPLICATED - 128)) @@ -17331,7 +18019,7 @@ antlrcpp::Any ClickHouseParser::KeywordForAliasContext::accept(tree::ParseTreeVi ClickHouseParser::KeywordForAliasContext* ClickHouseParser::keywordForAlias() { KeywordForAliasContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 202, ClickHouseParser::RuleKeywordForAlias); + enterRule(_localctx, 208, ClickHouseParser::RuleKeywordForAlias); size_t _la = 0; auto onExit = finally([=] { @@ -17339,7 +18027,7 @@ ClickHouseParser::KeywordForAliasContext* ClickHouseParser::keywordForAlias() { }); try { enterOuterAlt(_localctx, 1); - setState(1800); + setState(1892); _la = _input->LA(1); if (!(((((_la - 33) & ~ 0x3fULL) == 0) && ((1ULL << (_la - 33)) & ((1ULL << (ClickHouseParser::DATE - 33)) @@ -17392,18 +18080,18 @@ antlrcpp::Any ClickHouseParser::AliasContext::accept(tree::ParseTreeVisitor *vis ClickHouseParser::AliasContext* ClickHouseParser::alias() { AliasContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 204, ClickHouseParser::RuleAlias); + enterRule(_localctx, 210, ClickHouseParser::RuleAlias); auto onExit = finally([=] { exitRule(); }); try { - setState(1804); + setState(1896); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::IDENTIFIER: { enterOuterAlt(_localctx, 1); - setState(1802); + setState(1894); match(ClickHouseParser::IDENTIFIER); break; } @@ -17413,7 +18101,7 @@ ClickHouseParser::AliasContext* ClickHouseParser::alias() { case ClickHouseParser::ID: case ClickHouseParser::KEY: { enterOuterAlt(_localctx, 2); - setState(1803); + setState(1895); keywordForAlias(); break; } @@ -17465,18 +18153,18 @@ antlrcpp::Any ClickHouseParser::IdentifierContext::accept(tree::ParseTreeVisitor ClickHouseParser::IdentifierContext* ClickHouseParser::identifier() { IdentifierContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 206, ClickHouseParser::RuleIdentifier); + enterRule(_localctx, 212, ClickHouseParser::RuleIdentifier); auto onExit = finally([=] { exitRule(); }); try { - setState(1809); + setState(1901); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::IDENTIFIER: { enterOuterAlt(_localctx, 1); - setState(1806); + setState(1898); match(ClickHouseParser::IDENTIFIER); break; } @@ -17490,7 +18178,7 @@ ClickHouseParser::IdentifierContext* ClickHouseParser::identifier() { case ClickHouseParser::WEEK: case ClickHouseParser::YEAR: { enterOuterAlt(_localctx, 2); - setState(1807); + setState(1899); interval(); break; } @@ -17590,6 +18278,7 @@ ClickHouseParser::IdentifierContext* ClickHouseParser::identifier() { case ClickHouseParser::LOCAL: case ClickHouseParser::LOGS: case ClickHouseParser::MATERIALIZED: + case ClickHouseParser::MATERIALIZE: case ClickHouseParser::MAX: case ClickHouseParser::MERGES: case ClickHouseParser::MIN: @@ -17664,7 +18353,7 @@ ClickHouseParser::IdentifierContext* ClickHouseParser::identifier() { case ClickHouseParser::JSON_FALSE: case ClickHouseParser::JSON_TRUE: { enterOuterAlt(_localctx, 3); - setState(1808); + setState(1900); keyword(); break; } @@ -17712,13 +18401,13 @@ antlrcpp::Any ClickHouseParser::IdentifierOrNullContext::accept(tree::ParseTreeV ClickHouseParser::IdentifierOrNullContext* ClickHouseParser::identifierOrNull() { IdentifierOrNullContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 208, ClickHouseParser::RuleIdentifierOrNull); + enterRule(_localctx, 214, ClickHouseParser::RuleIdentifierOrNull); auto onExit = finally([=] { exitRule(); }); try { - setState(1813); + setState(1905); _errHandler->sync(this); switch (_input->LA(1)) { case ClickHouseParser::AFTER: @@ -17818,6 +18507,7 @@ ClickHouseParser::IdentifierOrNullContext* ClickHouseParser::identifierOrNull() case ClickHouseParser::LOCAL: case ClickHouseParser::LOGS: case ClickHouseParser::MATERIALIZED: + case ClickHouseParser::MATERIALIZE: case ClickHouseParser::MAX: case ClickHouseParser::MERGES: case ClickHouseParser::MIN: @@ -17899,14 +18589,14 @@ ClickHouseParser::IdentifierOrNullContext* ClickHouseParser::identifierOrNull() case ClickHouseParser::JSON_TRUE: case ClickHouseParser::IDENTIFIER: { enterOuterAlt(_localctx, 1); - setState(1811); + setState(1903); identifier(); break; } case ClickHouseParser::NULL_SQL: { enterOuterAlt(_localctx, 2); - setState(1812); + setState(1904); match(ClickHouseParser::NULL_SQL); break; } @@ -17958,18 +18648,18 @@ antlrcpp::Any ClickHouseParser::EnumValueContext::accept(tree::ParseTreeVisitor ClickHouseParser::EnumValueContext* ClickHouseParser::enumValue() { EnumValueContext *_localctx = _tracker.createInstance(_ctx, getState()); - enterRule(_localctx, 210, ClickHouseParser::RuleEnumValue); + enterRule(_localctx, 216, ClickHouseParser::RuleEnumValue); auto onExit = finally([=] { exitRule(); }); try { enterOuterAlt(_localctx, 1); - setState(1815); + setState(1907); match(ClickHouseParser::STRING_LITERAL); - setState(1816); + setState(1908); match(ClickHouseParser::EQ_SINGLE); - setState(1817); + setState(1909); numberLiteral(); } @@ -17987,9 +18677,9 @@ bool ClickHouseParser::sempred(RuleContext *context, size_t ruleIndex, size_t pr case 12: return dictionaryAttrDfntSempred(dynamic_cast(context), predicateIndex); case 13: return dictionaryEngineClauseSempred(dynamic_cast(context), predicateIndex); case 26: return engineClauseSempred(dynamic_cast(context), predicateIndex); - case 64: return joinExprSempred(dynamic_cast(context), predicateIndex); - case 84: return columnExprSempred(dynamic_cast(context), predicateIndex); - case 90: return tableExprSempred(dynamic_cast(context), predicateIndex); + case 67: return joinExprSempred(dynamic_cast(context), predicateIndex); + case 87: return columnExprSempred(dynamic_cast(context), predicateIndex); + case 93: return tableExprSempred(dynamic_cast(context), predicateIndex); default: break; @@ -18098,21 +18788,22 @@ std::vector ClickHouseParser::_ruleNames = { "clusterClause", "uuidClause", "destinationClause", "subqueryClause", "tableSchemaClause", "engineClause", "partitionByClause", "primaryKeyClause", "sampleByClause", "ttlClause", "engineExpr", "tableElementExpr", "tableColumnDfnt", - "tableColumnPropertyExpr", "tableIndexDfnt", "codecExpr", "codecArgExpr", - "ttlExpr", "describeStmt", "dropStmt", "existsStmt", "explainStmt", "insertStmt", - "columnsClause", "dataClause", "killStmt", "optimizeStmt", "renameStmt", - "selectUnionStmt", "selectStmtWithParens", "selectStmt", "withClause", - "topClause", "fromClause", "arrayJoinClause", "prewhereClause", "whereClause", - "groupByClause", "havingClause", "orderByClause", "limitByClause", "limitClause", - "settingsClause", "joinExpr", "joinOp", "joinOpCross", "joinConstraintClause", - "sampleClause", "limitExpr", "orderExprList", "orderExpr", "ratioExpr", - "settingExprList", "settingExpr", "setStmt", "showStmt", "systemStmt", - "truncateStmt", "useStmt", "watchStmt", "columnTypeExpr", "columnExprList", - "columnsExpr", "columnExpr", "columnArgList", "columnArgExpr", "columnLambdaExpr", - "columnIdentifier", "nestedIdentifier", "tableExpr", "tableFunctionExpr", - "tableIdentifier", "tableArgList", "tableArgExpr", "databaseIdentifier", - "floatingLiteral", "numberLiteral", "literal", "interval", "keyword", - "keywordForAlias", "alias", "identifier", "identifierOrNull", "enumValue" + "tableColumnPropertyExpr", "tableIndexDfnt", "tableProjectionDfnt", "codecExpr", + "codecArgExpr", "ttlExpr", "describeStmt", "dropStmt", "existsStmt", "explainStmt", + "insertStmt", "columnsClause", "dataClause", "killStmt", "optimizeStmt", + "renameStmt", "projectionSelectStmt", "selectUnionStmt", "selectStmtWithParens", + "selectStmt", "withClause", "topClause", "fromClause", "arrayJoinClause", + "prewhereClause", "whereClause", "groupByClause", "havingClause", "orderByClause", + "projectionOrderByClause", "limitByClause", "limitClause", "settingsClause", + "joinExpr", "joinOp", "joinOpCross", "joinConstraintClause", "sampleClause", + "limitExpr", "orderExprList", "orderExpr", "ratioExpr", "settingExprList", + "settingExpr", "setStmt", "showStmt", "systemStmt", "truncateStmt", "useStmt", + "watchStmt", "columnTypeExpr", "columnExprList", "columnsExpr", "columnExpr", + "columnArgList", "columnArgExpr", "columnLambdaExpr", "columnIdentifier", + "nestedIdentifier", "tableExpr", "tableFunctionExpr", "tableIdentifier", + "tableArgList", "tableArgExpr", "databaseIdentifier", "floatingLiteral", + "numberLiteral", "literal", "interval", "keyword", "keywordForAlias", + "alias", "identifier", "identifierOrNull", "enumValue" }; std::vector ClickHouseParser::_literalNames = { @@ -18125,11 +18816,11 @@ std::vector ClickHouseParser::_literalNames = { "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", - "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "'false'", - "'true'", "", "", "", "", "", "", "'->'", "'*'", "'`'", "'\\'", "':'", - "','", "'||'", "'-'", "'.'", "'=='", "'='", "'>='", "'>'", "'{'", "'['", - "'<='", "'('", "'<'", "", "'%'", "'+'", "'?'", "'\"'", "'''", "'}'", "']'", - "')'", "';'", "'/'", "'_'" + "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "", + "", "'false'", "'true'", "", "", "", "", "", "", "'->'", "'*'", "'`'", + "'\\'", "':'", "','", "'||'", "'-'", "'.'", "'=='", "'='", "'>='", "'>'", + "'{'", "'['", "'<='", "'('", "'<'", "", "'%'", "'+'", "'?'", "'\"'", "'''", + "'}'", "']'", "')'", "';'", "'/'", "'_'" }; std::vector ClickHouseParser::_symbolicNames = { @@ -18146,16 +18837,16 @@ std::vector ClickHouseParser::_symbolicNames = { "INDEX", "INF", "INJECTIVE", "INNER", "INSERT", "INTERVAL", "INTO", "IS", "IS_OBJECT_ID", "JOIN", "KEY", "KILL", "LAST", "LAYOUT", "LEADING", "LEFT", "LIFETIME", "LIKE", "LIMIT", "LIVE", "LOCAL", "LOGS", "MATERIALIZED", - "MAX", "MERGES", "MIN", "MINUTE", "MODIFY", "MONTH", "MOVE", "MUTATION", - "NAN_SQL", "NO", "NOT", "NULL_SQL", "NULLS", "OFFSET", "ON", "OPTIMIZE", - "OR", "ORDER", "OUTER", "OUTFILE", "PARTITION", "POPULATE", "PREWHERE", - "PRIMARY", "QUARTER", "RANGE", "RELOAD", "REMOVE", "RENAME", "REPLACE", - "REPLICA", "REPLICATED", "RIGHT", "ROLLUP", "SAMPLE", "SECOND", "SELECT", - "SEMI", "SENDS", "SET", "SETTINGS", "SHOW", "SOURCE", "START", "STOP", - "SUBSTRING", "SYNC", "SYNTAX", "SYSTEM", "TABLE", "TABLES", "TEMPORARY", - "TEST", "THEN", "TIES", "TIMEOUT", "TIMESTAMP", "TO", "TOP", "TOTALS", - "TRAILING", "TRIM", "TRUNCATE", "TTL", "TYPE", "UNION", "UPDATE", "USE", - "USING", "UUID", "VALUES", "VIEW", "VOLUME", "WATCH", "WEEK", "WHEN", + "MATERIALIZE", "MAX", "MERGES", "MIN", "MINUTE", "MODIFY", "MONTH", "MOVE", + "MUTATION", "NAN_SQL", "NO", "NOT", "NULL_SQL", "NULLS", "OFFSET", "ON", + "OPTIMIZE", "OR", "ORDER", "OUTER", "OUTFILE", "PARTITION", "POPULATE", + "PREWHERE", "PRIMARY", "PROJECTION", "QUARTER", "RANGE", "RELOAD", "REMOVE", + "RENAME", "REPLACE", "REPLICA", "REPLICATED", "RIGHT", "ROLLUP", "SAMPLE", + "SECOND", "SELECT", "SEMI", "SENDS", "SET", "SETTINGS", "SHOW", "SOURCE", + "START", "STOP", "SUBSTRING", "SYNC", "SYNTAX", "SYSTEM", "TABLE", "TABLES", + "TEMPORARY", "TEST", "THEN", "TIES", "TIMEOUT", "TIMESTAMP", "TO", "TOP", + "TOTALS", "TRAILING", "TRIM", "TRUNCATE", "TTL", "TYPE", "UNION", "UPDATE", + "USE", "USING", "UUID", "VALUES", "VIEW", "VOLUME", "WATCH", "WEEK", "WHEN", "WHERE", "WITH", "YEAR", "JSON_FALSE", "JSON_TRUE", "IDENTIFIER", "FLOATING_LITERAL", "OCTAL_LITERAL", "DECIMAL_LITERAL", "HEXADECIMAL_LITERAL", "STRING_LITERAL", "ARROW", "ASTERISK", "BACKQUOTE", "BACKSLASH", "COLON", "COMMA", "CONCAT", @@ -18185,7 +18876,7 @@ ClickHouseParser::Initializer::Initializer() { _serializedATN = { 0x3, 0x608b, 0xa72a, 0x8133, 0xb9ed, 0x417c, 0x3be7, 0x7786, 0x5964, - 0x3, 0xdd, 0x71e, 0x4, 0x2, 0x9, 0x2, 0x4, 0x3, 0x9, 0x3, 0x4, 0x4, + 0x3, 0xdf, 0x77a, 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, @@ -18217,275 +18908,290 @@ ClickHouseParser::Initializer::Initializer() { 0x60, 0x9, 0x60, 0x4, 0x61, 0x9, 0x61, 0x4, 0x62, 0x9, 0x62, 0x4, 0x63, 0x9, 0x63, 0x4, 0x64, 0x9, 0x64, 0x4, 0x65, 0x9, 0x65, 0x4, 0x66, 0x9, 0x66, 0x4, 0x67, 0x9, 0x67, 0x4, 0x68, 0x9, 0x68, 0x4, 0x69, 0x9, 0x69, - 0x4, 0x6a, 0x9, 0x6a, 0x4, 0x6b, 0x9, 0x6b, 0x3, 0x2, 0x3, 0x2, 0x3, - 0x2, 0x3, 0x2, 0x5, 0x2, 0xdb, 0xa, 0x2, 0x3, 0x2, 0x3, 0x2, 0x5, 0x2, - 0xdf, 0xa, 0x2, 0x3, 0x2, 0x5, 0x2, 0xe2, 0xa, 0x2, 0x3, 0x2, 0x5, 0x2, - 0xe5, 0xa, 0x2, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, + 0x4, 0x6a, 0x9, 0x6a, 0x4, 0x6b, 0x9, 0x6b, 0x4, 0x6c, 0x9, 0x6c, 0x4, + 0x6d, 0x9, 0x6d, 0x4, 0x6e, 0x9, 0x6e, 0x3, 0x2, 0x3, 0x2, 0x3, 0x2, + 0x3, 0x2, 0x5, 0x2, 0xe1, 0xa, 0x2, 0x3, 0x2, 0x3, 0x2, 0x5, 0x2, 0xe5, + 0xa, 0x2, 0x3, 0x2, 0x5, 0x2, 0xe8, 0xa, 0x2, 0x3, 0x2, 0x5, 0x2, 0xeb, + 0xa, 0x2, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, - 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x5, 0x3, 0xf9, - 0xa, 0x3, 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x5, 0x4, 0xff, 0xa, - 0x4, 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x7, 0x4, 0x104, 0xa, 0x4, 0xc, 0x4, - 0xe, 0x4, 0x107, 0xb, 0x4, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, - 0x5, 0x5, 0x5, 0x10e, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, - 0x113, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, - 0x5, 0x11a, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x11f, - 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x125, 0xa, - 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x12b, 0xa, 0x5, - 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x130, 0xa, 0x5, 0x3, 0x5, 0x3, - 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x136, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, + 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x3, 0x5, 0x3, 0xff, 0xa, + 0x3, 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x5, 0x4, 0x105, 0xa, 0x4, + 0x3, 0x4, 0x3, 0x4, 0x3, 0x4, 0x7, 0x4, 0x10a, 0xa, 0x4, 0xc, 0x4, 0xe, + 0x4, 0x10d, 0xb, 0x4, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, + 0x5, 0x5, 0x114, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x119, + 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, + 0x120, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x125, 0xa, + 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x12c, + 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x131, 0xa, 0x5, 0x3, + 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x137, 0xa, 0x5, 0x3, 0x5, + 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x13d, 0xa, 0x5, 0x3, 0x5, 0x3, + 0x5, 0x3, 0x5, 0x5, 0x5, 0x142, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, + 0x3, 0x5, 0x5, 0x5, 0x148, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, + 0x5, 0x14d, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, + 0x153, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x158, 0xa, + 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x15e, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, - 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x144, 0xa, 0x5, 0x3, 0x5, 0x3, - 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x14b, 0xa, 0x5, 0x3, 0x5, - 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x152, 0xa, 0x5, 0x3, - 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x158, 0xa, 0x5, 0x3, 0x5, - 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, - 0x161, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, - 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x16b, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, - 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, - 0x175, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, + 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x16c, 0xa, + 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x173, + 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, + 0x17a, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, + 0x5, 0x181, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, + 0x187, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x18c, 0xa, + 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x192, 0xa, 0x5, + 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x197, 0xa, 0x5, 0x3, 0x5, 0x3, + 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x19d, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, + 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x1a6, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, - 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x189, - 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, - 0x5, 0x5, 0x191, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, + 0x5, 0x3, 0x5, 0x5, 0x5, 0x1b0, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, + 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x1ba, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, - 0x5, 0x3, 0x5, 0x5, 0x5, 0x1a0, 0xa, 0x5, 0x3, 0x6, 0x3, 0x6, 0x3, 0x6, - 0x7, 0x6, 0x1a5, 0xa, 0x6, 0xc, 0x6, 0xe, 0x6, 0x1a8, 0xb, 0x6, 0x3, - 0x7, 0x3, 0x7, 0x3, 0x7, 0x3, 0x7, 0x3, 0x8, 0x3, 0x8, 0x3, 0x9, 0x3, - 0x9, 0x3, 0x9, 0x3, 0x9, 0x3, 0x9, 0x5, 0x9, 0x1b5, 0xa, 0x9, 0x3, 0xa, - 0x3, 0xa, 0x3, 0xa, 0x3, 0xa, 0x5, 0xa, 0x1bb, 0xa, 0xa, 0x3, 0xb, 0x3, - 0xb, 0x3, 0xb, 0x3, 0xb, 0x5, 0xb, 0x1c1, 0xa, 0xb, 0x3, 0xc, 0x3, 0xc, - 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x1c8, 0xa, 0xc, 0x3, 0xc, 0x3, - 0xc, 0x5, 0xc, 0x1cc, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x1cf, 0xa, 0xc, - 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x1d6, 0xa, - 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x1da, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, - 0x1dd, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, - 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x1e8, 0xa, 0xc, 0x3, 0xc, - 0x3, 0xc, 0x5, 0xc, 0x1ec, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x1ef, 0xa, - 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x1f4, 0xa, 0xc, 0x5, 0xc, - 0x1f6, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x1f9, 0xa, 0xc, 0x3, 0xc, 0x5, - 0xc, 0x1fc, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, - 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x206, 0xa, 0xc, 0x3, 0xc, 0x3, - 0xc, 0x5, 0xc, 0x20a, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x20d, 0xa, 0xc, - 0x3, 0xc, 0x5, 0xc, 0x210, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, - 0xc, 0x215, 0xa, 0xc, 0x5, 0xc, 0x217, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, - 0x3, 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, 0x5, - 0xc, 0x22d, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x230, 0xa, 0xc, 0x3, 0xc, - 0x5, 0xc, 0x233, 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x238, - 0xa, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x23e, 0xa, - 0xc, 0x3, 0xc, 0x3, 0xc, 0x5, 0xc, 0x242, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, - 0x245, 0xa, 0xc, 0x3, 0xc, 0x5, 0xc, 0x248, 0xa, 0xc, 0x3, 0xc, 0x3, - 0xc, 0x5, 0xc, 0x24c, 0xa, 0xc, 0x3, 0xd, 0x3, 0xd, 0x3, 0xd, 0x3, 0xd, - 0x7, 0xd, 0x252, 0xa, 0xd, 0xc, 0xd, 0xe, 0xd, 0x255, 0xb, 0xd, 0x3, - 0xd, 0x3, 0xd, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, 0xe, 0x3, + 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, + 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, 0x1ce, 0xa, 0x5, + 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x5, 0x5, + 0x1d6, 0xa, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, + 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, 0x5, 0x3, + 0x5, 0x5, 0x5, 0x1e5, 0xa, 0x5, 0x3, 0x6, 0x3, 0x6, 0x3, 0x6, 0x7, 0x6, + 0x1ea, 0xa, 0x6, 0xc, 0x6, 0xe, 0x6, 0x1ed, 0xb, 0x6, 0x3, 0x7, 0x3, + 0x7, 0x3, 0x7, 0x3, 0x7, 0x3, 0x8, 0x3, 0x8, 0x3, 0x9, 0x3, 0x9, 0x3, + 0x9, 0x3, 0x9, 0x3, 0x9, 0x5, 0x9, 0x1fa, 0xa, 0x9, 0x3, 0xa, 0x3, 0xa, + 0x3, 0xa, 0x3, 0xa, 0x5, 0xa, 0x200, 0xa, 0xa, 0x3, 0xb, 0x3, 0xb, 0x3, + 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, + 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, 0x7, 0xe, 0x26e, 0xa, 0xe, 0xc, 0xe, 0xe, 0xe, 0x271, - 0xb, 0xe, 0x3, 0xf, 0x5, 0xf, 0x274, 0xa, 0xf, 0x3, 0xf, 0x3, 0xf, 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, 0x3, 0xf, 0x3, 0xf, 0x7, 0xf, 0x28a, 0xa, 0xf, 0xc, 0xf, - 0xe, 0xf, 0x28d, 0xb, 0xf, 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, 0x3, 0x10, - 0x3, 0x11, 0x3, 0x11, 0x3, 0x11, 0x3, 0x11, 0x5, 0x11, 0x297, 0xa, 0x11, - 0x3, 0x11, 0x5, 0x11, 0x29a, 0xa, 0x11, 0x3, 0x12, 0x3, 0x12, 0x3, 0x12, - 0x3, 0x12, 0x3, 0x12, 0x7, 0x12, 0x2a1, 0xa, 0x12, 0xc, 0x12, 0xe, 0x12, - 0x2a4, 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, 0x2b4, 0xa, 0x13, 0x3, 0x13, - 0x3, 0x13, 0x3, 0x14, 0x3, 0x14, 0x3, 0x14, 0x3, 0x14, 0x3, 0x14, 0x7, - 0x14, 0x2bd, 0xa, 0x14, 0xc, 0x14, 0xe, 0x14, 0x2c0, 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, 0x2d1, 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, 0x2de, 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, - 0x2ed, 0xa, 0x1b, 0xc, 0x1b, 0xe, 0x1b, 0x2f0, 0xb, 0x1b, 0x3, 0x1b, - 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x3, 0x1b, 0x5, 0x1b, 0x2f8, - 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, + 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, 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, 0x313, 0xa, 0x1c, 0xc, 0x1c, 0xe, 0x1c, 0x316, 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, 0x328, 0xa, 0x20, 0xc, 0x20, - 0xe, 0x20, 0x32b, 0xb, 0x20, 0x3, 0x21, 0x3, 0x21, 0x5, 0x21, 0x32f, - 0xa, 0x21, 0x3, 0x21, 0x3, 0x21, 0x3, 0x21, 0x5, 0x21, 0x334, 0xa, 0x21, - 0x3, 0x21, 0x5, 0x21, 0x337, 0xa, 0x21, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, - 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x3, 0x22, 0x5, 0x22, 0x341, - 0xa, 0x22, 0x3, 0x23, 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x346, 0xa, 0x23, - 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x34a, 0xa, 0x23, 0x3, 0x23, 0x5, 0x23, - 0x34d, 0xa, 0x23, 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x351, 0xa, 0x23, - 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x355, 0xa, 0x23, 0x3, 0x23, 0x3, 0x23, - 0x3, 0x23, 0x5, 0x23, 0x35a, 0xa, 0x23, 0x3, 0x23, 0x5, 0x23, 0x35d, - 0xa, 0x23, 0x3, 0x23, 0x3, 0x23, 0x5, 0x23, 0x361, 0xa, 0x23, 0x5, 0x23, - 0x363, 0xa, 0x23, 0x3, 0x24, 0x3, 0x24, 0x3, 0x24, 0x3, 0x25, 0x3, 0x25, + 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, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x25, 0x3, 0x26, 0x3, - 0x26, 0x3, 0x26, 0x3, 0x26, 0x3, 0x26, 0x7, 0x26, 0x374, 0xa, 0x26, - 0xc, 0x26, 0xe, 0x26, 0x377, 0xb, 0x26, 0x3, 0x26, 0x3, 0x26, 0x3, 0x27, - 0x3, 0x27, 0x3, 0x27, 0x5, 0x27, 0x37e, 0xa, 0x27, 0x3, 0x27, 0x5, 0x27, - 0x381, 0xa, 0x27, 0x3, 0x28, 0x3, 0x28, 0x3, 0x28, 0x3, 0x28, 0x3, 0x28, - 0x3, 0x28, 0x3, 0x28, 0x3, 0x28, 0x5, 0x28, 0x38b, 0xa, 0x28, 0x3, 0x29, - 0x3, 0x29, 0x5, 0x29, 0x38f, 0xa, 0x29, 0x3, 0x29, 0x3, 0x29, 0x3, 0x2a, - 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x5, 0x2a, 0x397, 0xa, 0x2a, 0x3, 0x2a, - 0x3, 0x2a, 0x5, 0x2a, 0x39b, 0xa, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x3, 0x2a, - 0x5, 0x2a, 0x3a0, 0xa, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x5, 0x2a, 0x3a4, - 0xa, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x5, 0x2a, 0x3a8, 0xa, 0x2a, 0x3, 0x2a, - 0x3, 0x2a, 0x5, 0x2a, 0x3ac, 0xa, 0x2a, 0x3, 0x2a, 0x3, 0x2a, 0x5, 0x2a, - 0x3b0, 0xa, 0x2a, 0x5, 0x2a, 0x3b2, 0xa, 0x2a, 0x3, 0x2b, 0x3, 0x2b, - 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x3ba, 0xa, 0x2b, - 0x3, 0x2b, 0x3, 0x2b, 0x5, 0x2b, 0x3be, 0xa, 0x2b, 0x3, 0x2b, 0x5, 0x2b, - 0x3c1, 0xa, 0x2b, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2c, 0x3, 0x2d, - 0x3, 0x2d, 0x3, 0x2d, 0x5, 0x2d, 0x3ca, 0xa, 0x2d, 0x3, 0x2d, 0x3, 0x2d, - 0x3, 0x2d, 0x5, 0x2d, 0x3cf, 0xa, 0x2d, 0x3, 0x2d, 0x5, 0x2d, 0x3d2, - 0xa, 0x2d, 0x3, 0x2d, 0x3, 0x2d, 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x3, - 0x2e, 0x7, 0x2e, 0x3da, 0xa, 0x2e, 0xc, 0x2e, 0xe, 0x2e, 0x3dd, 0xb, - 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, - 0x3, 0x2f, 0x5, 0x2f, 0x3e6, 0xa, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x5, 0x2f, - 0x3ea, 0xa, 0x2f, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x5, 0x30, 0x3ef, - 0xa, 0x30, 0x3, 0x30, 0x3, 0x30, 0x5, 0x30, 0x3f3, 0xa, 0x30, 0x3, 0x31, - 0x3, 0x31, 0x3, 0x31, 0x3, 0x31, 0x5, 0x31, 0x3f9, 0xa, 0x31, 0x3, 0x31, - 0x5, 0x31, 0x3fc, 0xa, 0x31, 0x3, 0x31, 0x5, 0x31, 0x3ff, 0xa, 0x31, - 0x3, 0x31, 0x5, 0x31, 0x402, 0xa, 0x31, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, - 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, - 0x32, 0x7, 0x32, 0x40e, 0xa, 0x32, 0xc, 0x32, 0xe, 0x32, 0x411, 0xb, - 0x32, 0x3, 0x32, 0x5, 0x32, 0x414, 0xa, 0x32, 0x3, 0x33, 0x3, 0x33, - 0x3, 0x33, 0x3, 0x33, 0x7, 0x33, 0x41a, 0xa, 0x33, 0xc, 0x33, 0xe, 0x33, - 0x41d, 0xb, 0x33, 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, - 0x5, 0x34, 0x424, 0xa, 0x34, 0x3, 0x35, 0x5, 0x35, 0x427, 0xa, 0x35, - 0x3, 0x35, 0x3, 0x35, 0x5, 0x35, 0x42b, 0xa, 0x35, 0x3, 0x35, 0x5, 0x35, - 0x42e, 0xa, 0x35, 0x3, 0x35, 0x3, 0x35, 0x5, 0x35, 0x432, 0xa, 0x35, - 0x3, 0x35, 0x5, 0x35, 0x435, 0xa, 0x35, 0x3, 0x35, 0x5, 0x35, 0x438, - 0xa, 0x35, 0x3, 0x35, 0x5, 0x35, 0x43b, 0xa, 0x35, 0x3, 0x35, 0x5, 0x35, - 0x43e, 0xa, 0x35, 0x3, 0x35, 0x3, 0x35, 0x5, 0x35, 0x442, 0xa, 0x35, - 0x3, 0x35, 0x3, 0x35, 0x5, 0x35, 0x446, 0xa, 0x35, 0x3, 0x35, 0x5, 0x35, - 0x449, 0xa, 0x35, 0x3, 0x35, 0x5, 0x35, 0x44c, 0xa, 0x35, 0x3, 0x35, - 0x5, 0x35, 0x44f, 0xa, 0x35, 0x3, 0x35, 0x5, 0x35, 0x452, 0xa, 0x35, - 0x3, 0x35, 0x5, 0x35, 0x455, 0xa, 0x35, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, - 0x3, 0x37, 0x3, 0x37, 0x3, 0x37, 0x3, 0x37, 0x5, 0x37, 0x45e, 0xa, 0x37, - 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, 0x3, 0x39, 0x5, 0x39, 0x464, 0xa, 0x39, - 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, 0x3, 0x3a, 0x3, 0x3a, 0x3, - 0x3a, 0x3, 0x3b, 0x3, 0x3b, 0x3, 0x3b, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3c, - 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3c, 0x3, 0x3c, 0x5, 0x3c, 0x478, - 0xa, 0x3c, 0x3, 0x3d, 0x3, 0x3d, 0x3, 0x3d, 0x3, 0x3e, 0x3, 0x3e, 0x3, - 0x3e, 0x3, 0x3e, 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x3f, 0x3, 0x3f, - 0x3, 0x40, 0x3, 0x40, 0x3, 0x40, 0x3, 0x40, 0x5, 0x40, 0x48a, 0xa, 0x40, - 0x3, 0x41, 0x3, 0x41, 0x3, 0x41, 0x3, 0x42, 0x3, 0x42, 0x3, 0x42, 0x5, - 0x42, 0x492, 0xa, 0x42, 0x3, 0x42, 0x5, 0x42, 0x495, 0xa, 0x42, 0x3, - 0x42, 0x3, 0x42, 0x3, 0x42, 0x3, 0x42, 0x5, 0x42, 0x49b, 0xa, 0x42, - 0x3, 0x42, 0x3, 0x42, 0x3, 0x42, 0x3, 0x42, 0x3, 0x42, 0x3, 0x42, 0x5, - 0x42, 0x4a3, 0xa, 0x42, 0x3, 0x42, 0x5, 0x42, 0x4a6, 0xa, 0x42, 0x3, - 0x42, 0x3, 0x42, 0x3, 0x42, 0x3, 0x42, 0x7, 0x42, 0x4ac, 0xa, 0x42, - 0xc, 0x42, 0xe, 0x42, 0x4af, 0xb, 0x42, 0x3, 0x43, 0x5, 0x43, 0x4b2, - 0xa, 0x43, 0x3, 0x43, 0x3, 0x43, 0x3, 0x43, 0x5, 0x43, 0x4b7, 0xa, 0x43, - 0x3, 0x43, 0x5, 0x43, 0x4ba, 0xa, 0x43, 0x3, 0x43, 0x5, 0x43, 0x4bd, - 0xa, 0x43, 0x3, 0x43, 0x3, 0x43, 0x5, 0x43, 0x4c1, 0xa, 0x43, 0x3, 0x43, - 0x3, 0x43, 0x5, 0x43, 0x4c5, 0xa, 0x43, 0x3, 0x43, 0x5, 0x43, 0x4c8, - 0xa, 0x43, 0x5, 0x43, 0x4ca, 0xa, 0x43, 0x3, 0x43, 0x5, 0x43, 0x4cd, - 0xa, 0x43, 0x3, 0x43, 0x3, 0x43, 0x5, 0x43, 0x4d1, 0xa, 0x43, 0x3, 0x43, - 0x3, 0x43, 0x5, 0x43, 0x4d5, 0xa, 0x43, 0x3, 0x43, 0x5, 0x43, 0x4d8, - 0xa, 0x43, 0x5, 0x43, 0x4da, 0xa, 0x43, 0x5, 0x43, 0x4dc, 0xa, 0x43, - 0x3, 0x44, 0x5, 0x44, 0x4df, 0xa, 0x44, 0x3, 0x44, 0x3, 0x44, 0x3, 0x44, - 0x5, 0x44, 0x4e4, 0xa, 0x44, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, - 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x5, 0x45, 0x4ef, - 0xa, 0x45, 0x3, 0x46, 0x3, 0x46, 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, 0x4f5, - 0xa, 0x46, 0x3, 0x47, 0x3, 0x47, 0x3, 0x47, 0x5, 0x47, 0x4fa, 0xa, 0x47, - 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x7, 0x48, 0x4ff, 0xa, 0x48, 0xc, 0x48, - 0xe, 0x48, 0x502, 0xb, 0x48, 0x3, 0x49, 0x3, 0x49, 0x5, 0x49, 0x506, - 0xa, 0x49, 0x3, 0x49, 0x3, 0x49, 0x5, 0x49, 0x50a, 0xa, 0x49, 0x3, 0x49, - 0x3, 0x49, 0x5, 0x49, 0x50e, 0xa, 0x49, 0x3, 0x4a, 0x3, 0x4a, 0x3, 0x4a, - 0x5, 0x4a, 0x513, 0xa, 0x4a, 0x3, 0x4b, 0x3, 0x4b, 0x3, 0x4b, 0x7, 0x4b, - 0x518, 0xa, 0x4b, 0xc, 0x4b, 0xe, 0x4b, 0x51b, 0xb, 0x4b, 0x3, 0x4c, - 0x3, 0x4c, 0x3, 0x4c, 0x3, 0x4c, 0x3, 0x4d, 0x3, 0x4d, 0x3, 0x4d, 0x3, - 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, - 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x5, 0x4e, 0x52f, 0xa, 0x4e, - 0x3, 0x4e, 0x5, 0x4e, 0x532, 0xa, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, - 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x5, 0x4e, 0x53b, 0xa, 0x4e, - 0x3, 0x4e, 0x3, 0x4e, 0x5, 0x4e, 0x53f, 0xa, 0x4e, 0x3, 0x4e, 0x3, 0x4e, - 0x3, 0x4e, 0x5, 0x4e, 0x544, 0xa, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, - 0x5, 0x4e, 0x549, 0xa, 0x4e, 0x3, 0x4e, 0x5, 0x4e, 0x54c, 0xa, 0x4e, - 0x5, 0x4e, 0x54e, 0xa, 0x4e, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, - 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, - 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, - 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x5, 0x4f, 0x564, 0xa, 0x4f, 0x3, 0x4f, - 0x5, 0x4f, 0x567, 0xa, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, - 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x3, 0x4f, 0x5, 0x4f, 0x572, - 0xa, 0x4f, 0x3, 0x50, 0x3, 0x50, 0x5, 0x50, 0x576, 0xa, 0x50, 0x3, 0x50, - 0x5, 0x50, 0x579, 0xa, 0x50, 0x3, 0x50, 0x3, 0x50, 0x5, 0x50, 0x57d, - 0xa, 0x50, 0x3, 0x50, 0x3, 0x50, 0x5, 0x50, 0x581, 0xa, 0x50, 0x3, 0x51, - 0x3, 0x51, 0x3, 0x51, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x5, 0x52, 0x589, - 0xa, 0x52, 0x3, 0x52, 0x3, 0x52, 0x5, 0x52, 0x58d, 0xa, 0x52, 0x3, 0x53, - 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, - 0x53, 0x3, 0x53, 0x7, 0x53, 0x598, 0xa, 0x53, 0xc, 0x53, 0xe, 0x53, - 0x59b, 0xb, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, - 0x3, 0x53, 0x3, 0x53, 0x7, 0x53, 0x5a4, 0xa, 0x53, 0xc, 0x53, 0xe, 0x53, - 0x5a7, 0xb, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, - 0x3, 0x53, 0x3, 0x53, 0x7, 0x53, 0x5b0, 0xa, 0x53, 0xc, 0x53, 0xe, 0x53, - 0x5b3, 0xb, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, 0x3, 0x53, - 0x5, 0x53, 0x5ba, 0xa, 0x53, 0x3, 0x53, 0x3, 0x53, 0x5, 0x53, 0x5be, - 0xa, 0x53, 0x3, 0x54, 0x3, 0x54, 0x3, 0x54, 0x7, 0x54, 0x5c3, 0xa, 0x54, - 0xc, 0x54, 0xe, 0x54, 0x5c6, 0xb, 0x54, 0x3, 0x55, 0x3, 0x55, 0x3, 0x55, - 0x5, 0x55, 0x5cb, 0xa, 0x55, 0x3, 0x55, 0x3, 0x55, 0x3, 0x55, 0x3, 0x55, - 0x3, 0x55, 0x3, 0x55, 0x5, 0x55, 0x5d3, 0xa, 0x55, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x5, 0x56, 0x5d8, 0xa, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x6, 0x56, 0x5df, 0xa, 0x56, 0xd, 0x56, 0xe, 0x56, - 0x5e0, 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, 0x5e5, 0xa, 0x56, 0x3, 0x56, + 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, + 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, + 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, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x5, + 0x2e, 0x414, 0xa, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x3, 0x2e, 0x5, 0x2e, + 0x419, 0xa, 0x2e, 0x3, 0x2e, 0x5, 0x2e, 0x41c, 0xa, 0x2e, 0x3, 0x2e, + 0x3, 0x2e, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x3, 0x2f, 0x7, 0x2f, 0x424, + 0xa, 0x2f, 0xc, 0x2f, 0xe, 0x2f, 0x427, 0xb, 0x2f, 0x3, 0x2f, 0x3, 0x2f, + 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x3, 0x30, 0x5, 0x30, 0x430, + 0xa, 0x30, 0x3, 0x30, 0x3, 0x30, 0x5, 0x30, 0x434, 0xa, 0x30, 0x3, 0x31, + 0x3, 0x31, 0x3, 0x31, 0x5, 0x31, 0x439, 0xa, 0x31, 0x3, 0x31, 0x3, 0x31, + 0x5, 0x31, 0x43d, 0xa, 0x31, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, 0x3, 0x32, + 0x5, 0x32, 0x443, 0xa, 0x32, 0x3, 0x32, 0x5, 0x32, 0x446, 0xa, 0x32, + 0x3, 0x32, 0x5, 0x32, 0x449, 0xa, 0x32, 0x3, 0x32, 0x5, 0x32, 0x44c, + 0xa, 0x32, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, + 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x3, 0x33, 0x7, 0x33, 0x458, + 0xa, 0x33, 0xc, 0x33, 0xe, 0x33, 0x45b, 0xb, 0x33, 0x3, 0x33, 0x5, 0x33, + 0x45e, 0xa, 0x33, 0x3, 0x34, 0x3, 0x34, 0x5, 0x34, 0x462, 0xa, 0x34, + 0x3, 0x34, 0x3, 0x34, 0x3, 0x34, 0x5, 0x34, 0x467, 0xa, 0x34, 0x3, 0x34, + 0x5, 0x34, 0x46a, 0xa, 0x34, 0x3, 0x34, 0x3, 0x34, 0x3, 0x35, 0x3, 0x35, + 0x3, 0x35, 0x3, 0x35, 0x7, 0x35, 0x472, 0xa, 0x35, 0xc, 0x35, 0xe, 0x35, + 0x475, 0xb, 0x35, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, 0x3, 0x36, + 0x5, 0x36, 0x47c, 0xa, 0x36, 0x3, 0x37, 0x5, 0x37, 0x47f, 0xa, 0x37, + 0x3, 0x37, 0x3, 0x37, 0x5, 0x37, 0x483, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, + 0x486, 0xa, 0x37, 0x3, 0x37, 0x3, 0x37, 0x5, 0x37, 0x48a, 0xa, 0x37, + 0x3, 0x37, 0x5, 0x37, 0x48d, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x490, + 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, 0x493, 0xa, 0x37, 0x3, 0x37, 0x5, 0x37, + 0x496, 0xa, 0x37, 0x3, 0x37, 0x3, 0x37, 0x5, 0x37, 0x49a, 0xa, 0x37, + 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, 0x5, 0x37, 0x4ad, 0xa, 0x37, 0x3, 0x38, 0x3, 0x38, 0x3, 0x38, + 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, 0x3, 0x39, 0x5, 0x39, 0x4b6, 0xa, 0x39, + 0x3, 0x3a, 0x3, 0x3a, 0x3, 0x3a, 0x3, 0x3b, 0x5, 0x3b, 0x4bc, 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, 0x4d0, + 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, 0x4e6, 0xa, 0x43, 0x3, 0x44, 0x3, 0x44, + 0x3, 0x44, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x5, 0x45, 0x4ee, 0xa, 0x45, + 0x3, 0x45, 0x5, 0x45, 0x4f1, 0xa, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, + 0x3, 0x45, 0x5, 0x45, 0x4f7, 0xa, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, + 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x5, 0x45, 0x4ff, 0xa, 0x45, 0x3, 0x45, + 0x5, 0x45, 0x502, 0xa, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, 0x3, 0x45, + 0x7, 0x45, 0x508, 0xa, 0x45, 0xc, 0x45, 0xe, 0x45, 0x50b, 0xb, 0x45, + 0x3, 0x46, 0x5, 0x46, 0x50e, 0xa, 0x46, 0x3, 0x46, 0x3, 0x46, 0x3, 0x46, + 0x5, 0x46, 0x513, 0xa, 0x46, 0x3, 0x46, 0x5, 0x46, 0x516, 0xa, 0x46, + 0x3, 0x46, 0x5, 0x46, 0x519, 0xa, 0x46, 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, + 0x51d, 0xa, 0x46, 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, 0x521, 0xa, 0x46, + 0x3, 0x46, 0x5, 0x46, 0x524, 0xa, 0x46, 0x5, 0x46, 0x526, 0xa, 0x46, + 0x3, 0x46, 0x5, 0x46, 0x529, 0xa, 0x46, 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, + 0x52d, 0xa, 0x46, 0x3, 0x46, 0x3, 0x46, 0x5, 0x46, 0x531, 0xa, 0x46, + 0x3, 0x46, 0x5, 0x46, 0x534, 0xa, 0x46, 0x5, 0x46, 0x536, 0xa, 0x46, + 0x5, 0x46, 0x538, 0xa, 0x46, 0x3, 0x47, 0x5, 0x47, 0x53b, 0xa, 0x47, + 0x3, 0x47, 0x3, 0x47, 0x3, 0x47, 0x5, 0x47, 0x540, 0xa, 0x47, 0x3, 0x48, + 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, 0x48, 0x3, + 0x48, 0x3, 0x48, 0x5, 0x48, 0x54b, 0xa, 0x48, 0x3, 0x49, 0x3, 0x49, + 0x3, 0x49, 0x3, 0x49, 0x5, 0x49, 0x551, 0xa, 0x49, 0x3, 0x4a, 0x3, 0x4a, + 0x3, 0x4a, 0x5, 0x4a, 0x556, 0xa, 0x4a, 0x3, 0x4b, 0x3, 0x4b, 0x3, 0x4b, + 0x7, 0x4b, 0x55b, 0xa, 0x4b, 0xc, 0x4b, 0xe, 0x4b, 0x55e, 0xb, 0x4b, + 0x3, 0x4c, 0x3, 0x4c, 0x5, 0x4c, 0x562, 0xa, 0x4c, 0x3, 0x4c, 0x3, 0x4c, + 0x5, 0x4c, 0x566, 0xa, 0x4c, 0x3, 0x4c, 0x3, 0x4c, 0x5, 0x4c, 0x56a, + 0xa, 0x4c, 0x3, 0x4d, 0x3, 0x4d, 0x3, 0x4d, 0x5, 0x4d, 0x56f, 0xa, 0x4d, + 0x3, 0x4e, 0x3, 0x4e, 0x3, 0x4e, 0x7, 0x4e, 0x574, 0xa, 0x4e, 0xc, 0x4e, + 0xe, 0x4e, 0x577, 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, 0x58b, 0xa, 0x51, 0x3, 0x51, 0x5, 0x51, 0x58e, + 0xa, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, + 0x51, 0x3, 0x51, 0x5, 0x51, 0x597, 0xa, 0x51, 0x3, 0x51, 0x3, 0x51, + 0x5, 0x51, 0x59b, 0xa, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x5, 0x51, + 0x5a0, 0xa, 0x51, 0x3, 0x51, 0x3, 0x51, 0x3, 0x51, 0x5, 0x51, 0x5a5, + 0xa, 0x51, 0x3, 0x51, 0x5, 0x51, 0x5a8, 0xa, 0x51, 0x5, 0x51, 0x5aa, + 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, 0x5c0, 0xa, 0x52, 0x3, 0x52, 0x5, 0x52, + 0x5c3, 0xa, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, + 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x3, 0x52, 0x5, 0x52, 0x5ce, 0xa, 0x52, + 0x3, 0x53, 0x3, 0x53, 0x5, 0x53, 0x5d2, 0xa, 0x53, 0x3, 0x53, 0x5, 0x53, + 0x5d5, 0xa, 0x53, 0x3, 0x53, 0x3, 0x53, 0x5, 0x53, 0x5d9, 0xa, 0x53, + 0x3, 0x53, 0x3, 0x53, 0x5, 0x53, 0x5dd, 0xa, 0x53, 0x3, 0x54, 0x3, 0x54, + 0x3, 0x54, 0x3, 0x55, 0x3, 0x55, 0x3, 0x55, 0x5, 0x55, 0x5e5, 0xa, 0x55, + 0x3, 0x55, 0x3, 0x55, 0x5, 0x55, 0x5e9, 0xa, 0x55, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, + 0x56, 0x7, 0x56, 0x5f4, 0xa, 0x56, 0xc, 0x56, 0xe, 0x56, 0x5f7, 0xb, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, - 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, 0x604, 0xa, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, - 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x5, 0x56, 0x615, 0xa, 0x56, 0x3, 0x56, 0x5, 0x56, 0x618, 0xa, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, 0x61c, 0xa, 0x56, 0x3, 0x56, 0x5, 0x56, - 0x61f, 0xa, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, 0x62b, - 0xa, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, - 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, 0x63c, 0xa, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x5, 0x56, 0x640, 0xa, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, - 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, - 0x651, 0xa, 0x56, 0x3, 0x56, 0x5, 0x56, 0x654, 0xa, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x5, 0x56, 0x658, 0xa, 0x56, 0x3, 0x56, 0x5, 0x56, 0x65b, - 0xa, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, - 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, 0x666, 0xa, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, - 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, - 0x56, 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, 0x67e, 0xa, 0x56, 0x3, 0x56, - 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, 0x685, 0xa, 0x56, - 0x7, 0x56, 0x687, 0xa, 0x56, 0xc, 0x56, 0xe, 0x56, 0x68a, 0xb, 0x56, - 0x3, 0x57, 0x3, 0x57, 0x3, 0x57, 0x7, 0x57, 0x68f, 0xa, 0x57, 0xc, 0x57, - 0xe, 0x57, 0x692, 0xb, 0x57, 0x3, 0x58, 0x3, 0x58, 0x5, 0x58, 0x696, - 0xa, 0x58, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x7, 0x59, 0x69c, - 0xa, 0x59, 0xc, 0x59, 0xe, 0x59, 0x69f, 0xb, 0x59, 0x3, 0x59, 0x3, 0x59, - 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x7, 0x59, 0x6a6, 0xa, 0x59, 0xc, 0x59, - 0xe, 0x59, 0x6a9, 0xb, 0x59, 0x5, 0x59, 0x6ab, 0xa, 0x59, 0x3, 0x59, - 0x3, 0x59, 0x3, 0x59, 0x3, 0x5a, 0x3, 0x5a, 0x3, 0x5a, 0x5, 0x5a, 0x6b3, - 0xa, 0x5a, 0x3, 0x5a, 0x3, 0x5a, 0x3, 0x5b, 0x3, 0x5b, 0x3, 0x5b, 0x5, - 0x5b, 0x6ba, 0xa, 0x5b, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, - 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x5, 0x5c, 0x6c3, 0xa, 0x5c, 0x3, 0x5c, - 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x5, 0x5c, 0x6c9, 0xa, 0x5c, 0x7, 0x5c, - 0x6cb, 0xa, 0x5c, 0xc, 0x5c, 0xe, 0x5c, 0x6ce, 0xb, 0x5c, 0x3, 0x5d, - 0x3, 0x5d, 0x3, 0x5d, 0x5, 0x5d, 0x6d3, 0xa, 0x5d, 0x3, 0x5d, 0x3, 0x5d, - 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x5, 0x5e, 0x6da, 0xa, 0x5e, 0x3, 0x5e, - 0x3, 0x5e, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x7, 0x5f, 0x6e1, 0xa, 0x5f, - 0xc, 0x5f, 0xe, 0x5f, 0x6e4, 0xb, 0x5f, 0x3, 0x60, 0x3, 0x60, 0x3, 0x60, - 0x5, 0x60, 0x6e9, 0xa, 0x60, 0x3, 0x61, 0x3, 0x61, 0x3, 0x62, 0x3, 0x62, - 0x3, 0x62, 0x3, 0x62, 0x3, 0x62, 0x3, 0x62, 0x5, 0x62, 0x6f3, 0xa, 0x62, - 0x5, 0x62, 0x6f5, 0xa, 0x62, 0x3, 0x63, 0x5, 0x63, 0x6f8, 0xa, 0x63, - 0x3, 0x63, 0x3, 0x63, 0x3, 0x63, 0x3, 0x63, 0x3, 0x63, 0x3, 0x63, 0x5, - 0x63, 0x700, 0xa, 0x63, 0x3, 0x64, 0x3, 0x64, 0x3, 0x64, 0x5, 0x64, - 0x705, 0xa, 0x64, 0x3, 0x65, 0x3, 0x65, 0x3, 0x66, 0x3, 0x66, 0x3, 0x67, - 0x3, 0x67, 0x3, 0x68, 0x3, 0x68, 0x5, 0x68, 0x70f, 0xa, 0x68, 0x3, 0x69, - 0x3, 0x69, 0x3, 0x69, 0x5, 0x69, 0x714, 0xa, 0x69, 0x3, 0x6a, 0x3, 0x6a, - 0x5, 0x6a, 0x718, 0xa, 0x6a, 0x3, 0x6b, 0x3, 0x6b, 0x3, 0x6b, 0x3, 0x6b, - 0x3, 0x6b, 0x2, 0x5, 0x82, 0xaa, 0xb6, 0x6c, 0x2, 0x4, 0x6, 0x8, 0xa, + 0x3, 0x56, 0x7, 0x56, 0x600, 0xa, 0x56, 0xc, 0x56, 0xe, 0x56, 0x603, + 0xb, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, + 0x56, 0x3, 0x56, 0x7, 0x56, 0x60c, 0xa, 0x56, 0xc, 0x56, 0xe, 0x56, + 0x60f, 0xb, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, 0x3, 0x56, + 0x5, 0x56, 0x616, 0xa, 0x56, 0x3, 0x56, 0x3, 0x56, 0x5, 0x56, 0x61a, + 0xa, 0x56, 0x3, 0x57, 0x3, 0x57, 0x3, 0x57, 0x7, 0x57, 0x61f, 0xa, 0x57, + 0xc, 0x57, 0xe, 0x57, 0x622, 0xb, 0x57, 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, + 0x5, 0x58, 0x627, 0xa, 0x58, 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, 0x3, 0x58, + 0x3, 0x58, 0x3, 0x58, 0x5, 0x58, 0x62f, 0xa, 0x58, 0x3, 0x59, 0x3, 0x59, + 0x3, 0x59, 0x5, 0x59, 0x634, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, + 0x3, 0x59, 0x3, 0x59, 0x6, 0x59, 0x63b, 0xa, 0x59, 0xd, 0x59, 0xe, 0x59, + 0x63c, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x641, 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, 0x660, 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, 0x671, 0xa, 0x59, 0x3, 0x59, 0x5, 0x59, 0x674, 0xa, 0x59, + 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x678, 0xa, 0x59, 0x3, 0x59, 0x5, 0x59, + 0x67b, 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, + 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x687, + 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, 0x698, 0xa, 0x59, 0x3, 0x59, + 0x3, 0x59, 0x5, 0x59, 0x69c, 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, + 0x6ad, 0xa, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6b0, 0xa, 0x59, 0x3, 0x59, + 0x3, 0x59, 0x5, 0x59, 0x6b4, 0xa, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6b7, + 0xa, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, + 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6c2, 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, 0x6da, 0xa, 0x59, 0x3, 0x59, + 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x3, 0x59, 0x5, 0x59, 0x6e1, 0xa, 0x59, + 0x7, 0x59, 0x6e3, 0xa, 0x59, 0xc, 0x59, 0xe, 0x59, 0x6e6, 0xb, 0x59, + 0x3, 0x5a, 0x3, 0x5a, 0x3, 0x5a, 0x7, 0x5a, 0x6eb, 0xa, 0x5a, 0xc, 0x5a, + 0xe, 0x5a, 0x6ee, 0xb, 0x5a, 0x3, 0x5b, 0x3, 0x5b, 0x5, 0x5b, 0x6f2, + 0xa, 0x5b, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x7, 0x5c, 0x6f8, + 0xa, 0x5c, 0xc, 0x5c, 0xe, 0x5c, 0x6fb, 0xb, 0x5c, 0x3, 0x5c, 0x3, 0x5c, + 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5c, 0x7, 0x5c, 0x702, 0xa, 0x5c, 0xc, 0x5c, + 0xe, 0x5c, 0x705, 0xb, 0x5c, 0x5, 0x5c, 0x707, 0xa, 0x5c, 0x3, 0x5c, + 0x3, 0x5c, 0x3, 0x5c, 0x3, 0x5d, 0x3, 0x5d, 0x3, 0x5d, 0x5, 0x5d, 0x70f, + 0xa, 0x5d, 0x3, 0x5d, 0x3, 0x5d, 0x3, 0x5e, 0x3, 0x5e, 0x3, 0x5e, 0x5, + 0x5e, 0x716, 0xa, 0x5e, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, + 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x5, 0x5f, 0x71f, 0xa, 0x5f, 0x3, 0x5f, + 0x3, 0x5f, 0x3, 0x5f, 0x3, 0x5f, 0x5, 0x5f, 0x725, 0xa, 0x5f, 0x7, 0x5f, + 0x727, 0xa, 0x5f, 0xc, 0x5f, 0xe, 0x5f, 0x72a, 0xb, 0x5f, 0x3, 0x60, + 0x3, 0x60, 0x3, 0x60, 0x5, 0x60, 0x72f, 0xa, 0x60, 0x3, 0x60, 0x3, 0x60, + 0x3, 0x61, 0x3, 0x61, 0x3, 0x61, 0x5, 0x61, 0x736, 0xa, 0x61, 0x3, 0x61, + 0x3, 0x61, 0x3, 0x62, 0x3, 0x62, 0x3, 0x62, 0x7, 0x62, 0x73d, 0xa, 0x62, + 0xc, 0x62, 0xe, 0x62, 0x740, 0xb, 0x62, 0x3, 0x63, 0x3, 0x63, 0x3, 0x63, + 0x5, 0x63, 0x745, 0xa, 0x63, 0x3, 0x64, 0x3, 0x64, 0x3, 0x65, 0x3, 0x65, + 0x3, 0x65, 0x3, 0x65, 0x3, 0x65, 0x3, 0x65, 0x5, 0x65, 0x74f, 0xa, 0x65, + 0x5, 0x65, 0x751, 0xa, 0x65, 0x3, 0x66, 0x5, 0x66, 0x754, 0xa, 0x66, + 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, 0x3, 0x66, 0x5, + 0x66, 0x75c, 0xa, 0x66, 0x3, 0x67, 0x3, 0x67, 0x3, 0x67, 0x5, 0x67, + 0x761, 0xa, 0x67, 0x3, 0x68, 0x3, 0x68, 0x3, 0x69, 0x3, 0x69, 0x3, 0x6a, + 0x3, 0x6a, 0x3, 0x6b, 0x3, 0x6b, 0x5, 0x6b, 0x76b, 0xa, 0x6b, 0x3, 0x6c, + 0x3, 0x6c, 0x3, 0x6c, 0x5, 0x6c, 0x770, 0xa, 0x6c, 0x3, 0x6d, 0x3, 0x6d, + 0x5, 0x6d, 0x774, 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, 0x2a, 0x2c, 0x2e, 0x30, 0x32, 0x34, 0x36, 0x38, 0x3a, 0x3c, 0x3e, 0x40, 0x42, 0x44, 0x46, 0x48, 0x4a, 0x4c, 0x4e, 0x50, 0x52, @@ -18494,1076 +19200,1132 @@ ClickHouseParser::Initializer::Initializer() { 0x84, 0x86, 0x88, 0x8a, 0x8c, 0x8e, 0x90, 0x92, 0x94, 0x96, 0x98, 0x9a, 0x9c, 0x9e, 0xa0, 0xa2, 0xa4, 0xa6, 0xa8, 0xaa, 0xac, 0xae, 0xb0, 0xb2, 0xb4, 0xb6, 0xb8, 0xba, 0xbc, 0xbe, 0xc0, 0xc2, 0xc4, 0xc6, 0xc8, 0xca, - 0xcc, 0xce, 0xd0, 0xd2, 0xd4, 0x2, 0x1d, 0x8, 0x2, 0x5, 0x5, 0x19, 0x19, - 0x1c, 0x1c, 0x26, 0x26, 0x65, 0x65, 0xa5, 0xa5, 0x4, 0x2, 0x10, 0x10, - 0x1e, 0x1e, 0x5, 0x2, 0x5, 0x5, 0x26, 0x26, 0x65, 0x65, 0x4, 0x2, 0x29, - 0x29, 0x2b, 0x2b, 0x4, 0x2, 0x2c, 0x2c, 0x32, 0x32, 0x5, 0x2, 0xf, 0xf, - 0x94, 0x94, 0x9a, 0x9a, 0x4, 0x2, 0x20, 0x20, 0x87, 0x87, 0x4, 0x2, - 0x52, 0x52, 0x5e, 0x5e, 0x4, 0x2, 0x45, 0x45, 0x63, 0x63, 0x5, 0x2, - 0x6, 0x6, 0xa, 0xa, 0xe, 0xe, 0x6, 0x2, 0x6, 0x6, 0x9, 0xa, 0xe, 0xe, - 0x8b, 0x8b, 0x4, 0x2, 0x5e, 0x5e, 0x86, 0x86, 0x4, 0x2, 0x6, 0x6, 0xa, - 0xa, 0x4, 0x2, 0x73, 0x73, 0xc2, 0xc2, 0x4, 0x2, 0xd, 0xd, 0x29, 0x2a, - 0x4, 0x2, 0x3d, 0x3d, 0x5b, 0x5b, 0x4, 0x2, 0x42, 0x42, 0x4e, 0x4e, - 0x3, 0x2, 0x91, 0x92, 0x5, 0x2, 0x12, 0x12, 0x5d, 0x5d, 0xa2, 0xa2, - 0x5, 0x2, 0xbe, 0xbe, 0xd0, 0xd0, 0xd9, 0xd9, 0x4, 0x2, 0xc3, 0xc4, - 0xd1, 0xd1, 0x4, 0x2, 0x4d, 0x4d, 0x60, 0x60, 0x3, 0x2, 0xb9, 0xba, - 0x4, 0x2, 0xc4, 0xc4, 0xd1, 0xd1, 0xa, 0x2, 0x24, 0x24, 0x4a, 0x4a, - 0x69, 0x69, 0x6b, 0x6b, 0x7e, 0x7e, 0x89, 0x89, 0xb0, 0xb0, 0xb4, 0xb4, - 0xe, 0x2, 0x4, 0x23, 0x25, 0x49, 0x4b, 0x4f, 0x51, 0x68, 0x6a, 0x6a, - 0x6c, 0x6d, 0x6f, 0x70, 0x72, 0x7d, 0x7f, 0x88, 0x8a, 0xaf, 0xb1, 0xb3, - 0xb5, 0xb6, 0x6, 0x2, 0x23, 0x23, 0x3d, 0x3d, 0x4b, 0x4b, 0x59, 0x59, - 0x2, 0x81c, 0x2, 0xe4, 0x3, 0x2, 0x2, 0x2, 0x4, 0xf8, 0x3, 0x2, 0x2, - 0x2, 0x6, 0xfa, 0x3, 0x2, 0x2, 0x2, 0x8, 0x19f, 0x3, 0x2, 0x2, 0x2, - 0xa, 0x1a1, 0x3, 0x2, 0x2, 0x2, 0xc, 0x1a9, 0x3, 0x2, 0x2, 0x2, 0xe, - 0x1ad, 0x3, 0x2, 0x2, 0x2, 0x10, 0x1b4, 0x3, 0x2, 0x2, 0x2, 0x12, 0x1b6, - 0x3, 0x2, 0x2, 0x2, 0x14, 0x1bc, 0x3, 0x2, 0x2, 0x2, 0x16, 0x24b, 0x3, - 0x2, 0x2, 0x2, 0x18, 0x24d, 0x3, 0x2, 0x2, 0x2, 0x1a, 0x258, 0x3, 0x2, - 0x2, 0x2, 0x1c, 0x273, 0x3, 0x2, 0x2, 0x2, 0x1e, 0x28e, 0x3, 0x2, 0x2, - 0x2, 0x20, 0x292, 0x3, 0x2, 0x2, 0x2, 0x22, 0x29b, 0x3, 0x2, 0x2, 0x2, - 0x24, 0x2a8, 0x3, 0x2, 0x2, 0x2, 0x26, 0x2b7, 0x3, 0x2, 0x2, 0x2, 0x28, - 0x2c4, 0x3, 0x2, 0x2, 0x2, 0x2a, 0x2d4, 0x3, 0x2, 0x2, 0x2, 0x2c, 0x2d9, - 0x3, 0x2, 0x2, 0x2, 0x2e, 0x2df, 0x3, 0x2, 0x2, 0x2, 0x30, 0x2e2, 0x3, - 0x2, 0x2, 0x2, 0x32, 0x2e5, 0x3, 0x2, 0x2, 0x2, 0x34, 0x2f7, 0x3, 0x2, - 0x2, 0x2, 0x36, 0x2f9, 0x3, 0x2, 0x2, 0x2, 0x38, 0x317, 0x3, 0x2, 0x2, - 0x2, 0x3a, 0x31b, 0x3, 0x2, 0x2, 0x2, 0x3c, 0x31f, 0x3, 0x2, 0x2, 0x2, - 0x3e, 0x323, 0x3, 0x2, 0x2, 0x2, 0x40, 0x32c, 0x3, 0x2, 0x2, 0x2, 0x42, - 0x340, 0x3, 0x2, 0x2, 0x2, 0x44, 0x362, 0x3, 0x2, 0x2, 0x2, 0x46, 0x364, - 0x3, 0x2, 0x2, 0x2, 0x48, 0x367, 0x3, 0x2, 0x2, 0x2, 0x4a, 0x36e, 0x3, - 0x2, 0x2, 0x2, 0x4c, 0x37a, 0x3, 0x2, 0x2, 0x2, 0x4e, 0x382, 0x3, 0x2, - 0x2, 0x2, 0x50, 0x38c, 0x3, 0x2, 0x2, 0x2, 0x52, 0x3b1, 0x3, 0x2, 0x2, - 0x2, 0x54, 0x3c0, 0x3, 0x2, 0x2, 0x2, 0x56, 0x3c2, 0x3, 0x2, 0x2, 0x2, - 0x58, 0x3c6, 0x3, 0x2, 0x2, 0x2, 0x5a, 0x3d5, 0x3, 0x2, 0x2, 0x2, 0x5c, - 0x3e9, 0x3, 0x2, 0x2, 0x2, 0x5e, 0x3eb, 0x3, 0x2, 0x2, 0x2, 0x60, 0x3f4, - 0x3, 0x2, 0x2, 0x2, 0x62, 0x403, 0x3, 0x2, 0x2, 0x2, 0x64, 0x415, 0x3, - 0x2, 0x2, 0x2, 0x66, 0x423, 0x3, 0x2, 0x2, 0x2, 0x68, 0x426, 0x3, 0x2, - 0x2, 0x2, 0x6a, 0x456, 0x3, 0x2, 0x2, 0x2, 0x6c, 0x459, 0x3, 0x2, 0x2, - 0x2, 0x6e, 0x45f, 0x3, 0x2, 0x2, 0x2, 0x70, 0x463, 0x3, 0x2, 0x2, 0x2, - 0x72, 0x469, 0x3, 0x2, 0x2, 0x2, 0x74, 0x46c, 0x3, 0x2, 0x2, 0x2, 0x76, - 0x46f, 0x3, 0x2, 0x2, 0x2, 0x78, 0x479, 0x3, 0x2, 0x2, 0x2, 0x7a, 0x47c, - 0x3, 0x2, 0x2, 0x2, 0x7c, 0x480, 0x3, 0x2, 0x2, 0x2, 0x7e, 0x485, 0x3, - 0x2, 0x2, 0x2, 0x80, 0x48b, 0x3, 0x2, 0x2, 0x2, 0x82, 0x49a, 0x3, 0x2, - 0x2, 0x2, 0x84, 0x4db, 0x3, 0x2, 0x2, 0x2, 0x86, 0x4e3, 0x3, 0x2, 0x2, - 0x2, 0x88, 0x4ee, 0x3, 0x2, 0x2, 0x2, 0x8a, 0x4f0, 0x3, 0x2, 0x2, 0x2, - 0x8c, 0x4f6, 0x3, 0x2, 0x2, 0x2, 0x8e, 0x4fb, 0x3, 0x2, 0x2, 0x2, 0x90, - 0x503, 0x3, 0x2, 0x2, 0x2, 0x92, 0x50f, 0x3, 0x2, 0x2, 0x2, 0x94, 0x514, - 0x3, 0x2, 0x2, 0x2, 0x96, 0x51c, 0x3, 0x2, 0x2, 0x2, 0x98, 0x520, 0x3, - 0x2, 0x2, 0x2, 0x9a, 0x54d, 0x3, 0x2, 0x2, 0x2, 0x9c, 0x571, 0x3, 0x2, - 0x2, 0x2, 0x9e, 0x573, 0x3, 0x2, 0x2, 0x2, 0xa0, 0x582, 0x3, 0x2, 0x2, - 0x2, 0xa2, 0x585, 0x3, 0x2, 0x2, 0x2, 0xa4, 0x5bd, 0x3, 0x2, 0x2, 0x2, - 0xa6, 0x5bf, 0x3, 0x2, 0x2, 0x2, 0xa8, 0x5d2, 0x3, 0x2, 0x2, 0x2, 0xaa, - 0x63f, 0x3, 0x2, 0x2, 0x2, 0xac, 0x68b, 0x3, 0x2, 0x2, 0x2, 0xae, 0x695, - 0x3, 0x2, 0x2, 0x2, 0xb0, 0x6aa, 0x3, 0x2, 0x2, 0x2, 0xb2, 0x6b2, 0x3, - 0x2, 0x2, 0x2, 0xb4, 0x6b6, 0x3, 0x2, 0x2, 0x2, 0xb6, 0x6c2, 0x3, 0x2, - 0x2, 0x2, 0xb8, 0x6cf, 0x3, 0x2, 0x2, 0x2, 0xba, 0x6d9, 0x3, 0x2, 0x2, - 0x2, 0xbc, 0x6dd, 0x3, 0x2, 0x2, 0x2, 0xbe, 0x6e8, 0x3, 0x2, 0x2, 0x2, - 0xc0, 0x6ea, 0x3, 0x2, 0x2, 0x2, 0xc2, 0x6f4, 0x3, 0x2, 0x2, 0x2, 0xc4, - 0x6f7, 0x3, 0x2, 0x2, 0x2, 0xc6, 0x704, 0x3, 0x2, 0x2, 0x2, 0xc8, 0x706, - 0x3, 0x2, 0x2, 0x2, 0xca, 0x708, 0x3, 0x2, 0x2, 0x2, 0xcc, 0x70a, 0x3, - 0x2, 0x2, 0x2, 0xce, 0x70e, 0x3, 0x2, 0x2, 0x2, 0xd0, 0x713, 0x3, 0x2, - 0x2, 0x2, 0xd2, 0x717, 0x3, 0x2, 0x2, 0x2, 0xd4, 0x719, 0x3, 0x2, 0x2, - 0x2, 0xd6, 0xda, 0x5, 0x4, 0x3, 0x2, 0xd7, 0xd8, 0x7, 0x55, 0x2, 0x2, - 0xd8, 0xd9, 0x7, 0x79, 0x2, 0x2, 0xd9, 0xdb, 0x7, 0xbc, 0x2, 0x2, 0xda, - 0xd7, 0x3, 0x2, 0x2, 0x2, 0xda, 0xdb, 0x3, 0x2, 0x2, 0x2, 0xdb, 0xde, - 0x3, 0x2, 0x2, 0x2, 0xdc, 0xdd, 0x7, 0x40, 0x2, 0x2, 0xdd, 0xdf, 0x5, - 0xd2, 0x6a, 0x2, 0xde, 0xdc, 0x3, 0x2, 0x2, 0x2, 0xde, 0xdf, 0x3, 0x2, - 0x2, 0x2, 0xdf, 0xe1, 0x3, 0x2, 0x2, 0x2, 0xe0, 0xe2, 0x7, 0xd8, 0x2, - 0x2, 0xe1, 0xe0, 0x3, 0x2, 0x2, 0x2, 0xe1, 0xe2, 0x3, 0x2, 0x2, 0x2, - 0xe2, 0xe5, 0x3, 0x2, 0x2, 0x2, 0xe3, 0xe5, 0x5, 0x58, 0x2d, 0x2, 0xe4, - 0xd6, 0x3, 0x2, 0x2, 0x2, 0xe4, 0xe3, 0x3, 0x2, 0x2, 0x2, 0xe5, 0x3, - 0x3, 0x2, 0x2, 0x2, 0xe6, 0xf9, 0x5, 0x6, 0x4, 0x2, 0xe7, 0xf9, 0x5, - 0x12, 0xa, 0x2, 0xe8, 0xf9, 0x5, 0x14, 0xb, 0x2, 0xe9, 0xf9, 0x5, 0x16, - 0xc, 0x2, 0xea, 0xf9, 0x5, 0x50, 0x29, 0x2, 0xeb, 0xf9, 0x5, 0x52, 0x2a, - 0x2, 0xec, 0xf9, 0x5, 0x54, 0x2b, 0x2, 0xed, 0xf9, 0x5, 0x56, 0x2c, - 0x2, 0xee, 0xf9, 0x5, 0x5e, 0x30, 0x2, 0xef, 0xf9, 0x5, 0x60, 0x31, - 0x2, 0xf0, 0xf9, 0x5, 0x62, 0x32, 0x2, 0xf1, 0xf9, 0x5, 0x64, 0x33, - 0x2, 0xf2, 0xf9, 0x5, 0x98, 0x4d, 0x2, 0xf3, 0xf9, 0x5, 0x9a, 0x4e, - 0x2, 0xf4, 0xf9, 0x5, 0x9c, 0x4f, 0x2, 0xf5, 0xf9, 0x5, 0x9e, 0x50, - 0x2, 0xf6, 0xf9, 0x5, 0xa0, 0x51, 0x2, 0xf7, 0xf9, 0x5, 0xa2, 0x52, - 0x2, 0xf8, 0xe6, 0x3, 0x2, 0x2, 0x2, 0xf8, 0xe7, 0x3, 0x2, 0x2, 0x2, - 0xf8, 0xe8, 0x3, 0x2, 0x2, 0x2, 0xf8, 0xe9, 0x3, 0x2, 0x2, 0x2, 0xf8, - 0xea, 0x3, 0x2, 0x2, 0x2, 0xf8, 0xeb, 0x3, 0x2, 0x2, 0x2, 0xf8, 0xec, - 0x3, 0x2, 0x2, 0x2, 0xf8, 0xed, 0x3, 0x2, 0x2, 0x2, 0xf8, 0xee, 0x3, - 0x2, 0x2, 0x2, 0xf8, 0xef, 0x3, 0x2, 0x2, 0x2, 0xf8, 0xf0, 0x3, 0x2, - 0x2, 0x2, 0xf8, 0xf1, 0x3, 0x2, 0x2, 0x2, 0xf8, 0xf2, 0x3, 0x2, 0x2, - 0x2, 0xf8, 0xf3, 0x3, 0x2, 0x2, 0x2, 0xf8, 0xf4, 0x3, 0x2, 0x2, 0x2, - 0xf8, 0xf5, 0x3, 0x2, 0x2, 0x2, 0xf8, 0xf6, 0x3, 0x2, 0x2, 0x2, 0xf8, - 0xf7, 0x3, 0x2, 0x2, 0x2, 0xf9, 0x5, 0x3, 0x2, 0x2, 0x2, 0xfa, 0xfb, - 0x7, 0x7, 0x2, 0x2, 0xfb, 0xfc, 0x7, 0x97, 0x2, 0x2, 0xfc, 0xfe, 0x5, - 0xba, 0x5e, 0x2, 0xfd, 0xff, 0x5, 0x2c, 0x17, 0x2, 0xfe, 0xfd, 0x3, - 0x2, 0x2, 0x2, 0xfe, 0xff, 0x3, 0x2, 0x2, 0x2, 0xff, 0x100, 0x3, 0x2, - 0x2, 0x2, 0x100, 0x105, 0x5, 0x8, 0x5, 0x2, 0x101, 0x102, 0x7, 0xc2, - 0x2, 0x2, 0x102, 0x104, 0x5, 0x8, 0x5, 0x2, 0x103, 0x101, 0x3, 0x2, - 0x2, 0x2, 0x104, 0x107, 0x3, 0x2, 0x2, 0x2, 0x105, 0x103, 0x3, 0x2, - 0x2, 0x2, 0x105, 0x106, 0x3, 0x2, 0x2, 0x2, 0x106, 0x7, 0x3, 0x2, 0x2, - 0x2, 0x107, 0x105, 0x3, 0x2, 0x2, 0x2, 0x108, 0x109, 0x7, 0x3, 0x2, - 0x2, 0x109, 0x10d, 0x7, 0x1b, 0x2, 0x2, 0x10a, 0x10b, 0x7, 0x4c, 0x2, - 0x2, 0x10b, 0x10c, 0x7, 0x70, 0x2, 0x2, 0x10c, 0x10e, 0x7, 0x37, 0x2, - 0x2, 0x10d, 0x10a, 0x3, 0x2, 0x2, 0x2, 0x10d, 0x10e, 0x3, 0x2, 0x2, - 0x2, 0x10e, 0x10f, 0x3, 0x2, 0x2, 0x2, 0x10f, 0x112, 0x5, 0x44, 0x23, - 0x2, 0x110, 0x111, 0x7, 0x4, 0x2, 0x2, 0x111, 0x113, 0x5, 0xb4, 0x5b, - 0x2, 0x112, 0x110, 0x3, 0x2, 0x2, 0x2, 0x112, 0x113, 0x3, 0x2, 0x2, - 0x2, 0x113, 0x1a0, 0x3, 0x2, 0x2, 0x2, 0x114, 0x115, 0x7, 0x3, 0x2, - 0x2, 0x115, 0x119, 0x7, 0x4f, 0x2, 0x2, 0x116, 0x117, 0x7, 0x4c, 0x2, - 0x2, 0x117, 0x118, 0x7, 0x70, 0x2, 0x2, 0x118, 0x11a, 0x7, 0x37, 0x2, - 0x2, 0x119, 0x116, 0x3, 0x2, 0x2, 0x2, 0x119, 0x11a, 0x3, 0x2, 0x2, - 0x2, 0x11a, 0x11b, 0x3, 0x2, 0x2, 0x2, 0x11b, 0x11e, 0x5, 0x48, 0x25, - 0x2, 0x11c, 0x11d, 0x7, 0x4, 0x2, 0x2, 0x11d, 0x11f, 0x5, 0xb4, 0x5b, - 0x2, 0x11e, 0x11c, 0x3, 0x2, 0x2, 0x2, 0x11e, 0x11f, 0x3, 0x2, 0x2, - 0x2, 0x11f, 0x1a0, 0x3, 0x2, 0x2, 0x2, 0x120, 0x121, 0x7, 0x10, 0x2, - 0x2, 0x121, 0x124, 0x5, 0x10, 0x9, 0x2, 0x122, 0x123, 0x7, 0x42, 0x2, - 0x2, 0x123, 0x125, 0x5, 0xba, 0x5e, 0x2, 0x124, 0x122, 0x3, 0x2, 0x2, - 0x2, 0x124, 0x125, 0x3, 0x2, 0x2, 0x2, 0x125, 0x1a0, 0x3, 0x2, 0x2, - 0x2, 0x126, 0x127, 0x7, 0x17, 0x2, 0x2, 0x127, 0x12a, 0x7, 0x1b, 0x2, - 0x2, 0x128, 0x129, 0x7, 0x4c, 0x2, 0x2, 0x129, 0x12b, 0x7, 0x37, 0x2, - 0x2, 0x12a, 0x128, 0x3, 0x2, 0x2, 0x2, 0x12a, 0x12b, 0x3, 0x2, 0x2, - 0x2, 0x12b, 0x12c, 0x3, 0x2, 0x2, 0x2, 0x12c, 0x12f, 0x5, 0xb4, 0x5b, - 0x2, 0x12d, 0x12e, 0x7, 0x4e, 0x2, 0x2, 0x12e, 0x130, 0x5, 0x10, 0x9, - 0x2, 0x12f, 0x12d, 0x3, 0x2, 0x2, 0x2, 0x12f, 0x130, 0x3, 0x2, 0x2, - 0x2, 0x130, 0x1a0, 0x3, 0x2, 0x2, 0x2, 0x131, 0x132, 0x7, 0x1c, 0x2, - 0x2, 0x132, 0x135, 0x7, 0x1b, 0x2, 0x2, 0x133, 0x134, 0x7, 0x4c, 0x2, - 0x2, 0x134, 0x136, 0x7, 0x37, 0x2, 0x2, 0x135, 0x133, 0x3, 0x2, 0x2, - 0x2, 0x135, 0x136, 0x3, 0x2, 0x2, 0x2, 0x136, 0x137, 0x3, 0x2, 0x2, - 0x2, 0x137, 0x138, 0x5, 0xb4, 0x5b, 0x2, 0x138, 0x139, 0x7, 0xbc, 0x2, - 0x2, 0x139, 0x1a0, 0x3, 0x2, 0x2, 0x2, 0x13a, 0x13b, 0x7, 0x28, 0x2, - 0x2, 0x13b, 0x13c, 0x7, 0xb2, 0x2, 0x2, 0x13c, 0x1a0, 0x5, 0xaa, 0x56, - 0x2, 0x13d, 0x13e, 0x7, 0x2c, 0x2, 0x2, 0x13e, 0x1a0, 0x5, 0x10, 0x9, - 0x2, 0x13f, 0x140, 0x7, 0x32, 0x2, 0x2, 0x140, 0x143, 0x7, 0x1b, 0x2, - 0x2, 0x141, 0x142, 0x7, 0x4c, 0x2, 0x2, 0x142, 0x144, 0x7, 0x37, 0x2, - 0x2, 0x143, 0x141, 0x3, 0x2, 0x2, 0x2, 0x143, 0x144, 0x3, 0x2, 0x2, - 0x2, 0x144, 0x145, 0x3, 0x2, 0x2, 0x2, 0x145, 0x1a0, 0x5, 0xb4, 0x5b, - 0x2, 0x146, 0x147, 0x7, 0x32, 0x2, 0x2, 0x147, 0x14a, 0x7, 0x4f, 0x2, - 0x2, 0x148, 0x149, 0x7, 0x4c, 0x2, 0x2, 0x149, 0x14b, 0x7, 0x37, 0x2, - 0x2, 0x14a, 0x148, 0x3, 0x2, 0x2, 0x2, 0x14a, 0x14b, 0x3, 0x2, 0x2, - 0x2, 0x14b, 0x14c, 0x3, 0x2, 0x2, 0x2, 0x14c, 0x1a0, 0x5, 0xb4, 0x5b, - 0x2, 0x14d, 0x14e, 0x7, 0x32, 0x2, 0x2, 0x14e, 0x1a0, 0x5, 0x10, 0x9, - 0x2, 0x14f, 0x151, 0x7, 0x41, 0x2, 0x2, 0x150, 0x152, 0x5, 0x10, 0x9, - 0x2, 0x151, 0x150, 0x3, 0x2, 0x2, 0x2, 0x151, 0x152, 0x3, 0x2, 0x2, - 0x2, 0x152, 0x1a0, 0x3, 0x2, 0x2, 0x2, 0x153, 0x154, 0x7, 0x6a, 0x2, - 0x2, 0x154, 0x157, 0x7, 0x1b, 0x2, 0x2, 0x155, 0x156, 0x7, 0x4c, 0x2, - 0x2, 0x156, 0x158, 0x7, 0x37, 0x2, 0x2, 0x157, 0x155, 0x3, 0x2, 0x2, - 0x2, 0x157, 0x158, 0x3, 0x2, 0x2, 0x2, 0x158, 0x159, 0x3, 0x2, 0x2, - 0x2, 0x159, 0x15a, 0x5, 0xb4, 0x5b, 0x2, 0x15a, 0x15b, 0x5, 0x4a, 0x26, - 0x2, 0x15b, 0x1a0, 0x3, 0x2, 0x2, 0x2, 0x15c, 0x15d, 0x7, 0x6a, 0x2, - 0x2, 0x15d, 0x160, 0x7, 0x1b, 0x2, 0x2, 0x15e, 0x15f, 0x7, 0x4c, 0x2, - 0x2, 0x15f, 0x161, 0x7, 0x37, 0x2, 0x2, 0x160, 0x15e, 0x3, 0x2, 0x2, - 0x2, 0x160, 0x161, 0x3, 0x2, 0x2, 0x2, 0x161, 0x162, 0x3, 0x2, 0x2, - 0x2, 0x162, 0x163, 0x5, 0xb4, 0x5b, 0x2, 0x163, 0x164, 0x7, 0x1c, 0x2, - 0x2, 0x164, 0x165, 0x7, 0xbc, 0x2, 0x2, 0x165, 0x1a0, 0x3, 0x2, 0x2, - 0x2, 0x166, 0x167, 0x7, 0x6a, 0x2, 0x2, 0x167, 0x16a, 0x7, 0x1b, 0x2, - 0x2, 0x168, 0x169, 0x7, 0x4c, 0x2, 0x2, 0x169, 0x16b, 0x7, 0x37, 0x2, - 0x2, 0x16a, 0x168, 0x3, 0x2, 0x2, 0x2, 0x16a, 0x16b, 0x3, 0x2, 0x2, - 0x2, 0x16b, 0x16c, 0x3, 0x2, 0x2, 0x2, 0x16c, 0x16d, 0x5, 0xb4, 0x5b, - 0x2, 0x16d, 0x16e, 0x7, 0x81, 0x2, 0x2, 0x16e, 0x16f, 0x5, 0xe, 0x8, - 0x2, 0x16f, 0x1a0, 0x3, 0x2, 0x2, 0x2, 0x170, 0x171, 0x7, 0x6a, 0x2, - 0x2, 0x171, 0x174, 0x7, 0x1b, 0x2, 0x2, 0x172, 0x173, 0x7, 0x4c, 0x2, - 0x2, 0x173, 0x175, 0x7, 0x37, 0x2, 0x2, 0x174, 0x172, 0x3, 0x2, 0x2, - 0x2, 0x174, 0x175, 0x3, 0x2, 0x2, 0x2, 0x175, 0x176, 0x3, 0x2, 0x2, - 0x2, 0x176, 0x1a0, 0x5, 0x44, 0x23, 0x2, 0x177, 0x178, 0x7, 0x6a, 0x2, - 0x2, 0x178, 0x179, 0x7, 0x77, 0x2, 0x2, 0x179, 0x17a, 0x7, 0x13, 0x2, - 0x2, 0x17a, 0x1a0, 0x5, 0xaa, 0x56, 0x2, 0x17b, 0x17c, 0x7, 0x6a, 0x2, - 0x2, 0x17c, 0x1a0, 0x5, 0x3e, 0x20, 0x2, 0x17d, 0x17e, 0x7, 0x6c, 0x2, - 0x2, 0x17e, 0x188, 0x5, 0x10, 0x9, 0x2, 0x17f, 0x180, 0x7, 0x9f, 0x2, - 0x2, 0x180, 0x181, 0x7, 0x2f, 0x2, 0x2, 0x181, 0x189, 0x7, 0xbc, 0x2, - 0x2, 0x182, 0x183, 0x7, 0x9f, 0x2, 0x2, 0x183, 0x184, 0x7, 0xae, 0x2, - 0x2, 0x184, 0x189, 0x7, 0xbc, 0x2, 0x2, 0x185, 0x186, 0x7, 0x9f, 0x2, - 0x2, 0x186, 0x187, 0x7, 0x97, 0x2, 0x2, 0x187, 0x189, 0x5, 0xba, 0x5e, - 0x2, 0x188, 0x17f, 0x3, 0x2, 0x2, 0x2, 0x188, 0x182, 0x3, 0x2, 0x2, - 0x2, 0x188, 0x185, 0x3, 0x2, 0x2, 0x2, 0x189, 0x1a0, 0x3, 0x2, 0x2, - 0x2, 0x18a, 0x18b, 0x7, 0x81, 0x2, 0x2, 0x18b, 0x1a0, 0x7, 0xa5, 0x2, - 0x2, 0x18c, 0x18d, 0x7, 0x82, 0x2, 0x2, 0x18d, 0x190, 0x7, 0x1b, 0x2, - 0x2, 0x18e, 0x18f, 0x7, 0x4c, 0x2, 0x2, 0x18f, 0x191, 0x7, 0x37, 0x2, - 0x2, 0x190, 0x18e, 0x3, 0x2, 0x2, 0x2, 0x190, 0x191, 0x3, 0x2, 0x2, - 0x2, 0x191, 0x192, 0x3, 0x2, 0x2, 0x2, 0x192, 0x193, 0x5, 0xb4, 0x5b, - 0x2, 0x193, 0x194, 0x7, 0x9f, 0x2, 0x2, 0x194, 0x195, 0x5, 0xb4, 0x5b, - 0x2, 0x195, 0x1a0, 0x3, 0x2, 0x2, 0x2, 0x196, 0x197, 0x7, 0x83, 0x2, - 0x2, 0x197, 0x198, 0x5, 0x10, 0x9, 0x2, 0x198, 0x199, 0x7, 0x42, 0x2, - 0x2, 0x199, 0x19a, 0x5, 0xba, 0x5e, 0x2, 0x19a, 0x1a0, 0x3, 0x2, 0x2, - 0x2, 0x19b, 0x19c, 0x7, 0xa8, 0x2, 0x2, 0x19c, 0x19d, 0x5, 0xa, 0x6, - 0x2, 0x19d, 0x19e, 0x5, 0x74, 0x3b, 0x2, 0x19e, 0x1a0, 0x3, 0x2, 0x2, - 0x2, 0x19f, 0x108, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x114, 0x3, 0x2, 0x2, - 0x2, 0x19f, 0x120, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x126, 0x3, 0x2, 0x2, - 0x2, 0x19f, 0x131, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x13a, 0x3, 0x2, 0x2, - 0x2, 0x19f, 0x13d, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x13f, 0x3, 0x2, 0x2, - 0x2, 0x19f, 0x146, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x14d, 0x3, 0x2, 0x2, - 0x2, 0x19f, 0x14f, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x153, 0x3, 0x2, 0x2, - 0x2, 0x19f, 0x15c, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x166, 0x3, 0x2, 0x2, - 0x2, 0x19f, 0x170, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x177, 0x3, 0x2, 0x2, - 0x2, 0x19f, 0x17b, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x17d, 0x3, 0x2, 0x2, - 0x2, 0x19f, 0x18a, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x18c, 0x3, 0x2, 0x2, - 0x2, 0x19f, 0x196, 0x3, 0x2, 0x2, 0x2, 0x19f, 0x19b, 0x3, 0x2, 0x2, - 0x2, 0x1a0, 0x9, 0x3, 0x2, 0x2, 0x2, 0x1a1, 0x1a6, 0x5, 0xc, 0x7, 0x2, - 0x1a2, 0x1a3, 0x7, 0xc2, 0x2, 0x2, 0x1a3, 0x1a5, 0x5, 0xc, 0x7, 0x2, - 0x1a4, 0x1a2, 0x3, 0x2, 0x2, 0x2, 0x1a5, 0x1a8, 0x3, 0x2, 0x2, 0x2, - 0x1a6, 0x1a4, 0x3, 0x2, 0x2, 0x2, 0x1a6, 0x1a7, 0x3, 0x2, 0x2, 0x2, - 0x1a7, 0xb, 0x3, 0x2, 0x2, 0x2, 0x1a8, 0x1a6, 0x3, 0x2, 0x2, 0x2, 0x1a9, - 0x1aa, 0x5, 0xb4, 0x5b, 0x2, 0x1aa, 0x1ab, 0x7, 0xc7, 0x2, 0x2, 0x1ab, - 0x1ac, 0x5, 0xaa, 0x56, 0x2, 0x1ac, 0xd, 0x3, 0x2, 0x2, 0x2, 0x1ad, - 0x1ae, 0x9, 0x2, 0x2, 0x2, 0x1ae, 0xf, 0x3, 0x2, 0x2, 0x2, 0x1af, 0x1b0, - 0x7, 0x7a, 0x2, 0x2, 0x1b0, 0x1b5, 0x5, 0xaa, 0x56, 0x2, 0x1b1, 0x1b2, - 0x7, 0x7a, 0x2, 0x2, 0x1b2, 0x1b3, 0x7, 0x4b, 0x2, 0x2, 0x1b3, 0x1b5, - 0x7, 0xbc, 0x2, 0x2, 0x1b4, 0x1af, 0x3, 0x2, 0x2, 0x2, 0x1b4, 0x1b1, - 0x3, 0x2, 0x2, 0x2, 0x1b5, 0x11, 0x3, 0x2, 0x2, 0x2, 0x1b6, 0x1b7, 0x7, - 0x10, 0x2, 0x2, 0x1b7, 0x1b8, 0x7, 0x2e, 0x2, 0x2, 0x1b8, 0x1ba, 0x5, - 0xba, 0x5e, 0x2, 0x1b9, 0x1bb, 0x5, 0x2c, 0x17, 0x2, 0x1ba, 0x1b9, 0x3, - 0x2, 0x2, 0x2, 0x1ba, 0x1bb, 0x3, 0x2, 0x2, 0x2, 0x1bb, 0x13, 0x3, 0x2, - 0x2, 0x2, 0x1bc, 0x1bd, 0x7, 0x16, 0x2, 0x2, 0x1bd, 0x1be, 0x7, 0x97, - 0x2, 0x2, 0x1be, 0x1c0, 0x5, 0xba, 0x5e, 0x2, 0x1bf, 0x1c1, 0x5, 0x10, - 0x9, 0x2, 0x1c0, 0x1bf, 0x3, 0x2, 0x2, 0x2, 0x1c0, 0x1c1, 0x3, 0x2, - 0x2, 0x2, 0x1c1, 0x15, 0x3, 0x2, 0x2, 0x2, 0x1c2, 0x1c3, 0x9, 0x3, 0x2, - 0x2, 0x1c3, 0x1c7, 0x7, 0x21, 0x2, 0x2, 0x1c4, 0x1c5, 0x7, 0x4c, 0x2, - 0x2, 0x1c5, 0x1c6, 0x7, 0x70, 0x2, 0x2, 0x1c6, 0x1c8, 0x7, 0x37, 0x2, - 0x2, 0x1c7, 0x1c4, 0x3, 0x2, 0x2, 0x2, 0x1c7, 0x1c8, 0x3, 0x2, 0x2, - 0x2, 0x1c8, 0x1c9, 0x3, 0x2, 0x2, 0x2, 0x1c9, 0x1cb, 0x5, 0xc0, 0x61, - 0x2, 0x1ca, 0x1cc, 0x5, 0x2c, 0x17, 0x2, 0x1cb, 0x1ca, 0x3, 0x2, 0x2, - 0x2, 0x1cb, 0x1cc, 0x3, 0x2, 0x2, 0x2, 0x1cc, 0x1ce, 0x3, 0x2, 0x2, - 0x2, 0x1cd, 0x1cf, 0x5, 0x40, 0x21, 0x2, 0x1ce, 0x1cd, 0x3, 0x2, 0x2, - 0x2, 0x1ce, 0x1cf, 0x3, 0x2, 0x2, 0x2, 0x1cf, 0x24c, 0x3, 0x2, 0x2, - 0x2, 0x1d0, 0x1d1, 0x9, 0x3, 0x2, 0x2, 0x1d1, 0x1d5, 0x7, 0x2e, 0x2, - 0x2, 0x1d2, 0x1d3, 0x7, 0x4c, 0x2, 0x2, 0x1d3, 0x1d4, 0x7, 0x70, 0x2, - 0x2, 0x1d4, 0x1d6, 0x7, 0x37, 0x2, 0x2, 0x1d5, 0x1d2, 0x3, 0x2, 0x2, - 0x2, 0x1d5, 0x1d6, 0x3, 0x2, 0x2, 0x2, 0x1d6, 0x1d7, 0x3, 0x2, 0x2, - 0x2, 0x1d7, 0x1d9, 0x5, 0xba, 0x5e, 0x2, 0x1d8, 0x1da, 0x5, 0x2e, 0x18, - 0x2, 0x1d9, 0x1d8, 0x3, 0x2, 0x2, 0x2, 0x1d9, 0x1da, 0x3, 0x2, 0x2, - 0x2, 0x1da, 0x1dc, 0x3, 0x2, 0x2, 0x2, 0x1db, 0x1dd, 0x5, 0x2c, 0x17, - 0x2, 0x1dc, 0x1db, 0x3, 0x2, 0x2, 0x2, 0x1dc, 0x1dd, 0x3, 0x2, 0x2, - 0x2, 0x1dd, 0x1de, 0x3, 0x2, 0x2, 0x2, 0x1de, 0x1df, 0x5, 0x18, 0xd, - 0x2, 0x1df, 0x1e0, 0x5, 0x1c, 0xf, 0x2, 0x1e0, 0x24c, 0x3, 0x2, 0x2, - 0x2, 0x1e1, 0x1e2, 0x9, 0x3, 0x2, 0x2, 0x1e2, 0x1e3, 0x7, 0x62, 0x2, - 0x2, 0x1e3, 0x1e7, 0x7, 0xad, 0x2, 0x2, 0x1e4, 0x1e5, 0x7, 0x4c, 0x2, - 0x2, 0x1e5, 0x1e6, 0x7, 0x70, 0x2, 0x2, 0x1e6, 0x1e8, 0x7, 0x37, 0x2, - 0x2, 0x1e7, 0x1e4, 0x3, 0x2, 0x2, 0x2, 0x1e7, 0x1e8, 0x3, 0x2, 0x2, - 0x2, 0x1e8, 0x1e9, 0x3, 0x2, 0x2, 0x2, 0x1e9, 0x1eb, 0x5, 0xba, 0x5e, - 0x2, 0x1ea, 0x1ec, 0x5, 0x2e, 0x18, 0x2, 0x1eb, 0x1ea, 0x3, 0x2, 0x2, - 0x2, 0x1eb, 0x1ec, 0x3, 0x2, 0x2, 0x2, 0x1ec, 0x1ee, 0x3, 0x2, 0x2, - 0x2, 0x1ed, 0x1ef, 0x5, 0x2c, 0x17, 0x2, 0x1ee, 0x1ed, 0x3, 0x2, 0x2, - 0x2, 0x1ee, 0x1ef, 0x3, 0x2, 0x2, 0x2, 0x1ef, 0x1f5, 0x3, 0x2, 0x2, - 0x2, 0x1f0, 0x1f1, 0x7, 0xb3, 0x2, 0x2, 0x1f1, 0x1f3, 0x7, 0x9d, 0x2, - 0x2, 0x1f2, 0x1f4, 0x7, 0xba, 0x2, 0x2, 0x1f3, 0x1f2, 0x3, 0x2, 0x2, - 0x2, 0x1f3, 0x1f4, 0x3, 0x2, 0x2, 0x2, 0x1f4, 0x1f6, 0x3, 0x2, 0x2, - 0x2, 0x1f5, 0x1f0, 0x3, 0x2, 0x2, 0x2, 0x1f5, 0x1f6, 0x3, 0x2, 0x2, - 0x2, 0x1f6, 0x1f8, 0x3, 0x2, 0x2, 0x2, 0x1f7, 0x1f9, 0x5, 0x30, 0x19, - 0x2, 0x1f8, 0x1f7, 0x3, 0x2, 0x2, 0x2, 0x1f8, 0x1f9, 0x3, 0x2, 0x2, - 0x2, 0x1f9, 0x1fb, 0x3, 0x2, 0x2, 0x2, 0x1fa, 0x1fc, 0x5, 0x34, 0x1b, - 0x2, 0x1fb, 0x1fa, 0x3, 0x2, 0x2, 0x2, 0x1fb, 0x1fc, 0x3, 0x2, 0x2, - 0x2, 0x1fc, 0x1fd, 0x3, 0x2, 0x2, 0x2, 0x1fd, 0x1fe, 0x5, 0x32, 0x1a, - 0x2, 0x1fe, 0x24c, 0x3, 0x2, 0x2, 0x2, 0x1ff, 0x200, 0x9, 0x3, 0x2, - 0x2, 0x200, 0x201, 0x7, 0x65, 0x2, 0x2, 0x201, 0x205, 0x7, 0xad, 0x2, - 0x2, 0x202, 0x203, 0x7, 0x4c, 0x2, 0x2, 0x203, 0x204, 0x7, 0x70, 0x2, - 0x2, 0x204, 0x206, 0x7, 0x37, 0x2, 0x2, 0x205, 0x202, 0x3, 0x2, 0x2, - 0x2, 0x205, 0x206, 0x3, 0x2, 0x2, 0x2, 0x206, 0x207, 0x3, 0x2, 0x2, - 0x2, 0x207, 0x209, 0x5, 0xba, 0x5e, 0x2, 0x208, 0x20a, 0x5, 0x2e, 0x18, - 0x2, 0x209, 0x208, 0x3, 0x2, 0x2, 0x2, 0x209, 0x20a, 0x3, 0x2, 0x2, - 0x2, 0x20a, 0x20c, 0x3, 0x2, 0x2, 0x2, 0x20b, 0x20d, 0x5, 0x2c, 0x17, - 0x2, 0x20c, 0x20b, 0x3, 0x2, 0x2, 0x2, 0x20c, 0x20d, 0x3, 0x2, 0x2, - 0x2, 0x20d, 0x20f, 0x3, 0x2, 0x2, 0x2, 0x20e, 0x210, 0x5, 0x34, 0x1b, - 0x2, 0x20f, 0x20e, 0x3, 0x2, 0x2, 0x2, 0x20f, 0x210, 0x3, 0x2, 0x2, - 0x2, 0x210, 0x216, 0x3, 0x2, 0x2, 0x2, 0x211, 0x217, 0x5, 0x30, 0x19, - 0x2, 0x212, 0x214, 0x5, 0x36, 0x1c, 0x2, 0x213, 0x215, 0x7, 0x7b, 0x2, - 0x2, 0x214, 0x213, 0x3, 0x2, 0x2, 0x2, 0x214, 0x215, 0x3, 0x2, 0x2, - 0x2, 0x215, 0x217, 0x3, 0x2, 0x2, 0x2, 0x216, 0x211, 0x3, 0x2, 0x2, - 0x2, 0x216, 0x212, 0x3, 0x2, 0x2, 0x2, 0x217, 0x218, 0x3, 0x2, 0x2, - 0x2, 0x218, 0x219, 0x5, 0x32, 0x1a, 0x2, 0x219, 0x24c, 0x3, 0x2, 0x2, - 0x2, 0x21a, 0x21c, 0x9, 0x3, 0x2, 0x2, 0x21b, 0x21d, 0x7, 0x99, 0x2, - 0x2, 0x21c, 0x21b, 0x3, 0x2, 0x2, 0x2, 0x21c, 0x21d, 0x3, 0x2, 0x2, - 0x2, 0x21d, 0x21e, 0x3, 0x2, 0x2, 0x2, 0x21e, 0x222, 0x7, 0x97, 0x2, - 0x2, 0x21f, 0x220, 0x7, 0x4c, 0x2, 0x2, 0x220, 0x221, 0x7, 0x70, 0x2, - 0x2, 0x221, 0x223, 0x7, 0x37, 0x2, 0x2, 0x222, 0x21f, 0x3, 0x2, 0x2, - 0x2, 0x222, 0x223, 0x3, 0x2, 0x2, 0x2, 0x223, 0x224, 0x3, 0x2, 0x2, - 0x2, 0x224, 0x226, 0x5, 0xba, 0x5e, 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, 0x22c, 0x3, 0x2, 0x2, 0x2, 0x22b, 0x22d, 0x5, 0x34, 0x1b, - 0x2, 0x22c, 0x22b, 0x3, 0x2, 0x2, 0x2, 0x22c, 0x22d, 0x3, 0x2, 0x2, - 0x2, 0x22d, 0x22f, 0x3, 0x2, 0x2, 0x2, 0x22e, 0x230, 0x5, 0x36, 0x1c, - 0x2, 0x22f, 0x22e, 0x3, 0x2, 0x2, 0x2, 0x22f, 0x230, 0x3, 0x2, 0x2, - 0x2, 0x230, 0x232, 0x3, 0x2, 0x2, 0x2, 0x231, 0x233, 0x5, 0x32, 0x1a, - 0x2, 0x232, 0x231, 0x3, 0x2, 0x2, 0x2, 0x232, 0x233, 0x3, 0x2, 0x2, - 0x2, 0x233, 0x24c, 0x3, 0x2, 0x2, 0x2, 0x234, 0x237, 0x9, 0x3, 0x2, - 0x2, 0x235, 0x236, 0x7, 0x76, 0x2, 0x2, 0x236, 0x238, 0x7, 0x83, 0x2, - 0x2, 0x237, 0x235, 0x3, 0x2, 0x2, 0x2, 0x237, 0x238, 0x3, 0x2, 0x2, - 0x2, 0x238, 0x239, 0x3, 0x2, 0x2, 0x2, 0x239, 0x23d, 0x7, 0xad, 0x2, - 0x2, 0x23a, 0x23b, 0x7, 0x4c, 0x2, 0x2, 0x23b, 0x23c, 0x7, 0x70, 0x2, - 0x2, 0x23c, 0x23e, 0x7, 0x37, 0x2, 0x2, 0x23d, 0x23a, 0x3, 0x2, 0x2, - 0x2, 0x23d, 0x23e, 0x3, 0x2, 0x2, 0x2, 0x23e, 0x23f, 0x3, 0x2, 0x2, - 0x2, 0x23f, 0x241, 0x5, 0xba, 0x5e, 0x2, 0x240, 0x242, 0x5, 0x2e, 0x18, - 0x2, 0x241, 0x240, 0x3, 0x2, 0x2, 0x2, 0x241, 0x242, 0x3, 0x2, 0x2, - 0x2, 0x242, 0x244, 0x3, 0x2, 0x2, 0x2, 0x243, 0x245, 0x5, 0x2c, 0x17, - 0x2, 0x244, 0x243, 0x3, 0x2, 0x2, 0x2, 0x244, 0x245, 0x3, 0x2, 0x2, - 0x2, 0x245, 0x247, 0x3, 0x2, 0x2, 0x2, 0x246, 0x248, 0x5, 0x34, 0x1b, - 0x2, 0x247, 0x246, 0x3, 0x2, 0x2, 0x2, 0x247, 0x248, 0x3, 0x2, 0x2, - 0x2, 0x248, 0x249, 0x3, 0x2, 0x2, 0x2, 0x249, 0x24a, 0x5, 0x32, 0x1a, - 0x2, 0x24a, 0x24c, 0x3, 0x2, 0x2, 0x2, 0x24b, 0x1c2, 0x3, 0x2, 0x2, - 0x2, 0x24b, 0x1d0, 0x3, 0x2, 0x2, 0x2, 0x24b, 0x1e1, 0x3, 0x2, 0x2, - 0x2, 0x24b, 0x1ff, 0x3, 0x2, 0x2, 0x2, 0x24b, 0x21a, 0x3, 0x2, 0x2, - 0x2, 0x24b, 0x234, 0x3, 0x2, 0x2, 0x2, 0x24c, 0x17, 0x3, 0x2, 0x2, 0x2, - 0x24d, 0x24e, 0x7, 0xcd, 0x2, 0x2, 0x24e, 0x253, 0x5, 0x1a, 0xe, 0x2, - 0x24f, 0x250, 0x7, 0xc2, 0x2, 0x2, 0x250, 0x252, 0x5, 0x1a, 0xe, 0x2, - 0x251, 0x24f, 0x3, 0x2, 0x2, 0x2, 0x252, 0x255, 0x3, 0x2, 0x2, 0x2, - 0x253, 0x251, 0x3, 0x2, 0x2, 0x2, 0x253, 0x254, 0x3, 0x2, 0x2, 0x2, - 0x254, 0x256, 0x3, 0x2, 0x2, 0x2, 0x255, 0x253, 0x3, 0x2, 0x2, 0x2, - 0x256, 0x257, 0x7, 0xd7, 0x2, 0x2, 0x257, 0x19, 0x3, 0x2, 0x2, 0x2, - 0x258, 0x259, 0x5, 0xd0, 0x69, 0x2, 0x259, 0x26f, 0x5, 0xa4, 0x53, 0x2, - 0x25a, 0x25b, 0x6, 0xe, 0x2, 0x3, 0x25b, 0x25c, 0x7, 0x26, 0x2, 0x2, - 0x25c, 0x25d, 0x5, 0xc6, 0x64, 0x2, 0x25d, 0x25e, 0x8, 0xe, 0x1, 0x2, - 0x25e, 0x26e, 0x3, 0x2, 0x2, 0x2, 0x25f, 0x260, 0x6, 0xe, 0x3, 0x3, - 0x260, 0x261, 0x7, 0x39, 0x2, 0x2, 0x261, 0x262, 0x5, 0xaa, 0x56, 0x2, - 0x262, 0x263, 0x8, 0xe, 0x1, 0x2, 0x263, 0x26e, 0x3, 0x2, 0x2, 0x2, - 0x264, 0x265, 0x6, 0xe, 0x4, 0x3, 0x265, 0x266, 0x7, 0x49, 0x2, 0x2, - 0x266, 0x26e, 0x8, 0xe, 0x1, 0x2, 0x267, 0x268, 0x6, 0xe, 0x5, 0x3, - 0x268, 0x269, 0x7, 0x51, 0x2, 0x2, 0x269, 0x26e, 0x8, 0xe, 0x1, 0x2, - 0x26a, 0x26b, 0x6, 0xe, 0x6, 0x3, 0x26b, 0x26c, 0x7, 0x57, 0x2, 0x2, - 0x26c, 0x26e, 0x8, 0xe, 0x1, 0x2, 0x26d, 0x25a, 0x3, 0x2, 0x2, 0x2, - 0x26d, 0x25f, 0x3, 0x2, 0x2, 0x2, 0x26d, 0x264, 0x3, 0x2, 0x2, 0x2, - 0x26d, 0x267, 0x3, 0x2, 0x2, 0x2, 0x26d, 0x26a, 0x3, 0x2, 0x2, 0x2, - 0x26e, 0x271, 0x3, 0x2, 0x2, 0x2, 0x26f, 0x26d, 0x3, 0x2, 0x2, 0x2, - 0x26f, 0x270, 0x3, 0x2, 0x2, 0x2, 0x270, 0x1b, 0x3, 0x2, 0x2, 0x2, 0x271, - 0x26f, 0x3, 0x2, 0x2, 0x2, 0x272, 0x274, 0x5, 0x1e, 0x10, 0x2, 0x273, - 0x272, 0x3, 0x2, 0x2, 0x2, 0x273, 0x274, 0x3, 0x2, 0x2, 0x2, 0x274, - 0x28b, 0x3, 0x2, 0x2, 0x2, 0x275, 0x276, 0x6, 0xf, 0x7, 0x3, 0x276, - 0x277, 0x5, 0x22, 0x12, 0x2, 0x277, 0x278, 0x8, 0xf, 0x1, 0x2, 0x278, - 0x28a, 0x3, 0x2, 0x2, 0x2, 0x279, 0x27a, 0x6, 0xf, 0x8, 0x3, 0x27a, - 0x27b, 0x5, 0x24, 0x13, 0x2, 0x27b, 0x27c, 0x8, 0xf, 0x1, 0x2, 0x27c, - 0x28a, 0x3, 0x2, 0x2, 0x2, 0x27d, 0x27e, 0x6, 0xf, 0x9, 0x3, 0x27e, - 0x27f, 0x5, 0x26, 0x14, 0x2, 0x27f, 0x280, 0x8, 0xf, 0x1, 0x2, 0x280, - 0x28a, 0x3, 0x2, 0x2, 0x2, 0x281, 0x282, 0x6, 0xf, 0xa, 0x3, 0x282, - 0x283, 0x5, 0x28, 0x15, 0x2, 0x283, 0x284, 0x8, 0xf, 0x1, 0x2, 0x284, - 0x28a, 0x3, 0x2, 0x2, 0x2, 0x285, 0x286, 0x6, 0xf, 0xb, 0x3, 0x286, - 0x287, 0x5, 0x2a, 0x16, 0x2, 0x287, 0x288, 0x8, 0xf, 0x1, 0x2, 0x288, - 0x28a, 0x3, 0x2, 0x2, 0x2, 0x289, 0x275, 0x3, 0x2, 0x2, 0x2, 0x289, - 0x279, 0x3, 0x2, 0x2, 0x2, 0x289, 0x27d, 0x3, 0x2, 0x2, 0x2, 0x289, - 0x281, 0x3, 0x2, 0x2, 0x2, 0x289, 0x285, 0x3, 0x2, 0x2, 0x2, 0x28a, - 0x28d, 0x3, 0x2, 0x2, 0x2, 0x28b, 0x289, 0x3, 0x2, 0x2, 0x2, 0x28b, - 0x28c, 0x3, 0x2, 0x2, 0x2, 0x28c, 0x1d, 0x3, 0x2, 0x2, 0x2, 0x28d, 0x28b, - 0x3, 0x2, 0x2, 0x2, 0x28e, 0x28f, 0x7, 0x7d, 0x2, 0x2, 0x28f, 0x290, - 0x7, 0x59, 0x2, 0x2, 0x290, 0x291, 0x5, 0xa6, 0x54, 0x2, 0x291, 0x1f, - 0x3, 0x2, 0x2, 0x2, 0x292, 0x299, 0x5, 0xd0, 0x69, 0x2, 0x293, 0x296, - 0x5, 0xd0, 0x69, 0x2, 0x294, 0x295, 0x7, 0xcd, 0x2, 0x2, 0x295, 0x297, - 0x7, 0xd7, 0x2, 0x2, 0x296, 0x294, 0x3, 0x2, 0x2, 0x2, 0x296, 0x297, - 0x3, 0x2, 0x2, 0x2, 0x297, 0x29a, 0x3, 0x2, 0x2, 0x2, 0x298, 0x29a, - 0x5, 0xc6, 0x64, 0x2, 0x299, 0x293, 0x3, 0x2, 0x2, 0x2, 0x299, 0x298, - 0x3, 0x2, 0x2, 0x2, 0x29a, 0x21, 0x3, 0x2, 0x2, 0x2, 0x29b, 0x29c, 0x7, - 0x90, 0x2, 0x2, 0x29c, 0x29d, 0x7, 0xcd, 0x2, 0x2, 0x29d, 0x29e, 0x5, - 0xd0, 0x69, 0x2, 0x29e, 0x2a2, 0x7, 0xcd, 0x2, 0x2, 0x29f, 0x2a1, 0x5, - 0x20, 0x11, 0x2, 0x2a0, 0x29f, 0x3, 0x2, 0x2, 0x2, 0x2a1, 0x2a4, 0x3, - 0x2, 0x2, 0x2, 0x2a2, 0x2a0, 0x3, 0x2, 0x2, 0x2, 0x2a2, 0x2a3, 0x3, - 0x2, 0x2, 0x2, 0x2a3, 0x2a5, 0x3, 0x2, 0x2, 0x2, 0x2a4, 0x2a2, 0x3, - 0x2, 0x2, 0x2, 0x2a5, 0x2a6, 0x7, 0xd7, 0x2, 0x2, 0x2a6, 0x2a7, 0x7, - 0xd7, 0x2, 0x2, 0x2a7, 0x23, 0x3, 0x2, 0x2, 0x2, 0x2a8, 0x2a9, 0x7, - 0x5f, 0x2, 0x2, 0x2a9, 0x2b3, 0x7, 0xcd, 0x2, 0x2, 0x2aa, 0x2b4, 0x7, - 0xba, 0x2, 0x2, 0x2ab, 0x2ac, 0x7, 0x68, 0x2, 0x2, 0x2ac, 0x2ad, 0x7, - 0xba, 0x2, 0x2, 0x2ad, 0x2ae, 0x7, 0x66, 0x2, 0x2, 0x2ae, 0x2b4, 0x7, - 0xba, 0x2, 0x2, 0x2af, 0x2b0, 0x7, 0x66, 0x2, 0x2, 0x2b0, 0x2b1, 0x7, - 0xba, 0x2, 0x2, 0x2b1, 0x2b2, 0x7, 0x68, 0x2, 0x2, 0x2b2, 0x2b4, 0x7, - 0xba, 0x2, 0x2, 0x2b3, 0x2aa, 0x3, 0x2, 0x2, 0x2, 0x2b3, 0x2ab, 0x3, - 0x2, 0x2, 0x2, 0x2b3, 0x2af, 0x3, 0x2, 0x2, 0x2, 0x2b4, 0x2b5, 0x3, - 0x2, 0x2, 0x2, 0x2b5, 0x2b6, 0x7, 0xd7, 0x2, 0x2, 0x2b6, 0x25, 0x3, - 0x2, 0x2, 0x2, 0x2b7, 0x2b8, 0x7, 0x5c, 0x2, 0x2, 0x2b8, 0x2b9, 0x7, - 0xcd, 0x2, 0x2, 0x2b9, 0x2ba, 0x5, 0xd0, 0x69, 0x2, 0x2ba, 0x2be, 0x7, - 0xcd, 0x2, 0x2, 0x2bb, 0x2bd, 0x5, 0x20, 0x11, 0x2, 0x2bc, 0x2bb, 0x3, - 0x2, 0x2, 0x2, 0x2bd, 0x2c0, 0x3, 0x2, 0x2, 0x2, 0x2be, 0x2bc, 0x3, - 0x2, 0x2, 0x2, 0x2be, 0x2bf, 0x3, 0x2, 0x2, 0x2, 0x2bf, 0x2c1, 0x3, - 0x2, 0x2, 0x2, 0x2c0, 0x2be, 0x3, 0x2, 0x2, 0x2, 0x2c1, 0x2c2, 0x7, - 0xd7, 0x2, 0x2, 0x2c2, 0x2c3, 0x7, 0xd7, 0x2, 0x2, 0x2c3, 0x27, 0x3, - 0x2, 0x2, 0x2, 0x2c4, 0x2c5, 0x7, 0x7f, 0x2, 0x2, 0x2c5, 0x2d0, 0x7, - 0xcd, 0x2, 0x2, 0x2c6, 0x2c7, 0x7, 0x68, 0x2, 0x2, 0x2c7, 0x2c8, 0x5, - 0xd0, 0x69, 0x2, 0x2c8, 0x2c9, 0x7, 0x66, 0x2, 0x2, 0x2c9, 0x2ca, 0x5, - 0xd0, 0x69, 0x2, 0x2ca, 0x2d1, 0x3, 0x2, 0x2, 0x2, 0x2cb, 0x2cc, 0x7, - 0x66, 0x2, 0x2, 0x2cc, 0x2cd, 0x5, 0xd0, 0x69, 0x2, 0x2cd, 0x2ce, 0x7, - 0x68, 0x2, 0x2, 0x2ce, 0x2cf, 0x5, 0xd0, 0x69, 0x2, 0x2cf, 0x2d1, 0x3, - 0x2, 0x2, 0x2, 0x2d0, 0x2c6, 0x3, 0x2, 0x2, 0x2, 0x2d0, 0x2cb, 0x3, - 0x2, 0x2, 0x2, 0x2d1, 0x2d2, 0x3, 0x2, 0x2, 0x2, 0x2d2, 0x2d3, 0x7, - 0xd7, 0x2, 0x2, 0x2d3, 0x29, 0x3, 0x2, 0x2, 0x2, 0x2d4, 0x2d5, 0x7, - 0x8e, 0x2, 0x2, 0x2d5, 0x2d6, 0x7, 0xcd, 0x2, 0x2, 0x2d6, 0x2d7, 0x5, - 0x94, 0x4b, 0x2, 0x2d7, 0x2d8, 0x7, 0xd7, 0x2, 0x2, 0x2d8, 0x2b, 0x3, - 0x2, 0x2, 0x2, 0x2d9, 0x2da, 0x7, 0x74, 0x2, 0x2, 0x2da, 0x2dd, 0x7, - 0x18, 0x2, 0x2, 0x2db, 0x2de, 0x5, 0xd0, 0x69, 0x2, 0x2dc, 0x2de, 0x7, - 0xbc, 0x2, 0x2, 0x2dd, 0x2db, 0x3, 0x2, 0x2, 0x2, 0x2dd, 0x2dc, 0x3, - 0x2, 0x2, 0x2, 0x2de, 0x2d, 0x3, 0x2, 0x2, 0x2, 0x2df, 0x2e0, 0x7, 0xab, - 0x2, 0x2, 0x2e0, 0x2e1, 0x7, 0xbc, 0x2, 0x2, 0x2e1, 0x2f, 0x3, 0x2, - 0x2, 0x2, 0x2e2, 0x2e3, 0x7, 0x9f, 0x2, 0x2, 0x2e3, 0x2e4, 0x5, 0xba, - 0x5e, 0x2, 0x2e4, 0x31, 0x3, 0x2, 0x2, 0x2, 0x2e5, 0x2e6, 0x7, 0xc, - 0x2, 0x2, 0x2e6, 0x2e7, 0x5, 0x64, 0x33, 0x2, 0x2e7, 0x33, 0x3, 0x2, - 0x2, 0x2, 0x2e8, 0x2e9, 0x7, 0xcd, 0x2, 0x2, 0x2e9, 0x2ee, 0x5, 0x42, - 0x22, 0x2, 0x2ea, 0x2eb, 0x7, 0xc2, 0x2, 0x2, 0x2eb, 0x2ed, 0x5, 0x42, - 0x22, 0x2, 0x2ec, 0x2ea, 0x3, 0x2, 0x2, 0x2, 0x2ed, 0x2f0, 0x3, 0x2, - 0x2, 0x2, 0x2ee, 0x2ec, 0x3, 0x2, 0x2, 0x2, 0x2ee, 0x2ef, 0x3, 0x2, - 0x2, 0x2, 0x2ef, 0x2f1, 0x3, 0x2, 0x2, 0x2, 0x2f0, 0x2ee, 0x3, 0x2, - 0x2, 0x2, 0x2f1, 0x2f2, 0x7, 0xd7, 0x2, 0x2, 0x2f2, 0x2f8, 0x3, 0x2, - 0x2, 0x2, 0x2f3, 0x2f4, 0x7, 0xc, 0x2, 0x2, 0x2f4, 0x2f8, 0x5, 0xba, - 0x5e, 0x2, 0x2f5, 0x2f6, 0x7, 0xc, 0x2, 0x2, 0x2f6, 0x2f8, 0x5, 0xb8, - 0x5d, 0x2, 0x2f7, 0x2e8, 0x3, 0x2, 0x2, 0x2, 0x2f7, 0x2f3, 0x3, 0x2, - 0x2, 0x2, 0x2f7, 0x2f5, 0x3, 0x2, 0x2, 0x2, 0x2f8, 0x35, 0x3, 0x2, 0x2, - 0x2, 0x2f9, 0x314, 0x5, 0x40, 0x21, 0x2, 0x2fa, 0x2fb, 0x6, 0x1c, 0xc, - 0x3, 0x2fb, 0x2fc, 0x5, 0x7a, 0x3e, 0x2, 0x2fc, 0x2fd, 0x8, 0x1c, 0x1, - 0x2, 0x2fd, 0x313, 0x3, 0x2, 0x2, 0x2, 0x2fe, 0x2ff, 0x6, 0x1c, 0xd, - 0x3, 0x2ff, 0x300, 0x5, 0x38, 0x1d, 0x2, 0x300, 0x301, 0x8, 0x1c, 0x1, - 0x2, 0x301, 0x313, 0x3, 0x2, 0x2, 0x2, 0x302, 0x303, 0x6, 0x1c, 0xe, - 0x3, 0x303, 0x304, 0x5, 0x3a, 0x1e, 0x2, 0x304, 0x305, 0x8, 0x1c, 0x1, - 0x2, 0x305, 0x313, 0x3, 0x2, 0x2, 0x2, 0x306, 0x307, 0x6, 0x1c, 0xf, - 0x3, 0x307, 0x308, 0x5, 0x3c, 0x1f, 0x2, 0x308, 0x309, 0x8, 0x1c, 0x1, - 0x2, 0x309, 0x313, 0x3, 0x2, 0x2, 0x2, 0x30a, 0x30b, 0x6, 0x1c, 0x10, - 0x3, 0x30b, 0x30c, 0x5, 0x3e, 0x20, 0x2, 0x30c, 0x30d, 0x8, 0x1c, 0x1, - 0x2, 0x30d, 0x313, 0x3, 0x2, 0x2, 0x2, 0x30e, 0x30f, 0x6, 0x1c, 0x11, - 0x3, 0x30f, 0x310, 0x5, 0x80, 0x41, 0x2, 0x310, 0x311, 0x8, 0x1c, 0x1, - 0x2, 0x311, 0x313, 0x3, 0x2, 0x2, 0x2, 0x312, 0x2fa, 0x3, 0x2, 0x2, - 0x2, 0x312, 0x2fe, 0x3, 0x2, 0x2, 0x2, 0x312, 0x302, 0x3, 0x2, 0x2, - 0x2, 0x312, 0x306, 0x3, 0x2, 0x2, 0x2, 0x312, 0x30a, 0x3, 0x2, 0x2, - 0x2, 0x312, 0x30e, 0x3, 0x2, 0x2, 0x2, 0x313, 0x316, 0x3, 0x2, 0x2, - 0x2, 0x314, 0x312, 0x3, 0x2, 0x2, 0x2, 0x314, 0x315, 0x3, 0x2, 0x2, - 0x2, 0x315, 0x37, 0x3, 0x2, 0x2, 0x2, 0x316, 0x314, 0x3, 0x2, 0x2, 0x2, - 0x317, 0x318, 0x7, 0x7a, 0x2, 0x2, 0x318, 0x319, 0x7, 0x13, 0x2, 0x2, - 0x319, 0x31a, 0x5, 0xaa, 0x56, 0x2, 0x31a, 0x39, 0x3, 0x2, 0x2, 0x2, - 0x31b, 0x31c, 0x7, 0x7d, 0x2, 0x2, 0x31c, 0x31d, 0x7, 0x59, 0x2, 0x2, - 0x31d, 0x31e, 0x5, 0xaa, 0x56, 0x2, 0x31e, 0x3b, 0x3, 0x2, 0x2, 0x2, - 0x31f, 0x320, 0x7, 0x88, 0x2, 0x2, 0x320, 0x321, 0x7, 0x13, 0x2, 0x2, - 0x321, 0x322, 0x5, 0xaa, 0x56, 0x2, 0x322, 0x3d, 0x3, 0x2, 0x2, 0x2, - 0x323, 0x324, 0x7, 0xa5, 0x2, 0x2, 0x324, 0x329, 0x5, 0x4e, 0x28, 0x2, - 0x325, 0x326, 0x7, 0xc2, 0x2, 0x2, 0x326, 0x328, 0x5, 0x4e, 0x28, 0x2, - 0x327, 0x325, 0x3, 0x2, 0x2, 0x2, 0x328, 0x32b, 0x3, 0x2, 0x2, 0x2, - 0x329, 0x327, 0x3, 0x2, 0x2, 0x2, 0x329, 0x32a, 0x3, 0x2, 0x2, 0x2, - 0x32a, 0x3f, 0x3, 0x2, 0x2, 0x2, 0x32b, 0x329, 0x3, 0x2, 0x2, 0x2, 0x32c, - 0x32e, 0x7, 0x35, 0x2, 0x2, 0x32d, 0x32f, 0x7, 0xc7, 0x2, 0x2, 0x32e, - 0x32d, 0x3, 0x2, 0x2, 0x2, 0x32e, 0x32f, 0x3, 0x2, 0x2, 0x2, 0x32f, - 0x330, 0x3, 0x2, 0x2, 0x2, 0x330, 0x336, 0x5, 0xd2, 0x6a, 0x2, 0x331, - 0x333, 0x7, 0xcd, 0x2, 0x2, 0x332, 0x334, 0x5, 0xa6, 0x54, 0x2, 0x333, - 0x332, 0x3, 0x2, 0x2, 0x2, 0x333, 0x334, 0x3, 0x2, 0x2, 0x2, 0x334, - 0x335, 0x3, 0x2, 0x2, 0x2, 0x335, 0x337, 0x7, 0xd7, 0x2, 0x2, 0x336, - 0x331, 0x3, 0x2, 0x2, 0x2, 0x336, 0x337, 0x3, 0x2, 0x2, 0x2, 0x337, - 0x41, 0x3, 0x2, 0x2, 0x2, 0x338, 0x341, 0x5, 0x44, 0x23, 0x2, 0x339, - 0x33a, 0x7, 0x1d, 0x2, 0x2, 0x33a, 0x33b, 0x5, 0xd0, 0x69, 0x2, 0x33b, - 0x33c, 0x7, 0x16, 0x2, 0x2, 0x33c, 0x33d, 0x5, 0xaa, 0x56, 0x2, 0x33d, - 0x341, 0x3, 0x2, 0x2, 0x2, 0x33e, 0x33f, 0x7, 0x4f, 0x2, 0x2, 0x33f, - 0x341, 0x5, 0x48, 0x25, 0x2, 0x340, 0x338, 0x3, 0x2, 0x2, 0x2, 0x340, - 0x339, 0x3, 0x2, 0x2, 0x2, 0x340, 0x33e, 0x3, 0x2, 0x2, 0x2, 0x341, - 0x43, 0x3, 0x2, 0x2, 0x2, 0x342, 0x343, 0x5, 0xb4, 0x5b, 0x2, 0x343, - 0x345, 0x5, 0xa4, 0x53, 0x2, 0x344, 0x346, 0x5, 0x46, 0x24, 0x2, 0x345, - 0x344, 0x3, 0x2, 0x2, 0x2, 0x345, 0x346, 0x3, 0x2, 0x2, 0x2, 0x346, - 0x349, 0x3, 0x2, 0x2, 0x2, 0x347, 0x348, 0x7, 0x1c, 0x2, 0x2, 0x348, - 0x34a, 0x7, 0xbc, 0x2, 0x2, 0x349, 0x347, 0x3, 0x2, 0x2, 0x2, 0x349, - 0x34a, 0x3, 0x2, 0x2, 0x2, 0x34a, 0x34c, 0x3, 0x2, 0x2, 0x2, 0x34b, - 0x34d, 0x5, 0x4a, 0x26, 0x2, 0x34c, 0x34b, 0x3, 0x2, 0x2, 0x2, 0x34c, - 0x34d, 0x3, 0x2, 0x2, 0x2, 0x34d, 0x350, 0x3, 0x2, 0x2, 0x2, 0x34e, - 0x34f, 0x7, 0xa5, 0x2, 0x2, 0x34f, 0x351, 0x5, 0xaa, 0x56, 0x2, 0x350, - 0x34e, 0x3, 0x2, 0x2, 0x2, 0x350, 0x351, 0x3, 0x2, 0x2, 0x2, 0x351, - 0x363, 0x3, 0x2, 0x2, 0x2, 0x352, 0x354, 0x5, 0xb4, 0x5b, 0x2, 0x353, - 0x355, 0x5, 0xa4, 0x53, 0x2, 0x354, 0x353, 0x3, 0x2, 0x2, 0x2, 0x354, - 0x355, 0x3, 0x2, 0x2, 0x2, 0x355, 0x356, 0x3, 0x2, 0x2, 0x2, 0x356, - 0x359, 0x5, 0x46, 0x24, 0x2, 0x357, 0x358, 0x7, 0x1c, 0x2, 0x2, 0x358, - 0x35a, 0x7, 0xbc, 0x2, 0x2, 0x359, 0x357, 0x3, 0x2, 0x2, 0x2, 0x359, - 0x35a, 0x3, 0x2, 0x2, 0x2, 0x35a, 0x35c, 0x3, 0x2, 0x2, 0x2, 0x35b, - 0x35d, 0x5, 0x4a, 0x26, 0x2, 0x35c, 0x35b, 0x3, 0x2, 0x2, 0x2, 0x35c, - 0x35d, 0x3, 0x2, 0x2, 0x2, 0x35d, 0x360, 0x3, 0x2, 0x2, 0x2, 0x35e, - 0x35f, 0x7, 0xa5, 0x2, 0x2, 0x35f, 0x361, 0x5, 0xaa, 0x56, 0x2, 0x360, - 0x35e, 0x3, 0x2, 0x2, 0x2, 0x360, 0x361, 0x3, 0x2, 0x2, 0x2, 0x361, - 0x363, 0x3, 0x2, 0x2, 0x2, 0x362, 0x342, 0x3, 0x2, 0x2, 0x2, 0x362, - 0x352, 0x3, 0x2, 0x2, 0x2, 0x363, 0x45, 0x3, 0x2, 0x2, 0x2, 0x364, 0x365, - 0x9, 0x4, 0x2, 0x2, 0x365, 0x366, 0x5, 0xaa, 0x56, 0x2, 0x366, 0x47, - 0x3, 0x2, 0x2, 0x2, 0x367, 0x368, 0x5, 0xb4, 0x5b, 0x2, 0x368, 0x369, - 0x5, 0xaa, 0x56, 0x2, 0x369, 0x36a, 0x7, 0xa6, 0x2, 0x2, 0x36a, 0x36b, - 0x5, 0xa4, 0x53, 0x2, 0x36b, 0x36c, 0x7, 0x46, 0x2, 0x2, 0x36c, 0x36d, - 0x7, 0xba, 0x2, 0x2, 0x36d, 0x49, 0x3, 0x2, 0x2, 0x2, 0x36e, 0x36f, - 0x7, 0x19, 0x2, 0x2, 0x36f, 0x370, 0x7, 0xcd, 0x2, 0x2, 0x370, 0x375, - 0x5, 0x4c, 0x27, 0x2, 0x371, 0x372, 0x7, 0xc2, 0x2, 0x2, 0x372, 0x374, - 0x5, 0x4c, 0x27, 0x2, 0x373, 0x371, 0x3, 0x2, 0x2, 0x2, 0x374, 0x377, - 0x3, 0x2, 0x2, 0x2, 0x375, 0x373, 0x3, 0x2, 0x2, 0x2, 0x375, 0x376, - 0x3, 0x2, 0x2, 0x2, 0x376, 0x378, 0x3, 0x2, 0x2, 0x2, 0x377, 0x375, - 0x3, 0x2, 0x2, 0x2, 0x378, 0x379, 0x7, 0xd7, 0x2, 0x2, 0x379, 0x4b, - 0x3, 0x2, 0x2, 0x2, 0x37a, 0x380, 0x5, 0xd0, 0x69, 0x2, 0x37b, 0x37d, - 0x7, 0xcd, 0x2, 0x2, 0x37c, 0x37e, 0x5, 0xa6, 0x54, 0x2, 0x37d, 0x37c, - 0x3, 0x2, 0x2, 0x2, 0x37d, 0x37e, 0x3, 0x2, 0x2, 0x2, 0x37e, 0x37f, - 0x3, 0x2, 0x2, 0x2, 0x37f, 0x381, 0x7, 0xd7, 0x2, 0x2, 0x380, 0x37b, - 0x3, 0x2, 0x2, 0x2, 0x380, 0x381, 0x3, 0x2, 0x2, 0x2, 0x381, 0x4d, 0x3, - 0x2, 0x2, 0x2, 0x382, 0x38a, 0x5, 0xaa, 0x56, 0x2, 0x383, 0x38b, 0x7, - 0x28, 0x2, 0x2, 0x384, 0x385, 0x7, 0x9f, 0x2, 0x2, 0x385, 0x386, 0x7, - 0x2f, 0x2, 0x2, 0x386, 0x38b, 0x7, 0xbc, 0x2, 0x2, 0x387, 0x388, 0x7, - 0x9f, 0x2, 0x2, 0x388, 0x389, 0x7, 0xae, 0x2, 0x2, 0x389, 0x38b, 0x7, - 0xbc, 0x2, 0x2, 0x38a, 0x383, 0x3, 0x2, 0x2, 0x2, 0x38a, 0x384, 0x3, - 0x2, 0x2, 0x2, 0x38a, 0x387, 0x3, 0x2, 0x2, 0x2, 0x38a, 0x38b, 0x3, - 0x2, 0x2, 0x2, 0x38b, 0x4f, 0x3, 0x2, 0x2, 0x2, 0x38c, 0x38e, 0x9, 0x5, - 0x2, 0x2, 0x38d, 0x38f, 0x7, 0x97, 0x2, 0x2, 0x38e, 0x38d, 0x3, 0x2, - 0x2, 0x2, 0x38e, 0x38f, 0x3, 0x2, 0x2, 0x2, 0x38f, 0x390, 0x3, 0x2, - 0x2, 0x2, 0x390, 0x391, 0x5, 0xb6, 0x5c, 0x2, 0x391, 0x51, 0x3, 0x2, - 0x2, 0x2, 0x392, 0x393, 0x9, 0x6, 0x2, 0x2, 0x393, 0x396, 0x7, 0x21, - 0x2, 0x2, 0x394, 0x395, 0x7, 0x4c, 0x2, 0x2, 0x395, 0x397, 0x7, 0x37, - 0x2, 0x2, 0x396, 0x394, 0x3, 0x2, 0x2, 0x2, 0x396, 0x397, 0x3, 0x2, - 0x2, 0x2, 0x397, 0x398, 0x3, 0x2, 0x2, 0x2, 0x398, 0x39a, 0x5, 0xc0, - 0x61, 0x2, 0x399, 0x39b, 0x5, 0x2c, 0x17, 0x2, 0x39a, 0x399, 0x3, 0x2, - 0x2, 0x2, 0x39a, 0x39b, 0x3, 0x2, 0x2, 0x2, 0x39b, 0x3b2, 0x3, 0x2, - 0x2, 0x2, 0x39c, 0x3a3, 0x9, 0x6, 0x2, 0x2, 0x39d, 0x3a4, 0x7, 0x2e, - 0x2, 0x2, 0x39e, 0x3a0, 0x7, 0x99, 0x2, 0x2, 0x39f, 0x39e, 0x3, 0x2, - 0x2, 0x2, 0x39f, 0x3a0, 0x3, 0x2, 0x2, 0x2, 0x3a0, 0x3a1, 0x3, 0x2, - 0x2, 0x2, 0x3a1, 0x3a4, 0x7, 0x97, 0x2, 0x2, 0x3a2, 0x3a4, 0x7, 0xad, - 0x2, 0x2, 0x3a3, 0x39d, 0x3, 0x2, 0x2, 0x2, 0x3a3, 0x39f, 0x3, 0x2, - 0x2, 0x2, 0x3a3, 0x3a2, 0x3, 0x2, 0x2, 0x2, 0x3a4, 0x3a7, 0x3, 0x2, - 0x2, 0x2, 0x3a5, 0x3a6, 0x7, 0x4c, 0x2, 0x2, 0x3a6, 0x3a8, 0x7, 0x37, - 0x2, 0x2, 0x3a7, 0x3a5, 0x3, 0x2, 0x2, 0x2, 0x3a7, 0x3a8, 0x3, 0x2, - 0x2, 0x2, 0x3a8, 0x3a9, 0x3, 0x2, 0x2, 0x2, 0x3a9, 0x3ab, 0x5, 0xba, - 0x5e, 0x2, 0x3aa, 0x3ac, 0x5, 0x2c, 0x17, 0x2, 0x3ab, 0x3aa, 0x3, 0x2, - 0x2, 0x2, 0x3ab, 0x3ac, 0x3, 0x2, 0x2, 0x2, 0x3ac, 0x3af, 0x3, 0x2, - 0x2, 0x2, 0x3ad, 0x3ae, 0x7, 0x6f, 0x2, 0x2, 0x3ae, 0x3b0, 0x7, 0x27, - 0x2, 0x2, 0x3af, 0x3ad, 0x3, 0x2, 0x2, 0x2, 0x3af, 0x3b0, 0x3, 0x2, - 0x2, 0x2, 0x3b0, 0x3b2, 0x3, 0x2, 0x2, 0x2, 0x3b1, 0x392, 0x3, 0x2, - 0x2, 0x2, 0x3b1, 0x39c, 0x3, 0x2, 0x2, 0x2, 0x3b2, 0x53, 0x3, 0x2, 0x2, - 0x2, 0x3b3, 0x3b4, 0x7, 0x37, 0x2, 0x2, 0x3b4, 0x3b5, 0x7, 0x21, 0x2, - 0x2, 0x3b5, 0x3c1, 0x5, 0xc0, 0x61, 0x2, 0x3b6, 0x3bd, 0x7, 0x37, 0x2, - 0x2, 0x3b7, 0x3be, 0x7, 0x2e, 0x2, 0x2, 0x3b8, 0x3ba, 0x7, 0x99, 0x2, - 0x2, 0x3b9, 0x3b8, 0x3, 0x2, 0x2, 0x2, 0x3b9, 0x3ba, 0x3, 0x2, 0x2, - 0x2, 0x3ba, 0x3bb, 0x3, 0x2, 0x2, 0x2, 0x3bb, 0x3be, 0x7, 0x97, 0x2, - 0x2, 0x3bc, 0x3be, 0x7, 0xad, 0x2, 0x2, 0x3bd, 0x3b7, 0x3, 0x2, 0x2, - 0x2, 0x3bd, 0x3b9, 0x3, 0x2, 0x2, 0x2, 0x3bd, 0x3bc, 0x3, 0x2, 0x2, - 0x2, 0x3bd, 0x3be, 0x3, 0x2, 0x2, 0x2, 0x3be, 0x3bf, 0x3, 0x2, 0x2, - 0x2, 0x3bf, 0x3c1, 0x5, 0xba, 0x5e, 0x2, 0x3c0, 0x3b3, 0x3, 0x2, 0x2, - 0x2, 0x3c0, 0x3b6, 0x3, 0x2, 0x2, 0x2, 0x3c1, 0x55, 0x3, 0x2, 0x2, 0x2, - 0x3c2, 0x3c3, 0x7, 0x38, 0x2, 0x2, 0x3c3, 0x3c4, 0x7, 0x95, 0x2, 0x2, - 0x3c4, 0x3c5, 0x5, 0x4, 0x3, 0x2, 0x3c5, 0x57, 0x3, 0x2, 0x2, 0x2, 0x3c6, - 0x3c7, 0x7, 0x53, 0x2, 0x2, 0x3c7, 0x3c9, 0x7, 0x55, 0x2, 0x2, 0x3c8, - 0x3ca, 0x7, 0x97, 0x2, 0x2, 0x3c9, 0x3c8, 0x3, 0x2, 0x2, 0x2, 0x3c9, - 0x3ca, 0x3, 0x2, 0x2, 0x2, 0x3ca, 0x3ce, 0x3, 0x2, 0x2, 0x2, 0x3cb, - 0x3cf, 0x5, 0xba, 0x5e, 0x2, 0x3cc, 0x3cd, 0x7, 0x44, 0x2, 0x2, 0x3cd, - 0x3cf, 0x5, 0xb8, 0x5d, 0x2, 0x3ce, 0x3cb, 0x3, 0x2, 0x2, 0x2, 0x3ce, - 0x3cc, 0x3, 0x2, 0x2, 0x2, 0x3cf, 0x3d1, 0x3, 0x2, 0x2, 0x2, 0x3d0, - 0x3d2, 0x5, 0x5a, 0x2e, 0x2, 0x3d1, 0x3d0, 0x3, 0x2, 0x2, 0x2, 0x3d1, - 0x3d2, 0x3, 0x2, 0x2, 0x2, 0x3d2, 0x3d3, 0x3, 0x2, 0x2, 0x2, 0x3d3, - 0x3d4, 0x5, 0x5c, 0x2f, 0x2, 0x3d4, 0x59, 0x3, 0x2, 0x2, 0x2, 0x3d5, - 0x3d6, 0x7, 0xcd, 0x2, 0x2, 0x3d6, 0x3db, 0x5, 0xb4, 0x5b, 0x2, 0x3d7, - 0x3d8, 0x7, 0xc2, 0x2, 0x2, 0x3d8, 0x3da, 0x5, 0xb4, 0x5b, 0x2, 0x3d9, - 0x3d7, 0x3, 0x2, 0x2, 0x2, 0x3da, 0x3dd, 0x3, 0x2, 0x2, 0x2, 0x3db, - 0x3d9, 0x3, 0x2, 0x2, 0x2, 0x3db, 0x3dc, 0x3, 0x2, 0x2, 0x2, 0x3dc, - 0x3de, 0x3, 0x2, 0x2, 0x2, 0x3dd, 0x3db, 0x3, 0x2, 0x2, 0x2, 0x3de, - 0x3df, 0x7, 0xd7, 0x2, 0x2, 0x3df, 0x5b, 0x3, 0x2, 0x2, 0x2, 0x3e0, - 0x3e1, 0x7, 0x40, 0x2, 0x2, 0x3e1, 0x3ea, 0x5, 0xd0, 0x69, 0x2, 0x3e2, - 0x3ea, 0x7, 0xac, 0x2, 0x2, 0x3e3, 0x3e5, 0x5, 0x64, 0x33, 0x2, 0x3e4, - 0x3e6, 0x7, 0xd8, 0x2, 0x2, 0x3e5, 0x3e4, 0x3, 0x2, 0x2, 0x2, 0x3e5, - 0x3e6, 0x3, 0x2, 0x2, 0x2, 0x3e6, 0x3e7, 0x3, 0x2, 0x2, 0x2, 0x3e7, - 0x3e8, 0x7, 0x2, 0x2, 0x3, 0x3e8, 0x3ea, 0x3, 0x2, 0x2, 0x2, 0x3e9, - 0x3e0, 0x3, 0x2, 0x2, 0x2, 0x3e9, 0x3e2, 0x3, 0x2, 0x2, 0x2, 0x3e9, - 0x3e3, 0x3, 0x2, 0x2, 0x2, 0x3ea, 0x5d, 0x3, 0x2, 0x2, 0x2, 0x3eb, 0x3ec, - 0x7, 0x5a, 0x2, 0x2, 0x3ec, 0x3ee, 0x7, 0x6d, 0x2, 0x2, 0x3ed, 0x3ef, - 0x5, 0x2c, 0x17, 0x2, 0x3ee, 0x3ed, 0x3, 0x2, 0x2, 0x2, 0x3ee, 0x3ef, - 0x3, 0x2, 0x2, 0x2, 0x3ef, 0x3f0, 0x3, 0x2, 0x2, 0x2, 0x3f0, 0x3f2, - 0x5, 0x74, 0x3b, 0x2, 0x3f1, 0x3f3, 0x9, 0x7, 0x2, 0x2, 0x3f2, 0x3f1, - 0x3, 0x2, 0x2, 0x2, 0x3f2, 0x3f3, 0x3, 0x2, 0x2, 0x2, 0x3f3, 0x5f, 0x3, - 0x2, 0x2, 0x2, 0x3f4, 0x3f5, 0x7, 0x75, 0x2, 0x2, 0x3f5, 0x3f6, 0x7, - 0x97, 0x2, 0x2, 0x3f6, 0x3f8, 0x5, 0xba, 0x5e, 0x2, 0x3f7, 0x3f9, 0x5, - 0x2c, 0x17, 0x2, 0x3f8, 0x3f7, 0x3, 0x2, 0x2, 0x2, 0x3f8, 0x3f9, 0x3, - 0x2, 0x2, 0x2, 0x3f9, 0x3fb, 0x3, 0x2, 0x2, 0x2, 0x3fa, 0x3fc, 0x5, - 0x10, 0x9, 0x2, 0x3fb, 0x3fa, 0x3, 0x2, 0x2, 0x2, 0x3fb, 0x3fc, 0x3, - 0x2, 0x2, 0x2, 0x3fc, 0x3fe, 0x3, 0x2, 0x2, 0x2, 0x3fd, 0x3ff, 0x7, - 0x3c, 0x2, 0x2, 0x3fe, 0x3fd, 0x3, 0x2, 0x2, 0x2, 0x3fe, 0x3ff, 0x3, - 0x2, 0x2, 0x2, 0x3ff, 0x401, 0x3, 0x2, 0x2, 0x2, 0x400, 0x402, 0x7, - 0x25, 0x2, 0x2, 0x401, 0x400, 0x3, 0x2, 0x2, 0x2, 0x401, 0x402, 0x3, - 0x2, 0x2, 0x2, 0x402, 0x61, 0x3, 0x2, 0x2, 0x2, 0x403, 0x404, 0x7, 0x82, - 0x2, 0x2, 0x404, 0x405, 0x7, 0x97, 0x2, 0x2, 0x405, 0x406, 0x5, 0xba, - 0x5e, 0x2, 0x406, 0x407, 0x7, 0x9f, 0x2, 0x2, 0x407, 0x40f, 0x5, 0xba, - 0x5e, 0x2, 0x408, 0x409, 0x7, 0xc2, 0x2, 0x2, 0x409, 0x40a, 0x5, 0xba, - 0x5e, 0x2, 0x40a, 0x40b, 0x7, 0x9f, 0x2, 0x2, 0x40b, 0x40c, 0x5, 0xba, - 0x5e, 0x2, 0x40c, 0x40e, 0x3, 0x2, 0x2, 0x2, 0x40d, 0x408, 0x3, 0x2, - 0x2, 0x2, 0x40e, 0x411, 0x3, 0x2, 0x2, 0x2, 0x40f, 0x40d, 0x3, 0x2, - 0x2, 0x2, 0x40f, 0x410, 0x3, 0x2, 0x2, 0x2, 0x410, 0x413, 0x3, 0x2, - 0x2, 0x2, 0x411, 0x40f, 0x3, 0x2, 0x2, 0x2, 0x412, 0x414, 0x5, 0x2c, - 0x17, 0x2, 0x413, 0x412, 0x3, 0x2, 0x2, 0x2, 0x413, 0x414, 0x3, 0x2, - 0x2, 0x2, 0x414, 0x63, 0x3, 0x2, 0x2, 0x2, 0x415, 0x41b, 0x5, 0x66, - 0x34, 0x2, 0x416, 0x417, 0x7, 0xa7, 0x2, 0x2, 0x417, 0x418, 0x7, 0x6, - 0x2, 0x2, 0x418, 0x41a, 0x5, 0x66, 0x34, 0x2, 0x419, 0x416, 0x3, 0x2, - 0x2, 0x2, 0x41a, 0x41d, 0x3, 0x2, 0x2, 0x2, 0x41b, 0x419, 0x3, 0x2, - 0x2, 0x2, 0x41b, 0x41c, 0x3, 0x2, 0x2, 0x2, 0x41c, 0x65, 0x3, 0x2, 0x2, - 0x2, 0x41d, 0x41b, 0x3, 0x2, 0x2, 0x2, 0x41e, 0x424, 0x5, 0x68, 0x35, - 0x2, 0x41f, 0x420, 0x7, 0xcd, 0x2, 0x2, 0x420, 0x421, 0x5, 0x64, 0x33, - 0x2, 0x421, 0x422, 0x7, 0xd7, 0x2, 0x2, 0x422, 0x424, 0x3, 0x2, 0x2, - 0x2, 0x423, 0x41e, 0x3, 0x2, 0x2, 0x2, 0x423, 0x41f, 0x3, 0x2, 0x2, - 0x2, 0x424, 0x67, 0x3, 0x2, 0x2, 0x2, 0x425, 0x427, 0x5, 0x6a, 0x36, - 0x2, 0x426, 0x425, 0x3, 0x2, 0x2, 0x2, 0x426, 0x427, 0x3, 0x2, 0x2, - 0x2, 0x427, 0x428, 0x3, 0x2, 0x2, 0x2, 0x428, 0x42a, 0x7, 0x8a, 0x2, - 0x2, 0x429, 0x42b, 0x7, 0x30, 0x2, 0x2, 0x42a, 0x429, 0x3, 0x2, 0x2, - 0x2, 0x42a, 0x42b, 0x3, 0x2, 0x2, 0x2, 0x42b, 0x42d, 0x3, 0x2, 0x2, - 0x2, 0x42c, 0x42e, 0x5, 0x6c, 0x37, 0x2, 0x42d, 0x42c, 0x3, 0x2, 0x2, - 0x2, 0x42d, 0x42e, 0x3, 0x2, 0x2, 0x2, 0x42e, 0x42f, 0x3, 0x2, 0x2, - 0x2, 0x42f, 0x431, 0x5, 0xa6, 0x54, 0x2, 0x430, 0x432, 0x5, 0x6e, 0x38, - 0x2, 0x431, 0x430, 0x3, 0x2, 0x2, 0x2, 0x431, 0x432, 0x3, 0x2, 0x2, - 0x2, 0x432, 0x434, 0x3, 0x2, 0x2, 0x2, 0x433, 0x435, 0x5, 0x70, 0x39, - 0x2, 0x434, 0x433, 0x3, 0x2, 0x2, 0x2, 0x434, 0x435, 0x3, 0x2, 0x2, - 0x2, 0x435, 0x437, 0x3, 0x2, 0x2, 0x2, 0x436, 0x438, 0x5, 0x72, 0x3a, - 0x2, 0x437, 0x436, 0x3, 0x2, 0x2, 0x2, 0x437, 0x438, 0x3, 0x2, 0x2, - 0x2, 0x438, 0x43a, 0x3, 0x2, 0x2, 0x2, 0x439, 0x43b, 0x5, 0x74, 0x3b, - 0x2, 0x43a, 0x439, 0x3, 0x2, 0x2, 0x2, 0x43a, 0x43b, 0x3, 0x2, 0x2, - 0x2, 0x43b, 0x43d, 0x3, 0x2, 0x2, 0x2, 0x43c, 0x43e, 0x5, 0x76, 0x3c, - 0x2, 0x43d, 0x43c, 0x3, 0x2, 0x2, 0x2, 0x43d, 0x43e, 0x3, 0x2, 0x2, - 0x2, 0x43e, 0x441, 0x3, 0x2, 0x2, 0x2, 0x43f, 0x440, 0x7, 0xb3, 0x2, - 0x2, 0x440, 0x442, 0x9, 0x8, 0x2, 0x2, 0x441, 0x43f, 0x3, 0x2, 0x2, - 0x2, 0x441, 0x442, 0x3, 0x2, 0x2, 0x2, 0x442, 0x445, 0x3, 0x2, 0x2, - 0x2, 0x443, 0x444, 0x7, 0xb3, 0x2, 0x2, 0x444, 0x446, 0x7, 0xa1, 0x2, - 0x2, 0x445, 0x443, 0x3, 0x2, 0x2, 0x2, 0x445, 0x446, 0x3, 0x2, 0x2, - 0x2, 0x446, 0x448, 0x3, 0x2, 0x2, 0x2, 0x447, 0x449, 0x5, 0x78, 0x3d, + 0xcc, 0xce, 0xd0, 0xd2, 0xd4, 0xd6, 0xd8, 0xda, 0x2, 0x1d, 0x8, 0x2, + 0x5, 0x5, 0x19, 0x19, 0x1c, 0x1c, 0x26, 0x26, 0x65, 0x65, 0xa7, 0xa7, + 0x4, 0x2, 0x10, 0x10, 0x1e, 0x1e, 0x5, 0x2, 0x5, 0x5, 0x26, 0x26, 0x65, + 0x65, 0x4, 0x2, 0x29, 0x29, 0x2b, 0x2b, 0x4, 0x2, 0x2c, 0x2c, 0x32, + 0x32, 0x5, 0x2, 0xf, 0xf, 0x96, 0x96, 0x9c, 0x9c, 0x4, 0x2, 0x20, 0x20, + 0x89, 0x89, 0x4, 0x2, 0x52, 0x52, 0x5e, 0x5e, 0x4, 0x2, 0x45, 0x45, + 0x63, 0x63, 0x5, 0x2, 0x6, 0x6, 0xa, 0xa, 0xe, 0xe, 0x6, 0x2, 0x6, 0x6, + 0x9, 0xa, 0xe, 0xe, 0x8d, 0x8d, 0x4, 0x2, 0x5e, 0x5e, 0x88, 0x88, 0x4, + 0x2, 0x6, 0x6, 0xa, 0xa, 0x4, 0x2, 0x74, 0x74, 0xc4, 0xc4, 0x4, 0x2, + 0xd, 0xd, 0x29, 0x2a, 0x4, 0x2, 0x3d, 0x3d, 0x5b, 0x5b, 0x4, 0x2, 0x42, + 0x42, 0x4e, 0x4e, 0x3, 0x2, 0x93, 0x94, 0x5, 0x2, 0x12, 0x12, 0x5d, + 0x5d, 0xa4, 0xa4, 0x5, 0x2, 0xc0, 0xc0, 0xd2, 0xd2, 0xdb, 0xdb, 0x4, + 0x2, 0xc5, 0xc6, 0xd3, 0xd3, 0x4, 0x2, 0x4d, 0x4d, 0x60, 0x60, 0x3, + 0x2, 0xbb, 0xbc, 0x4, 0x2, 0xc6, 0xc6, 0xd3, 0xd3, 0xa, 0x2, 0x24, 0x24, + 0x4a, 0x4a, 0x6a, 0x6a, 0x6c, 0x6c, 0x80, 0x80, 0x8b, 0x8b, 0xb2, 0xb2, + 0xb6, 0xb6, 0xe, 0x2, 0x4, 0x23, 0x25, 0x49, 0x4b, 0x4f, 0x51, 0x69, + 0x6b, 0x6b, 0x6d, 0x6e, 0x70, 0x71, 0x73, 0x7e, 0x81, 0x8a, 0x8c, 0xb1, + 0xb3, 0xb5, 0xb7, 0xb8, 0x6, 0x2, 0x23, 0x23, 0x3d, 0x3d, 0x4b, 0x4b, + 0x59, 0x59, 0x2, 0x88a, 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, 0x40c, 0x3, 0x2, 0x2, 0x2, 0x5a, 0x410, 0x3, 0x2, 0x2, + 0x2, 0x5c, 0x41f, 0x3, 0x2, 0x2, 0x2, 0x5e, 0x433, 0x3, 0x2, 0x2, 0x2, + 0x60, 0x435, 0x3, 0x2, 0x2, 0x2, 0x62, 0x43e, 0x3, 0x2, 0x2, 0x2, 0x64, + 0x44d, 0x3, 0x2, 0x2, 0x2, 0x66, 0x45f, 0x3, 0x2, 0x2, 0x2, 0x68, 0x46d, + 0x3, 0x2, 0x2, 0x2, 0x6a, 0x47b, 0x3, 0x2, 0x2, 0x2, 0x6c, 0x47e, 0x3, + 0x2, 0x2, 0x2, 0x6e, 0x4ae, 0x3, 0x2, 0x2, 0x2, 0x70, 0x4b1, 0x3, 0x2, + 0x2, 0x2, 0x72, 0x4b7, 0x3, 0x2, 0x2, 0x2, 0x74, 0x4bb, 0x3, 0x2, 0x2, + 0x2, 0x76, 0x4c1, 0x3, 0x2, 0x2, 0x2, 0x78, 0x4c4, 0x3, 0x2, 0x2, 0x2, + 0x7a, 0x4c7, 0x3, 0x2, 0x2, 0x2, 0x7c, 0x4d1, 0x3, 0x2, 0x2, 0x2, 0x7e, + 0x4d4, 0x3, 0x2, 0x2, 0x2, 0x80, 0x4d8, 0x3, 0x2, 0x2, 0x2, 0x82, 0x4dc, + 0x3, 0x2, 0x2, 0x2, 0x84, 0x4e1, 0x3, 0x2, 0x2, 0x2, 0x86, 0x4e7, 0x3, + 0x2, 0x2, 0x2, 0x88, 0x4f6, 0x3, 0x2, 0x2, 0x2, 0x8a, 0x537, 0x3, 0x2, + 0x2, 0x2, 0x8c, 0x53f, 0x3, 0x2, 0x2, 0x2, 0x8e, 0x54a, 0x3, 0x2, 0x2, + 0x2, 0x90, 0x54c, 0x3, 0x2, 0x2, 0x2, 0x92, 0x552, 0x3, 0x2, 0x2, 0x2, + 0x94, 0x557, 0x3, 0x2, 0x2, 0x2, 0x96, 0x55f, 0x3, 0x2, 0x2, 0x2, 0x98, + 0x56b, 0x3, 0x2, 0x2, 0x2, 0x9a, 0x570, 0x3, 0x2, 0x2, 0x2, 0x9c, 0x578, + 0x3, 0x2, 0x2, 0x2, 0x9e, 0x57c, 0x3, 0x2, 0x2, 0x2, 0xa0, 0x5a9, 0x3, + 0x2, 0x2, 0x2, 0xa2, 0x5cd, 0x3, 0x2, 0x2, 0x2, 0xa4, 0x5cf, 0x3, 0x2, + 0x2, 0x2, 0xa6, 0x5de, 0x3, 0x2, 0x2, 0x2, 0xa8, 0x5e1, 0x3, 0x2, 0x2, + 0x2, 0xaa, 0x619, 0x3, 0x2, 0x2, 0x2, 0xac, 0x61b, 0x3, 0x2, 0x2, 0x2, + 0xae, 0x62e, 0x3, 0x2, 0x2, 0x2, 0xb0, 0x69b, 0x3, 0x2, 0x2, 0x2, 0xb2, + 0x6e7, 0x3, 0x2, 0x2, 0x2, 0xb4, 0x6f1, 0x3, 0x2, 0x2, 0x2, 0xb6, 0x706, + 0x3, 0x2, 0x2, 0x2, 0xb8, 0x70e, 0x3, 0x2, 0x2, 0x2, 0xba, 0x712, 0x3, + 0x2, 0x2, 0x2, 0xbc, 0x71e, 0x3, 0x2, 0x2, 0x2, 0xbe, 0x72b, 0x3, 0x2, + 0x2, 0x2, 0xc0, 0x735, 0x3, 0x2, 0x2, 0x2, 0xc2, 0x739, 0x3, 0x2, 0x2, + 0x2, 0xc4, 0x744, 0x3, 0x2, 0x2, 0x2, 0xc6, 0x746, 0x3, 0x2, 0x2, 0x2, + 0xc8, 0x750, 0x3, 0x2, 0x2, 0x2, 0xca, 0x753, 0x3, 0x2, 0x2, 0x2, 0xcc, + 0x760, 0x3, 0x2, 0x2, 0x2, 0xce, 0x762, 0x3, 0x2, 0x2, 0x2, 0xd0, 0x764, + 0x3, 0x2, 0x2, 0x2, 0xd2, 0x766, 0x3, 0x2, 0x2, 0x2, 0xd4, 0x76a, 0x3, + 0x2, 0x2, 0x2, 0xd6, 0x76f, 0x3, 0x2, 0x2, 0x2, 0xd8, 0x773, 0x3, 0x2, + 0x2, 0x2, 0xda, 0x775, 0x3, 0x2, 0x2, 0x2, 0xdc, 0xe0, 0x5, 0x4, 0x3, + 0x2, 0xdd, 0xde, 0x7, 0x55, 0x2, 0x2, 0xde, 0xdf, 0x7, 0x7a, 0x2, 0x2, + 0xdf, 0xe1, 0x7, 0xbe, 0x2, 0x2, 0xe0, 0xdd, 0x3, 0x2, 0x2, 0x2, 0xe0, + 0xe1, 0x3, 0x2, 0x2, 0x2, 0xe1, 0xe4, 0x3, 0x2, 0x2, 0x2, 0xe2, 0xe3, + 0x7, 0x40, 0x2, 0x2, 0xe3, 0xe5, 0x5, 0xd8, 0x6d, 0x2, 0xe4, 0xe2, 0x3, + 0x2, 0x2, 0x2, 0xe4, 0xe5, 0x3, 0x2, 0x2, 0x2, 0xe5, 0xe7, 0x3, 0x2, + 0x2, 0x2, 0xe6, 0xe8, 0x7, 0xda, 0x2, 0x2, 0xe7, 0xe6, 0x3, 0x2, 0x2, + 0x2, 0xe7, 0xe8, 0x3, 0x2, 0x2, 0x2, 0xe8, 0xeb, 0x3, 0x2, 0x2, 0x2, + 0xe9, 0xeb, 0x5, 0x5a, 0x2e, 0x2, 0xea, 0xdc, 0x3, 0x2, 0x2, 0x2, 0xea, + 0xe9, 0x3, 0x2, 0x2, 0x2, 0xeb, 0x3, 0x3, 0x2, 0x2, 0x2, 0xec, 0xff, + 0x5, 0x6, 0x4, 0x2, 0xed, 0xff, 0x5, 0x12, 0xa, 0x2, 0xee, 0xff, 0x5, + 0x14, 0xb, 0x2, 0xef, 0xff, 0x5, 0x16, 0xc, 0x2, 0xf0, 0xff, 0x5, 0x52, + 0x2a, 0x2, 0xf1, 0xff, 0x5, 0x54, 0x2b, 0x2, 0xf2, 0xff, 0x5, 0x56, + 0x2c, 0x2, 0xf3, 0xff, 0x5, 0x58, 0x2d, 0x2, 0xf4, 0xff, 0x5, 0x60, + 0x31, 0x2, 0xf5, 0xff, 0x5, 0x62, 0x32, 0x2, 0xf6, 0xff, 0x5, 0x64, + 0x33, 0x2, 0xf7, 0xff, 0x5, 0x68, 0x35, 0x2, 0xf8, 0xff, 0x5, 0x9e, + 0x50, 0x2, 0xf9, 0xff, 0x5, 0xa0, 0x51, 0x2, 0xfa, 0xff, 0x5, 0xa2, + 0x52, 0x2, 0xfb, 0xff, 0x5, 0xa4, 0x53, 0x2, 0xfc, 0xff, 0x5, 0xa6, + 0x54, 0x2, 0xfd, 0xff, 0x5, 0xa8, 0x55, 0x2, 0xfe, 0xec, 0x3, 0x2, 0x2, + 0x2, 0xfe, 0xed, 0x3, 0x2, 0x2, 0x2, 0xfe, 0xee, 0x3, 0x2, 0x2, 0x2, + 0xfe, 0xef, 0x3, 0x2, 0x2, 0x2, 0xfe, 0xf0, 0x3, 0x2, 0x2, 0x2, 0xfe, + 0xf1, 0x3, 0x2, 0x2, 0x2, 0xfe, 0xf2, 0x3, 0x2, 0x2, 0x2, 0xfe, 0xf3, + 0x3, 0x2, 0x2, 0x2, 0xfe, 0xf4, 0x3, 0x2, 0x2, 0x2, 0xfe, 0xf5, 0x3, + 0x2, 0x2, 0x2, 0xfe, 0xf6, 0x3, 0x2, 0x2, 0x2, 0xfe, 0xf7, 0x3, 0x2, + 0x2, 0x2, 0xfe, 0xf8, 0x3, 0x2, 0x2, 0x2, 0xfe, 0xf9, 0x3, 0x2, 0x2, + 0x2, 0xfe, 0xfa, 0x3, 0x2, 0x2, 0x2, 0xfe, 0xfb, 0x3, 0x2, 0x2, 0x2, + 0xfe, 0xfc, 0x3, 0x2, 0x2, 0x2, 0xfe, 0xfd, 0x3, 0x2, 0x2, 0x2, 0xff, + 0x5, 0x3, 0x2, 0x2, 0x2, 0x100, 0x101, 0x7, 0x7, 0x2, 0x2, 0x101, 0x102, + 0x7, 0x99, 0x2, 0x2, 0x102, 0x104, 0x5, 0xc0, 0x61, 0x2, 0x103, 0x105, + 0x5, 0x2c, 0x17, 0x2, 0x104, 0x103, 0x3, 0x2, 0x2, 0x2, 0x104, 0x105, + 0x3, 0x2, 0x2, 0x2, 0x105, 0x106, 0x3, 0x2, 0x2, 0x2, 0x106, 0x10b, + 0x5, 0x8, 0x5, 0x2, 0x107, 0x108, 0x7, 0xc4, 0x2, 0x2, 0x108, 0x10a, + 0x5, 0x8, 0x5, 0x2, 0x109, 0x107, 0x3, 0x2, 0x2, 0x2, 0x10a, 0x10d, + 0x3, 0x2, 0x2, 0x2, 0x10b, 0x109, 0x3, 0x2, 0x2, 0x2, 0x10b, 0x10c, + 0x3, 0x2, 0x2, 0x2, 0x10c, 0x7, 0x3, 0x2, 0x2, 0x2, 0x10d, 0x10b, 0x3, + 0x2, 0x2, 0x2, 0x10e, 0x10f, 0x7, 0x3, 0x2, 0x2, 0x10f, 0x113, 0x7, + 0x1b, 0x2, 0x2, 0x110, 0x111, 0x7, 0x4c, 0x2, 0x2, 0x111, 0x112, 0x7, + 0x71, 0x2, 0x2, 0x112, 0x114, 0x7, 0x37, 0x2, 0x2, 0x113, 0x110, 0x3, + 0x2, 0x2, 0x2, 0x113, 0x114, 0x3, 0x2, 0x2, 0x2, 0x114, 0x115, 0x3, + 0x2, 0x2, 0x2, 0x115, 0x118, 0x5, 0x44, 0x23, 0x2, 0x116, 0x117, 0x7, + 0x4, 0x2, 0x2, 0x117, 0x119, 0x5, 0xba, 0x5e, 0x2, 0x118, 0x116, 0x3, + 0x2, 0x2, 0x2, 0x118, 0x119, 0x3, 0x2, 0x2, 0x2, 0x119, 0x1e5, 0x3, + 0x2, 0x2, 0x2, 0x11a, 0x11b, 0x7, 0x3, 0x2, 0x2, 0x11b, 0x11f, 0x7, + 0x4f, 0x2, 0x2, 0x11c, 0x11d, 0x7, 0x4c, 0x2, 0x2, 0x11d, 0x11e, 0x7, + 0x71, 0x2, 0x2, 0x11e, 0x120, 0x7, 0x37, 0x2, 0x2, 0x11f, 0x11c, 0x3, + 0x2, 0x2, 0x2, 0x11f, 0x120, 0x3, 0x2, 0x2, 0x2, 0x120, 0x121, 0x3, + 0x2, 0x2, 0x2, 0x121, 0x124, 0x5, 0x48, 0x25, 0x2, 0x122, 0x123, 0x7, + 0x4, 0x2, 0x2, 0x123, 0x125, 0x5, 0xba, 0x5e, 0x2, 0x124, 0x122, 0x3, + 0x2, 0x2, 0x2, 0x124, 0x125, 0x3, 0x2, 0x2, 0x2, 0x125, 0x1e5, 0x3, + 0x2, 0x2, 0x2, 0x126, 0x127, 0x7, 0x3, 0x2, 0x2, 0x127, 0x12b, 0x7, + 0x7f, 0x2, 0x2, 0x128, 0x129, 0x7, 0x4c, 0x2, 0x2, 0x129, 0x12a, 0x7, + 0x71, 0x2, 0x2, 0x12a, 0x12c, 0x7, 0x37, 0x2, 0x2, 0x12b, 0x128, 0x3, + 0x2, 0x2, 0x2, 0x12b, 0x12c, 0x3, 0x2, 0x2, 0x2, 0x12c, 0x12d, 0x3, + 0x2, 0x2, 0x2, 0x12d, 0x130, 0x5, 0x4a, 0x26, 0x2, 0x12e, 0x12f, 0x7, + 0x4, 0x2, 0x2, 0x12f, 0x131, 0x5, 0xba, 0x5e, 0x2, 0x130, 0x12e, 0x3, + 0x2, 0x2, 0x2, 0x130, 0x131, 0x3, 0x2, 0x2, 0x2, 0x131, 0x1e5, 0x3, + 0x2, 0x2, 0x2, 0x132, 0x133, 0x7, 0x10, 0x2, 0x2, 0x133, 0x136, 0x5, + 0x10, 0x9, 0x2, 0x134, 0x135, 0x7, 0x42, 0x2, 0x2, 0x135, 0x137, 0x5, + 0xc0, 0x61, 0x2, 0x136, 0x134, 0x3, 0x2, 0x2, 0x2, 0x136, 0x137, 0x3, + 0x2, 0x2, 0x2, 0x137, 0x1e5, 0x3, 0x2, 0x2, 0x2, 0x138, 0x139, 0x7, + 0x17, 0x2, 0x2, 0x139, 0x13c, 0x7, 0x1b, 0x2, 0x2, 0x13a, 0x13b, 0x7, + 0x4c, 0x2, 0x2, 0x13b, 0x13d, 0x7, 0x37, 0x2, 0x2, 0x13c, 0x13a, 0x3, + 0x2, 0x2, 0x2, 0x13c, 0x13d, 0x3, 0x2, 0x2, 0x2, 0x13d, 0x13e, 0x3, + 0x2, 0x2, 0x2, 0x13e, 0x141, 0x5, 0xba, 0x5e, 0x2, 0x13f, 0x140, 0x7, + 0x4e, 0x2, 0x2, 0x140, 0x142, 0x5, 0x10, 0x9, 0x2, 0x141, 0x13f, 0x3, + 0x2, 0x2, 0x2, 0x141, 0x142, 0x3, 0x2, 0x2, 0x2, 0x142, 0x1e5, 0x3, + 0x2, 0x2, 0x2, 0x143, 0x144, 0x7, 0x17, 0x2, 0x2, 0x144, 0x147, 0x7, + 0x4f, 0x2, 0x2, 0x145, 0x146, 0x7, 0x4c, 0x2, 0x2, 0x146, 0x148, 0x7, + 0x37, 0x2, 0x2, 0x147, 0x145, 0x3, 0x2, 0x2, 0x2, 0x147, 0x148, 0x3, + 0x2, 0x2, 0x2, 0x148, 0x149, 0x3, 0x2, 0x2, 0x2, 0x149, 0x14c, 0x5, + 0xba, 0x5e, 0x2, 0x14a, 0x14b, 0x7, 0x4e, 0x2, 0x2, 0x14b, 0x14d, 0x5, + 0x10, 0x9, 0x2, 0x14c, 0x14a, 0x3, 0x2, 0x2, 0x2, 0x14c, 0x14d, 0x3, + 0x2, 0x2, 0x2, 0x14d, 0x1e5, 0x3, 0x2, 0x2, 0x2, 0x14e, 0x14f, 0x7, + 0x17, 0x2, 0x2, 0x14f, 0x152, 0x7, 0x7f, 0x2, 0x2, 0x150, 0x151, 0x7, + 0x4c, 0x2, 0x2, 0x151, 0x153, 0x7, 0x37, 0x2, 0x2, 0x152, 0x150, 0x3, + 0x2, 0x2, 0x2, 0x152, 0x153, 0x3, 0x2, 0x2, 0x2, 0x153, 0x154, 0x3, + 0x2, 0x2, 0x2, 0x154, 0x157, 0x5, 0xba, 0x5e, 0x2, 0x155, 0x156, 0x7, + 0x4e, 0x2, 0x2, 0x156, 0x158, 0x5, 0x10, 0x9, 0x2, 0x157, 0x155, 0x3, + 0x2, 0x2, 0x2, 0x157, 0x158, 0x3, 0x2, 0x2, 0x2, 0x158, 0x1e5, 0x3, + 0x2, 0x2, 0x2, 0x159, 0x15a, 0x7, 0x1c, 0x2, 0x2, 0x15a, 0x15d, 0x7, + 0x1b, 0x2, 0x2, 0x15b, 0x15c, 0x7, 0x4c, 0x2, 0x2, 0x15c, 0x15e, 0x7, + 0x37, 0x2, 0x2, 0x15d, 0x15b, 0x3, 0x2, 0x2, 0x2, 0x15d, 0x15e, 0x3, + 0x2, 0x2, 0x2, 0x15e, 0x15f, 0x3, 0x2, 0x2, 0x2, 0x15f, 0x160, 0x5, + 0xba, 0x5e, 0x2, 0x160, 0x161, 0x7, 0xbe, 0x2, 0x2, 0x161, 0x1e5, 0x3, + 0x2, 0x2, 0x2, 0x162, 0x163, 0x7, 0x28, 0x2, 0x2, 0x163, 0x164, 0x7, + 0xb4, 0x2, 0x2, 0x164, 0x1e5, 0x5, 0xb0, 0x59, 0x2, 0x165, 0x166, 0x7, + 0x2c, 0x2, 0x2, 0x166, 0x1e5, 0x5, 0x10, 0x9, 0x2, 0x167, 0x168, 0x7, + 0x32, 0x2, 0x2, 0x168, 0x16b, 0x7, 0x1b, 0x2, 0x2, 0x169, 0x16a, 0x7, + 0x4c, 0x2, 0x2, 0x16a, 0x16c, 0x7, 0x37, 0x2, 0x2, 0x16b, 0x169, 0x3, + 0x2, 0x2, 0x2, 0x16b, 0x16c, 0x3, 0x2, 0x2, 0x2, 0x16c, 0x16d, 0x3, + 0x2, 0x2, 0x2, 0x16d, 0x1e5, 0x5, 0xba, 0x5e, 0x2, 0x16e, 0x16f, 0x7, + 0x32, 0x2, 0x2, 0x16f, 0x172, 0x7, 0x4f, 0x2, 0x2, 0x170, 0x171, 0x7, + 0x4c, 0x2, 0x2, 0x171, 0x173, 0x7, 0x37, 0x2, 0x2, 0x172, 0x170, 0x3, + 0x2, 0x2, 0x2, 0x172, 0x173, 0x3, 0x2, 0x2, 0x2, 0x173, 0x174, 0x3, + 0x2, 0x2, 0x2, 0x174, 0x1e5, 0x5, 0xba, 0x5e, 0x2, 0x175, 0x176, 0x7, + 0x32, 0x2, 0x2, 0x176, 0x179, 0x7, 0x7f, 0x2, 0x2, 0x177, 0x178, 0x7, + 0x4c, 0x2, 0x2, 0x178, 0x17a, 0x7, 0x37, 0x2, 0x2, 0x179, 0x177, 0x3, + 0x2, 0x2, 0x2, 0x179, 0x17a, 0x3, 0x2, 0x2, 0x2, 0x17a, 0x17b, 0x3, + 0x2, 0x2, 0x2, 0x17b, 0x1e5, 0x5, 0xba, 0x5e, 0x2, 0x17c, 0x17d, 0x7, + 0x32, 0x2, 0x2, 0x17d, 0x1e5, 0x5, 0x10, 0x9, 0x2, 0x17e, 0x180, 0x7, + 0x41, 0x2, 0x2, 0x17f, 0x181, 0x5, 0x10, 0x9, 0x2, 0x180, 0x17f, 0x3, + 0x2, 0x2, 0x2, 0x180, 0x181, 0x3, 0x2, 0x2, 0x2, 0x181, 0x1e5, 0x3, + 0x2, 0x2, 0x2, 0x182, 0x183, 0x7, 0x66, 0x2, 0x2, 0x183, 0x186, 0x7, + 0x4f, 0x2, 0x2, 0x184, 0x185, 0x7, 0x4c, 0x2, 0x2, 0x185, 0x187, 0x7, + 0x37, 0x2, 0x2, 0x186, 0x184, 0x3, 0x2, 0x2, 0x2, 0x186, 0x187, 0x3, + 0x2, 0x2, 0x2, 0x187, 0x188, 0x3, 0x2, 0x2, 0x2, 0x188, 0x18b, 0x5, + 0xba, 0x5e, 0x2, 0x189, 0x18a, 0x7, 0x4e, 0x2, 0x2, 0x18a, 0x18c, 0x5, + 0x10, 0x9, 0x2, 0x18b, 0x189, 0x3, 0x2, 0x2, 0x2, 0x18b, 0x18c, 0x3, + 0x2, 0x2, 0x2, 0x18c, 0x1e5, 0x3, 0x2, 0x2, 0x2, 0x18d, 0x18e, 0x7, + 0x66, 0x2, 0x2, 0x18e, 0x191, 0x7, 0x7f, 0x2, 0x2, 0x18f, 0x190, 0x7, + 0x4c, 0x2, 0x2, 0x190, 0x192, 0x7, 0x37, 0x2, 0x2, 0x191, 0x18f, 0x3, + 0x2, 0x2, 0x2, 0x191, 0x192, 0x3, 0x2, 0x2, 0x2, 0x192, 0x193, 0x3, + 0x2, 0x2, 0x2, 0x193, 0x196, 0x5, 0xba, 0x5e, 0x2, 0x194, 0x195, 0x7, + 0x4e, 0x2, 0x2, 0x195, 0x197, 0x5, 0x10, 0x9, 0x2, 0x196, 0x194, 0x3, + 0x2, 0x2, 0x2, 0x196, 0x197, 0x3, 0x2, 0x2, 0x2, 0x197, 0x1e5, 0x3, + 0x2, 0x2, 0x2, 0x198, 0x199, 0x7, 0x6b, 0x2, 0x2, 0x199, 0x19c, 0x7, + 0x1b, 0x2, 0x2, 0x19a, 0x19b, 0x7, 0x4c, 0x2, 0x2, 0x19b, 0x19d, 0x7, + 0x37, 0x2, 0x2, 0x19c, 0x19a, 0x3, 0x2, 0x2, 0x2, 0x19c, 0x19d, 0x3, + 0x2, 0x2, 0x2, 0x19d, 0x19e, 0x3, 0x2, 0x2, 0x2, 0x19e, 0x19f, 0x5, + 0xba, 0x5e, 0x2, 0x19f, 0x1a0, 0x5, 0x4c, 0x27, 0x2, 0x1a0, 0x1e5, 0x3, + 0x2, 0x2, 0x2, 0x1a1, 0x1a2, 0x7, 0x6b, 0x2, 0x2, 0x1a2, 0x1a5, 0x7, + 0x1b, 0x2, 0x2, 0x1a3, 0x1a4, 0x7, 0x4c, 0x2, 0x2, 0x1a4, 0x1a6, 0x7, + 0x37, 0x2, 0x2, 0x1a5, 0x1a3, 0x3, 0x2, 0x2, 0x2, 0x1a5, 0x1a6, 0x3, + 0x2, 0x2, 0x2, 0x1a6, 0x1a7, 0x3, 0x2, 0x2, 0x2, 0x1a7, 0x1a8, 0x5, + 0xba, 0x5e, 0x2, 0x1a8, 0x1a9, 0x7, 0x1c, 0x2, 0x2, 0x1a9, 0x1aa, 0x7, + 0xbe, 0x2, 0x2, 0x1aa, 0x1e5, 0x3, 0x2, 0x2, 0x2, 0x1ab, 0x1ac, 0x7, + 0x6b, 0x2, 0x2, 0x1ac, 0x1af, 0x7, 0x1b, 0x2, 0x2, 0x1ad, 0x1ae, 0x7, + 0x4c, 0x2, 0x2, 0x1ae, 0x1b0, 0x7, 0x37, 0x2, 0x2, 0x1af, 0x1ad, 0x3, + 0x2, 0x2, 0x2, 0x1af, 0x1b0, 0x3, 0x2, 0x2, 0x2, 0x1b0, 0x1b1, 0x3, + 0x2, 0x2, 0x2, 0x1b1, 0x1b2, 0x5, 0xba, 0x5e, 0x2, 0x1b2, 0x1b3, 0x7, + 0x83, 0x2, 0x2, 0x1b3, 0x1b4, 0x5, 0xe, 0x8, 0x2, 0x1b4, 0x1e5, 0x3, + 0x2, 0x2, 0x2, 0x1b5, 0x1b6, 0x7, 0x6b, 0x2, 0x2, 0x1b6, 0x1b9, 0x7, + 0x1b, 0x2, 0x2, 0x1b7, 0x1b8, 0x7, 0x4c, 0x2, 0x2, 0x1b8, 0x1ba, 0x7, + 0x37, 0x2, 0x2, 0x1b9, 0x1b7, 0x3, 0x2, 0x2, 0x2, 0x1b9, 0x1ba, 0x3, + 0x2, 0x2, 0x2, 0x1ba, 0x1bb, 0x3, 0x2, 0x2, 0x2, 0x1bb, 0x1e5, 0x5, + 0x44, 0x23, 0x2, 0x1bc, 0x1bd, 0x7, 0x6b, 0x2, 0x2, 0x1bd, 0x1be, 0x7, + 0x78, 0x2, 0x2, 0x1be, 0x1bf, 0x7, 0x13, 0x2, 0x2, 0x1bf, 0x1e5, 0x5, + 0xb0, 0x59, 0x2, 0x1c0, 0x1c1, 0x7, 0x6b, 0x2, 0x2, 0x1c1, 0x1e5, 0x5, + 0x3e, 0x20, 0x2, 0x1c2, 0x1c3, 0x7, 0x6d, 0x2, 0x2, 0x1c3, 0x1cd, 0x5, + 0x10, 0x9, 0x2, 0x1c4, 0x1c5, 0x7, 0xa1, 0x2, 0x2, 0x1c5, 0x1c6, 0x7, + 0x2f, 0x2, 0x2, 0x1c6, 0x1ce, 0x7, 0xbe, 0x2, 0x2, 0x1c7, 0x1c8, 0x7, + 0xa1, 0x2, 0x2, 0x1c8, 0x1c9, 0x7, 0xb0, 0x2, 0x2, 0x1c9, 0x1ce, 0x7, + 0xbe, 0x2, 0x2, 0x1ca, 0x1cb, 0x7, 0xa1, 0x2, 0x2, 0x1cb, 0x1cc, 0x7, + 0x99, 0x2, 0x2, 0x1cc, 0x1ce, 0x5, 0xc0, 0x61, 0x2, 0x1cd, 0x1c4, 0x3, + 0x2, 0x2, 0x2, 0x1cd, 0x1c7, 0x3, 0x2, 0x2, 0x2, 0x1cd, 0x1ca, 0x3, + 0x2, 0x2, 0x2, 0x1ce, 0x1e5, 0x3, 0x2, 0x2, 0x2, 0x1cf, 0x1d0, 0x7, + 0x83, 0x2, 0x2, 0x1d0, 0x1e5, 0x7, 0xa7, 0x2, 0x2, 0x1d1, 0x1d2, 0x7, + 0x84, 0x2, 0x2, 0x1d2, 0x1d5, 0x7, 0x1b, 0x2, 0x2, 0x1d3, 0x1d4, 0x7, + 0x4c, 0x2, 0x2, 0x1d4, 0x1d6, 0x7, 0x37, 0x2, 0x2, 0x1d5, 0x1d3, 0x3, + 0x2, 0x2, 0x2, 0x1d5, 0x1d6, 0x3, 0x2, 0x2, 0x2, 0x1d6, 0x1d7, 0x3, + 0x2, 0x2, 0x2, 0x1d7, 0x1d8, 0x5, 0xba, 0x5e, 0x2, 0x1d8, 0x1d9, 0x7, + 0xa1, 0x2, 0x2, 0x1d9, 0x1da, 0x5, 0xba, 0x5e, 0x2, 0x1da, 0x1e5, 0x3, + 0x2, 0x2, 0x2, 0x1db, 0x1dc, 0x7, 0x85, 0x2, 0x2, 0x1dc, 0x1dd, 0x5, + 0x10, 0x9, 0x2, 0x1dd, 0x1de, 0x7, 0x42, 0x2, 0x2, 0x1de, 0x1df, 0x5, + 0xc0, 0x61, 0x2, 0x1df, 0x1e5, 0x3, 0x2, 0x2, 0x2, 0x1e0, 0x1e1, 0x7, + 0xaa, 0x2, 0x2, 0x1e1, 0x1e2, 0x5, 0xa, 0x6, 0x2, 0x1e2, 0x1e3, 0x5, + 0x78, 0x3d, 0x2, 0x1e3, 0x1e5, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x10e, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x11a, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x126, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x132, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x138, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x143, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x14e, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x159, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x162, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x165, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x167, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x16e, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x175, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x17c, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x17e, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x182, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x18d, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x198, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x1a1, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x1ab, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x1b5, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x1bc, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x1c0, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x1c2, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x1cf, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x1d1, 0x3, 0x2, 0x2, 0x2, 0x1e4, 0x1db, 0x3, + 0x2, 0x2, 0x2, 0x1e4, 0x1e0, 0x3, 0x2, 0x2, 0x2, 0x1e5, 0x9, 0x3, 0x2, + 0x2, 0x2, 0x1e6, 0x1eb, 0x5, 0xc, 0x7, 0x2, 0x1e7, 0x1e8, 0x7, 0xc4, + 0x2, 0x2, 0x1e8, 0x1ea, 0x5, 0xc, 0x7, 0x2, 0x1e9, 0x1e7, 0x3, 0x2, + 0x2, 0x2, 0x1ea, 0x1ed, 0x3, 0x2, 0x2, 0x2, 0x1eb, 0x1e9, 0x3, 0x2, + 0x2, 0x2, 0x1eb, 0x1ec, 0x3, 0x2, 0x2, 0x2, 0x1ec, 0xb, 0x3, 0x2, 0x2, + 0x2, 0x1ed, 0x1eb, 0x3, 0x2, 0x2, 0x2, 0x1ee, 0x1ef, 0x5, 0xba, 0x5e, + 0x2, 0x1ef, 0x1f0, 0x7, 0xc9, 0x2, 0x2, 0x1f0, 0x1f1, 0x5, 0xb0, 0x59, + 0x2, 0x1f1, 0xd, 0x3, 0x2, 0x2, 0x2, 0x1f2, 0x1f3, 0x9, 0x2, 0x2, 0x2, + 0x1f3, 0xf, 0x3, 0x2, 0x2, 0x2, 0x1f4, 0x1f5, 0x7, 0x7b, 0x2, 0x2, 0x1f5, + 0x1fa, 0x5, 0xb0, 0x59, 0x2, 0x1f6, 0x1f7, 0x7, 0x7b, 0x2, 0x2, 0x1f7, + 0x1f8, 0x7, 0x4b, 0x2, 0x2, 0x1f8, 0x1fa, 0x7, 0xbe, 0x2, 0x2, 0x1f9, + 0x1f4, 0x3, 0x2, 0x2, 0x2, 0x1f9, 0x1f6, 0x3, 0x2, 0x2, 0x2, 0x1fa, + 0x11, 0x3, 0x2, 0x2, 0x2, 0x1fb, 0x1fc, 0x7, 0x10, 0x2, 0x2, 0x1fc, + 0x1fd, 0x7, 0x2e, 0x2, 0x2, 0x1fd, 0x1ff, 0x5, 0xc0, 0x61, 0x2, 0x1fe, + 0x200, 0x5, 0x2c, 0x17, 0x2, 0x1ff, 0x1fe, 0x3, 0x2, 0x2, 0x2, 0x1ff, + 0x200, 0x3, 0x2, 0x2, 0x2, 0x200, 0x13, 0x3, 0x2, 0x2, 0x2, 0x201, 0x202, + 0x7, 0x16, 0x2, 0x2, 0x202, 0x203, 0x7, 0x99, 0x2, 0x2, 0x203, 0x205, + 0x5, 0xc0, 0x61, 0x2, 0x204, 0x206, 0x5, 0x10, 0x9, 0x2, 0x205, 0x204, + 0x3, 0x2, 0x2, 0x2, 0x205, 0x206, 0x3, 0x2, 0x2, 0x2, 0x206, 0x15, 0x3, + 0x2, 0x2, 0x2, 0x207, 0x208, 0x9, 0x3, 0x2, 0x2, 0x208, 0x20c, 0x7, + 0x21, 0x2, 0x2, 0x209, 0x20a, 0x7, 0x4c, 0x2, 0x2, 0x20a, 0x20b, 0x7, + 0x71, 0x2, 0x2, 0x20b, 0x20d, 0x7, 0x37, 0x2, 0x2, 0x20c, 0x209, 0x3, + 0x2, 0x2, 0x2, 0x20c, 0x20d, 0x3, 0x2, 0x2, 0x2, 0x20d, 0x20e, 0x3, + 0x2, 0x2, 0x2, 0x20e, 0x210, 0x5, 0xc6, 0x64, 0x2, 0x20f, 0x211, 0x5, + 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, 0x2e, 0x2, 0x2, 0x217, 0x218, 0x7, + 0x4c, 0x2, 0x2, 0x218, 0x219, 0x7, 0x71, 0x2, 0x2, 0x219, 0x21b, 0x7, + 0x37, 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, 0x62, 0x2, 0x2, 0x228, 0x22c, 0x7, + 0xaf, 0x2, 0x2, 0x229, 0x22a, 0x7, 0x4c, 0x2, 0x2, 0x22a, 0x22b, 0x7, + 0x71, 0x2, 0x2, 0x22b, 0x22d, 0x7, 0x37, 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, + 0xb5, 0x2, 0x2, 0x236, 0x238, 0x7, 0x9f, 0x2, 0x2, 0x237, 0x239, 0x7, + 0xbc, 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, + 0x65, 0x2, 0x2, 0x246, 0x24a, 0x7, 0xaf, 0x2, 0x2, 0x247, 0x248, 0x7, + 0x4c, 0x2, 0x2, 0x248, 0x249, 0x7, 0x71, 0x2, 0x2, 0x249, 0x24b, 0x7, + 0x37, 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, 0x7c, 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, 0x9b, 0x2, 0x2, 0x261, 0x260, 0x3, + 0x2, 0x2, 0x2, 0x261, 0x262, 0x3, 0x2, 0x2, 0x2, 0x262, 0x263, 0x3, + 0x2, 0x2, 0x2, 0x263, 0x267, 0x7, 0x99, 0x2, 0x2, 0x264, 0x265, 0x7, + 0x4c, 0x2, 0x2, 0x265, 0x266, 0x7, 0x71, 0x2, 0x2, 0x266, 0x268, 0x7, + 0x37, 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, + 0x77, 0x2, 0x2, 0x27b, 0x27d, 0x7, 0x85, 0x2, 0x2, 0x27c, 0x27a, 0x3, + 0x2, 0x2, 0x2, 0x27c, 0x27d, 0x3, 0x2, 0x2, 0x2, 0x27d, 0x27e, 0x3, + 0x2, 0x2, 0x2, 0x27e, 0x282, 0x7, 0xaf, 0x2, 0x2, 0x27f, 0x280, 0x7, + 0x4c, 0x2, 0x2, 0x280, 0x281, 0x7, 0x71, 0x2, 0x2, 0x281, 0x283, 0x7, + 0x37, 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, 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, 0xcf, + 0x2, 0x2, 0x293, 0x298, 0x5, 0x1a, 0xe, 0x2, 0x294, 0x295, 0x7, 0xc4, + 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, 0xd9, + 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, 0x26, 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, 0x39, + 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, 0x49, 0x2, 0x2, 0x2ab, 0x2b3, 0x8, 0xe, + 0x1, 0x2, 0x2ac, 0x2ad, 0x6, 0xe, 0x5, 0x3, 0x2ad, 0x2ae, 0x7, 0x51, + 0x2, 0x2, 0x2ae, 0x2b3, 0x8, 0xe, 0x1, 0x2, 0x2af, 0x2b0, 0x6, 0xe, + 0x6, 0x3, 0x2b0, 0x2b1, 0x7, 0x57, 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, 0x7e, 0x2, 0x2, 0x2d4, 0x2d5, 0x7, 0x59, 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, 0xcf, 0x2, 0x2, 0x2da, 0x2dc, 0x7, 0xd9, 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, 0x92, 0x2, 0x2, + 0x2e1, 0x2e2, 0x7, 0xcf, 0x2, 0x2, 0x2e2, 0x2e3, 0x5, 0xd6, 0x6c, 0x2, + 0x2e3, 0x2e7, 0x7, 0xcf, 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, 0xd9, 0x2, 0x2, 0x2eb, 0x2ec, 0x7, 0xd9, 0x2, 0x2, + 0x2ec, 0x23, 0x3, 0x2, 0x2, 0x2, 0x2ed, 0x2ee, 0x7, 0x5f, 0x2, 0x2, + 0x2ee, 0x2f8, 0x7, 0xcf, 0x2, 0x2, 0x2ef, 0x2f9, 0x7, 0xbc, 0x2, 0x2, + 0x2f0, 0x2f1, 0x7, 0x69, 0x2, 0x2, 0x2f1, 0x2f2, 0x7, 0xbc, 0x2, 0x2, + 0x2f2, 0x2f3, 0x7, 0x67, 0x2, 0x2, 0x2f3, 0x2f9, 0x7, 0xbc, 0x2, 0x2, + 0x2f4, 0x2f5, 0x7, 0x67, 0x2, 0x2, 0x2f5, 0x2f6, 0x7, 0xbc, 0x2, 0x2, + 0x2f6, 0x2f7, 0x7, 0x69, 0x2, 0x2, 0x2f7, 0x2f9, 0x7, 0xbc, 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, 0xd9, 0x2, 0x2, 0x2fb, 0x25, 0x3, 0x2, 0x2, 0x2, + 0x2fc, 0x2fd, 0x7, 0x5c, 0x2, 0x2, 0x2fd, 0x2fe, 0x7, 0xcf, 0x2, 0x2, + 0x2fe, 0x2ff, 0x5, 0xd6, 0x6c, 0x2, 0x2ff, 0x303, 0x7, 0xcf, 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, 0xd9, 0x2, 0x2, + 0x307, 0x308, 0x7, 0xd9, 0x2, 0x2, 0x308, 0x27, 0x3, 0x2, 0x2, 0x2, + 0x309, 0x30a, 0x7, 0x81, 0x2, 0x2, 0x30a, 0x315, 0x7, 0xcf, 0x2, 0x2, + 0x30b, 0x30c, 0x7, 0x69, 0x2, 0x2, 0x30c, 0x30d, 0x5, 0xd6, 0x6c, 0x2, + 0x30d, 0x30e, 0x7, 0x67, 0x2, 0x2, 0x30e, 0x30f, 0x5, 0xd6, 0x6c, 0x2, + 0x30f, 0x316, 0x3, 0x2, 0x2, 0x2, 0x310, 0x311, 0x7, 0x67, 0x2, 0x2, + 0x311, 0x312, 0x5, 0xd6, 0x6c, 0x2, 0x312, 0x313, 0x7, 0x69, 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, 0xd9, 0x2, 0x2, + 0x318, 0x29, 0x3, 0x2, 0x2, 0x2, 0x319, 0x31a, 0x7, 0x90, 0x2, 0x2, + 0x31a, 0x31b, 0x7, 0xcf, 0x2, 0x2, 0x31b, 0x31c, 0x5, 0x9a, 0x4e, 0x2, + 0x31c, 0x31d, 0x7, 0xd9, 0x2, 0x2, 0x31d, 0x2b, 0x3, 0x2, 0x2, 0x2, + 0x31e, 0x31f, 0x7, 0x75, 0x2, 0x2, 0x31f, 0x322, 0x7, 0x18, 0x2, 0x2, + 0x320, 0x323, 0x5, 0xd6, 0x6c, 0x2, 0x321, 0x323, 0x7, 0xbe, 0x2, 0x2, + 0x322, 0x320, 0x3, 0x2, 0x2, 0x2, 0x322, 0x321, 0x3, 0x2, 0x2, 0x2, + 0x323, 0x2d, 0x3, 0x2, 0x2, 0x2, 0x324, 0x325, 0x7, 0xad, 0x2, 0x2, + 0x325, 0x326, 0x7, 0xbe, 0x2, 0x2, 0x326, 0x2f, 0x3, 0x2, 0x2, 0x2, + 0x327, 0x328, 0x7, 0xa1, 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, 0xcf, 0x2, 0x2, 0x32e, 0x333, 0x5, 0x42, 0x22, 0x2, 0x32f, + 0x330, 0x7, 0xc4, 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, 0xd9, 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, + 0x7b, 0x2, 0x2, 0x35d, 0x35e, 0x7, 0x13, 0x2, 0x2, 0x35e, 0x35f, 0x5, + 0xb0, 0x59, 0x2, 0x35f, 0x39, 0x3, 0x2, 0x2, 0x2, 0x360, 0x361, 0x7, + 0x7e, 0x2, 0x2, 0x361, 0x362, 0x7, 0x59, 0x2, 0x2, 0x362, 0x363, 0x5, + 0xb0, 0x59, 0x2, 0x363, 0x3b, 0x3, 0x2, 0x2, 0x2, 0x364, 0x365, 0x7, + 0x8a, 0x2, 0x2, 0x365, 0x366, 0x7, 0x13, 0x2, 0x2, 0x366, 0x367, 0x5, + 0xb0, 0x59, 0x2, 0x367, 0x3d, 0x3, 0x2, 0x2, 0x2, 0x368, 0x369, 0x7, + 0xa7, 0x2, 0x2, 0x369, 0x36e, 0x5, 0x50, 0x29, 0x2, 0x36a, 0x36b, 0x7, + 0xc4, 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, 0x35, + 0x2, 0x2, 0x372, 0x374, 0x7, 0xc9, 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, 0xcf, + 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, 0xd9, 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, 0x1d, 0x2, + 0x2, 0x37f, 0x380, 0x5, 0xd6, 0x6c, 0x2, 0x380, 0x381, 0x7, 0x16, 0x2, + 0x2, 0x381, 0x382, 0x5, 0xb0, 0x59, 0x2, 0x382, 0x388, 0x3, 0x2, 0x2, + 0x2, 0x383, 0x384, 0x7, 0x4f, 0x2, 0x2, 0x384, 0x388, 0x5, 0x48, 0x25, + 0x2, 0x385, 0x386, 0x7, 0x7f, 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, 0x1c, 0x2, + 0x2, 0x38f, 0x391, 0x7, 0xbe, 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, 0xa7, 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, 0x1c, 0x2, + 0x2, 0x39f, 0x3a1, 0x7, 0xbe, 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, 0xa7, 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, 0xa8, 0x2, 0x2, + 0x3b1, 0x3b2, 0x5, 0xaa, 0x56, 0x2, 0x3b2, 0x3b3, 0x7, 0x46, 0x2, 0x2, + 0x3b3, 0x3b4, 0x7, 0xbc, 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, 0x19, 0x2, 0x2, + 0x3b9, 0x3ba, 0x7, 0xcf, 0x2, 0x2, 0x3ba, 0x3bf, 0x5, 0x4e, 0x28, 0x2, + 0x3bb, 0x3bc, 0x7, 0xc4, 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, 0xd9, 0x2, 0x2, 0x3c3, 0x4d, 0x3, 0x2, 0x2, 0x2, + 0x3c4, 0x3ca, 0x5, 0xd6, 0x6c, 0x2, 0x3c5, 0x3c7, 0x7, 0xcf, 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, 0xd9, 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, 0x28, 0x2, 0x2, 0x3ce, + 0x3cf, 0x7, 0xa1, 0x2, 0x2, 0x3cf, 0x3d0, 0x7, 0x2f, 0x2, 0x2, 0x3d0, + 0x3d5, 0x7, 0xbe, 0x2, 0x2, 0x3d1, 0x3d2, 0x7, 0xa1, 0x2, 0x2, 0x3d2, + 0x3d3, 0x7, 0xb0, 0x2, 0x2, 0x3d3, 0x3d5, 0x7, 0xbe, 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, 0x99, 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, 0x21, 0x2, 0x2, 0x3de, 0x3df, + 0x7, 0x4c, 0x2, 0x2, 0x3df, 0x3e1, 0x7, 0x37, 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, 0x2e, 0x2, 0x2, 0x3e8, 0x3ea, + 0x7, 0x9b, 0x2, 0x2, 0x3e9, 0x3e8, 0x3, 0x2, 0x2, 0x2, 0x3e9, 0x3ea, + 0x3, 0x2, 0x2, 0x2, 0x3ea, 0x3eb, 0x3, 0x2, 0x2, 0x2, 0x3eb, 0x3ee, + 0x7, 0x99, 0x2, 0x2, 0x3ec, 0x3ee, 0x7, 0xaf, 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, 0x4c, 0x2, 0x2, 0x3f0, 0x3f2, 0x7, 0x37, 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, 0x70, 0x2, 0x2, 0x3f8, 0x3fa, 0x7, 0x27, 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, + 0x37, 0x2, 0x2, 0x3fe, 0x3ff, 0x7, 0x21, 0x2, 0x2, 0x3ff, 0x40b, 0x5, + 0xc6, 0x64, 0x2, 0x400, 0x407, 0x7, 0x37, 0x2, 0x2, 0x401, 0x408, 0x7, + 0x2e, 0x2, 0x2, 0x402, 0x404, 0x7, 0x9b, 0x2, 0x2, 0x403, 0x402, 0x3, + 0x2, 0x2, 0x2, 0x403, 0x404, 0x3, 0x2, 0x2, 0x2, 0x404, 0x405, 0x3, + 0x2, 0x2, 0x2, 0x405, 0x408, 0x7, 0x99, 0x2, 0x2, 0x406, 0x408, 0x7, + 0xaf, 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, 0x38, + 0x2, 0x2, 0x40d, 0x40e, 0x7, 0x97, 0x2, 0x2, 0x40e, 0x40f, 0x5, 0x4, + 0x3, 0x2, 0x40f, 0x59, 0x3, 0x2, 0x2, 0x2, 0x410, 0x411, 0x7, 0x53, + 0x2, 0x2, 0x411, 0x413, 0x7, 0x55, 0x2, 0x2, 0x412, 0x414, 0x7, 0x99, + 0x2, 0x2, 0x413, 0x412, 0x3, 0x2, 0x2, 0x2, 0x413, 0x414, 0x3, 0x2, + 0x2, 0x2, 0x414, 0x418, 0x3, 0x2, 0x2, 0x2, 0x415, 0x419, 0x5, 0xc0, + 0x61, 0x2, 0x416, 0x417, 0x7, 0x44, 0x2, 0x2, 0x417, 0x419, 0x5, 0xbe, + 0x60, 0x2, 0x418, 0x415, 0x3, 0x2, 0x2, 0x2, 0x418, 0x416, 0x3, 0x2, + 0x2, 0x2, 0x419, 0x41b, 0x3, 0x2, 0x2, 0x2, 0x41a, 0x41c, 0x5, 0x5c, + 0x2f, 0x2, 0x41b, 0x41a, 0x3, 0x2, 0x2, 0x2, 0x41b, 0x41c, 0x3, 0x2, + 0x2, 0x2, 0x41c, 0x41d, 0x3, 0x2, 0x2, 0x2, 0x41d, 0x41e, 0x5, 0x5e, + 0x30, 0x2, 0x41e, 0x5b, 0x3, 0x2, 0x2, 0x2, 0x41f, 0x420, 0x7, 0xcf, + 0x2, 0x2, 0x420, 0x425, 0x5, 0xba, 0x5e, 0x2, 0x421, 0x422, 0x7, 0xc4, + 0x2, 0x2, 0x422, 0x424, 0x5, 0xba, 0x5e, 0x2, 0x423, 0x421, 0x3, 0x2, + 0x2, 0x2, 0x424, 0x427, 0x3, 0x2, 0x2, 0x2, 0x425, 0x423, 0x3, 0x2, + 0x2, 0x2, 0x425, 0x426, 0x3, 0x2, 0x2, 0x2, 0x426, 0x428, 0x3, 0x2, + 0x2, 0x2, 0x427, 0x425, 0x3, 0x2, 0x2, 0x2, 0x428, 0x429, 0x7, 0xd9, + 0x2, 0x2, 0x429, 0x5d, 0x3, 0x2, 0x2, 0x2, 0x42a, 0x42b, 0x7, 0x40, + 0x2, 0x2, 0x42b, 0x434, 0x5, 0xd6, 0x6c, 0x2, 0x42c, 0x434, 0x7, 0xae, + 0x2, 0x2, 0x42d, 0x42f, 0x5, 0x68, 0x35, 0x2, 0x42e, 0x430, 0x7, 0xda, + 0x2, 0x2, 0x42f, 0x42e, 0x3, 0x2, 0x2, 0x2, 0x42f, 0x430, 0x3, 0x2, + 0x2, 0x2, 0x430, 0x431, 0x3, 0x2, 0x2, 0x2, 0x431, 0x432, 0x7, 0x2, + 0x2, 0x3, 0x432, 0x434, 0x3, 0x2, 0x2, 0x2, 0x433, 0x42a, 0x3, 0x2, + 0x2, 0x2, 0x433, 0x42c, 0x3, 0x2, 0x2, 0x2, 0x433, 0x42d, 0x3, 0x2, + 0x2, 0x2, 0x434, 0x5f, 0x3, 0x2, 0x2, 0x2, 0x435, 0x436, 0x7, 0x5a, + 0x2, 0x2, 0x436, 0x438, 0x7, 0x6e, 0x2, 0x2, 0x437, 0x439, 0x5, 0x2c, + 0x17, 0x2, 0x438, 0x437, 0x3, 0x2, 0x2, 0x2, 0x438, 0x439, 0x3, 0x2, + 0x2, 0x2, 0x439, 0x43a, 0x3, 0x2, 0x2, 0x2, 0x43a, 0x43c, 0x5, 0x78, + 0x3d, 0x2, 0x43b, 0x43d, 0x9, 0x7, 0x2, 0x2, 0x43c, 0x43b, 0x3, 0x2, + 0x2, 0x2, 0x43c, 0x43d, 0x3, 0x2, 0x2, 0x2, 0x43d, 0x61, 0x3, 0x2, 0x2, + 0x2, 0x43e, 0x43f, 0x7, 0x76, 0x2, 0x2, 0x43f, 0x440, 0x7, 0x99, 0x2, + 0x2, 0x440, 0x442, 0x5, 0xc0, 0x61, 0x2, 0x441, 0x443, 0x5, 0x2c, 0x17, + 0x2, 0x442, 0x441, 0x3, 0x2, 0x2, 0x2, 0x442, 0x443, 0x3, 0x2, 0x2, + 0x2, 0x443, 0x445, 0x3, 0x2, 0x2, 0x2, 0x444, 0x446, 0x5, 0x10, 0x9, + 0x2, 0x445, 0x444, 0x3, 0x2, 0x2, 0x2, 0x445, 0x446, 0x3, 0x2, 0x2, + 0x2, 0x446, 0x448, 0x3, 0x2, 0x2, 0x2, 0x447, 0x449, 0x7, 0x3c, 0x2, 0x2, 0x448, 0x447, 0x3, 0x2, 0x2, 0x2, 0x448, 0x449, 0x3, 0x2, 0x2, - 0x2, 0x449, 0x44b, 0x3, 0x2, 0x2, 0x2, 0x44a, 0x44c, 0x5, 0x7a, 0x3e, + 0x2, 0x449, 0x44b, 0x3, 0x2, 0x2, 0x2, 0x44a, 0x44c, 0x7, 0x25, 0x2, 0x2, 0x44b, 0x44a, 0x3, 0x2, 0x2, 0x2, 0x44b, 0x44c, 0x3, 0x2, 0x2, - 0x2, 0x44c, 0x44e, 0x3, 0x2, 0x2, 0x2, 0x44d, 0x44f, 0x5, 0x7c, 0x3f, - 0x2, 0x44e, 0x44d, 0x3, 0x2, 0x2, 0x2, 0x44e, 0x44f, 0x3, 0x2, 0x2, - 0x2, 0x44f, 0x451, 0x3, 0x2, 0x2, 0x2, 0x450, 0x452, 0x5, 0x7e, 0x40, - 0x2, 0x451, 0x450, 0x3, 0x2, 0x2, 0x2, 0x451, 0x452, 0x3, 0x2, 0x2, - 0x2, 0x452, 0x454, 0x3, 0x2, 0x2, 0x2, 0x453, 0x455, 0x5, 0x80, 0x41, - 0x2, 0x454, 0x453, 0x3, 0x2, 0x2, 0x2, 0x454, 0x455, 0x3, 0x2, 0x2, - 0x2, 0x455, 0x69, 0x3, 0x2, 0x2, 0x2, 0x456, 0x457, 0x7, 0xb3, 0x2, - 0x2, 0x457, 0x458, 0x5, 0xa6, 0x54, 0x2, 0x458, 0x6b, 0x3, 0x2, 0x2, - 0x2, 0x459, 0x45a, 0x7, 0xa0, 0x2, 0x2, 0x45a, 0x45d, 0x7, 0xba, 0x2, - 0x2, 0x45b, 0x45c, 0x7, 0xb3, 0x2, 0x2, 0x45c, 0x45e, 0x7, 0x9c, 0x2, - 0x2, 0x45d, 0x45b, 0x3, 0x2, 0x2, 0x2, 0x45d, 0x45e, 0x3, 0x2, 0x2, - 0x2, 0x45e, 0x6d, 0x3, 0x2, 0x2, 0x2, 0x45f, 0x460, 0x7, 0x42, 0x2, - 0x2, 0x460, 0x461, 0x5, 0x82, 0x42, 0x2, 0x461, 0x6f, 0x3, 0x2, 0x2, - 0x2, 0x462, 0x464, 0x9, 0x9, 0x2, 0x2, 0x463, 0x462, 0x3, 0x2, 0x2, - 0x2, 0x463, 0x464, 0x3, 0x2, 0x2, 0x2, 0x464, 0x465, 0x3, 0x2, 0x2, - 0x2, 0x465, 0x466, 0x7, 0xb, 0x2, 0x2, 0x466, 0x467, 0x7, 0x58, 0x2, - 0x2, 0x467, 0x468, 0x5, 0xa6, 0x54, 0x2, 0x468, 0x71, 0x3, 0x2, 0x2, - 0x2, 0x469, 0x46a, 0x7, 0x7c, 0x2, 0x2, 0x46a, 0x46b, 0x5, 0xaa, 0x56, - 0x2, 0x46b, 0x73, 0x3, 0x2, 0x2, 0x2, 0x46c, 0x46d, 0x7, 0xb2, 0x2, - 0x2, 0x46d, 0x46e, 0x5, 0xaa, 0x56, 0x2, 0x46e, 0x75, 0x3, 0x2, 0x2, - 0x2, 0x46f, 0x470, 0x7, 0x47, 0x2, 0x2, 0x470, 0x477, 0x7, 0x13, 0x2, - 0x2, 0x471, 0x472, 0x9, 0x8, 0x2, 0x2, 0x472, 0x473, 0x7, 0xcd, 0x2, - 0x2, 0x473, 0x474, 0x5, 0xa6, 0x54, 0x2, 0x474, 0x475, 0x7, 0xd7, 0x2, - 0x2, 0x475, 0x478, 0x3, 0x2, 0x2, 0x2, 0x476, 0x478, 0x5, 0xa6, 0x54, - 0x2, 0x477, 0x471, 0x3, 0x2, 0x2, 0x2, 0x477, 0x476, 0x3, 0x2, 0x2, - 0x2, 0x478, 0x77, 0x3, 0x2, 0x2, 0x2, 0x479, 0x47a, 0x7, 0x48, 0x2, - 0x2, 0x47a, 0x47b, 0x5, 0xaa, 0x56, 0x2, 0x47b, 0x79, 0x3, 0x2, 0x2, - 0x2, 0x47c, 0x47d, 0x7, 0x77, 0x2, 0x2, 0x47d, 0x47e, 0x7, 0x13, 0x2, - 0x2, 0x47e, 0x47f, 0x5, 0x8e, 0x48, 0x2, 0x47f, 0x7b, 0x3, 0x2, 0x2, - 0x2, 0x480, 0x481, 0x7, 0x61, 0x2, 0x2, 0x481, 0x482, 0x5, 0x8c, 0x47, - 0x2, 0x482, 0x483, 0x7, 0x13, 0x2, 0x2, 0x483, 0x484, 0x5, 0xa6, 0x54, - 0x2, 0x484, 0x7d, 0x3, 0x2, 0x2, 0x2, 0x485, 0x486, 0x7, 0x61, 0x2, - 0x2, 0x486, 0x489, 0x5, 0x8c, 0x47, 0x2, 0x487, 0x488, 0x7, 0xb3, 0x2, - 0x2, 0x488, 0x48a, 0x7, 0x9c, 0x2, 0x2, 0x489, 0x487, 0x3, 0x2, 0x2, - 0x2, 0x489, 0x48a, 0x3, 0x2, 0x2, 0x2, 0x48a, 0x7f, 0x3, 0x2, 0x2, 0x2, - 0x48b, 0x48c, 0x7, 0x8e, 0x2, 0x2, 0x48c, 0x48d, 0x5, 0x94, 0x4b, 0x2, - 0x48d, 0x81, 0x3, 0x2, 0x2, 0x2, 0x48e, 0x48f, 0x8, 0x42, 0x1, 0x2, - 0x48f, 0x491, 0x5, 0xb6, 0x5c, 0x2, 0x490, 0x492, 0x7, 0x3c, 0x2, 0x2, - 0x491, 0x490, 0x3, 0x2, 0x2, 0x2, 0x491, 0x492, 0x3, 0x2, 0x2, 0x2, - 0x492, 0x494, 0x3, 0x2, 0x2, 0x2, 0x493, 0x495, 0x5, 0x8a, 0x46, 0x2, - 0x494, 0x493, 0x3, 0x2, 0x2, 0x2, 0x494, 0x495, 0x3, 0x2, 0x2, 0x2, - 0x495, 0x49b, 0x3, 0x2, 0x2, 0x2, 0x496, 0x497, 0x7, 0xcd, 0x2, 0x2, - 0x497, 0x498, 0x5, 0x82, 0x42, 0x2, 0x498, 0x499, 0x7, 0xd7, 0x2, 0x2, - 0x499, 0x49b, 0x3, 0x2, 0x2, 0x2, 0x49a, 0x48e, 0x3, 0x2, 0x2, 0x2, - 0x49a, 0x496, 0x3, 0x2, 0x2, 0x2, 0x49b, 0x4ad, 0x3, 0x2, 0x2, 0x2, - 0x49c, 0x49d, 0xc, 0x5, 0x2, 0x2, 0x49d, 0x49e, 0x5, 0x86, 0x44, 0x2, - 0x49e, 0x49f, 0x5, 0x82, 0x42, 0x6, 0x49f, 0x4ac, 0x3, 0x2, 0x2, 0x2, - 0x4a0, 0x4a2, 0xc, 0x6, 0x2, 0x2, 0x4a1, 0x4a3, 0x9, 0xa, 0x2, 0x2, - 0x4a2, 0x4a1, 0x3, 0x2, 0x2, 0x2, 0x4a2, 0x4a3, 0x3, 0x2, 0x2, 0x2, - 0x4a3, 0x4a5, 0x3, 0x2, 0x2, 0x2, 0x4a4, 0x4a6, 0x5, 0x84, 0x43, 0x2, - 0x4a5, 0x4a4, 0x3, 0x2, 0x2, 0x2, 0x4a5, 0x4a6, 0x3, 0x2, 0x2, 0x2, - 0x4a6, 0x4a7, 0x3, 0x2, 0x2, 0x2, 0x4a7, 0x4a8, 0x7, 0x58, 0x2, 0x2, - 0x4a8, 0x4a9, 0x5, 0x82, 0x42, 0x2, 0x4a9, 0x4aa, 0x5, 0x88, 0x45, 0x2, - 0x4aa, 0x4ac, 0x3, 0x2, 0x2, 0x2, 0x4ab, 0x49c, 0x3, 0x2, 0x2, 0x2, - 0x4ab, 0x4a0, 0x3, 0x2, 0x2, 0x2, 0x4ac, 0x4af, 0x3, 0x2, 0x2, 0x2, - 0x4ad, 0x4ab, 0x3, 0x2, 0x2, 0x2, 0x4ad, 0x4ae, 0x3, 0x2, 0x2, 0x2, - 0x4ae, 0x83, 0x3, 0x2, 0x2, 0x2, 0x4af, 0x4ad, 0x3, 0x2, 0x2, 0x2, 0x4b0, - 0x4b2, 0x9, 0xb, 0x2, 0x2, 0x4b1, 0x4b0, 0x3, 0x2, 0x2, 0x2, 0x4b1, - 0x4b2, 0x3, 0x2, 0x2, 0x2, 0x4b2, 0x4b3, 0x3, 0x2, 0x2, 0x2, 0x4b3, - 0x4ba, 0x7, 0x52, 0x2, 0x2, 0x4b4, 0x4b6, 0x7, 0x52, 0x2, 0x2, 0x4b5, - 0x4b7, 0x9, 0xb, 0x2, 0x2, 0x4b6, 0x4b5, 0x3, 0x2, 0x2, 0x2, 0x4b6, - 0x4b7, 0x3, 0x2, 0x2, 0x2, 0x4b7, 0x4ba, 0x3, 0x2, 0x2, 0x2, 0x4b8, - 0x4ba, 0x9, 0xb, 0x2, 0x2, 0x4b9, 0x4b1, 0x3, 0x2, 0x2, 0x2, 0x4b9, - 0x4b4, 0x3, 0x2, 0x2, 0x2, 0x4b9, 0x4b8, 0x3, 0x2, 0x2, 0x2, 0x4ba, - 0x4dc, 0x3, 0x2, 0x2, 0x2, 0x4bb, 0x4bd, 0x9, 0xc, 0x2, 0x2, 0x4bc, - 0x4bb, 0x3, 0x2, 0x2, 0x2, 0x4bc, 0x4bd, 0x3, 0x2, 0x2, 0x2, 0x4bd, - 0x4be, 0x3, 0x2, 0x2, 0x2, 0x4be, 0x4c0, 0x9, 0xd, 0x2, 0x2, 0x4bf, - 0x4c1, 0x7, 0x78, 0x2, 0x2, 0x4c0, 0x4bf, 0x3, 0x2, 0x2, 0x2, 0x4c0, - 0x4c1, 0x3, 0x2, 0x2, 0x2, 0x4c1, 0x4ca, 0x3, 0x2, 0x2, 0x2, 0x4c2, - 0x4c4, 0x9, 0xd, 0x2, 0x2, 0x4c3, 0x4c5, 0x7, 0x78, 0x2, 0x2, 0x4c4, - 0x4c3, 0x3, 0x2, 0x2, 0x2, 0x4c4, 0x4c5, 0x3, 0x2, 0x2, 0x2, 0x4c5, - 0x4c7, 0x3, 0x2, 0x2, 0x2, 0x4c6, 0x4c8, 0x9, 0xc, 0x2, 0x2, 0x4c7, - 0x4c6, 0x3, 0x2, 0x2, 0x2, 0x4c7, 0x4c8, 0x3, 0x2, 0x2, 0x2, 0x4c8, - 0x4ca, 0x3, 0x2, 0x2, 0x2, 0x4c9, 0x4bc, 0x3, 0x2, 0x2, 0x2, 0x4c9, - 0x4c2, 0x3, 0x2, 0x2, 0x2, 0x4ca, 0x4dc, 0x3, 0x2, 0x2, 0x2, 0x4cb, - 0x4cd, 0x9, 0xe, 0x2, 0x2, 0x4cc, 0x4cb, 0x3, 0x2, 0x2, 0x2, 0x4cc, - 0x4cd, 0x3, 0x2, 0x2, 0x2, 0x4cd, 0x4ce, 0x3, 0x2, 0x2, 0x2, 0x4ce, - 0x4d0, 0x7, 0x43, 0x2, 0x2, 0x4cf, 0x4d1, 0x7, 0x78, 0x2, 0x2, 0x4d0, - 0x4cf, 0x3, 0x2, 0x2, 0x2, 0x4d0, 0x4d1, 0x3, 0x2, 0x2, 0x2, 0x4d1, - 0x4da, 0x3, 0x2, 0x2, 0x2, 0x4d2, 0x4d4, 0x7, 0x43, 0x2, 0x2, 0x4d3, - 0x4d5, 0x7, 0x78, 0x2, 0x2, 0x4d4, 0x4d3, 0x3, 0x2, 0x2, 0x2, 0x4d4, - 0x4d5, 0x3, 0x2, 0x2, 0x2, 0x4d5, 0x4d7, 0x3, 0x2, 0x2, 0x2, 0x4d6, - 0x4d8, 0x9, 0xe, 0x2, 0x2, 0x4d7, 0x4d6, 0x3, 0x2, 0x2, 0x2, 0x4d7, - 0x4d8, 0x3, 0x2, 0x2, 0x2, 0x4d8, 0x4da, 0x3, 0x2, 0x2, 0x2, 0x4d9, - 0x4cc, 0x3, 0x2, 0x2, 0x2, 0x4d9, 0x4d2, 0x3, 0x2, 0x2, 0x2, 0x4da, - 0x4dc, 0x3, 0x2, 0x2, 0x2, 0x4db, 0x4b9, 0x3, 0x2, 0x2, 0x2, 0x4db, - 0x4c9, 0x3, 0x2, 0x2, 0x2, 0x4db, 0x4d9, 0x3, 0x2, 0x2, 0x2, 0x4dc, - 0x85, 0x3, 0x2, 0x2, 0x2, 0x4dd, 0x4df, 0x9, 0xa, 0x2, 0x2, 0x4de, 0x4dd, - 0x3, 0x2, 0x2, 0x2, 0x4de, 0x4df, 0x3, 0x2, 0x2, 0x2, 0x4df, 0x4e0, - 0x3, 0x2, 0x2, 0x2, 0x4e0, 0x4e1, 0x7, 0x1f, 0x2, 0x2, 0x4e1, 0x4e4, - 0x7, 0x58, 0x2, 0x2, 0x4e2, 0x4e4, 0x7, 0xc2, 0x2, 0x2, 0x4e3, 0x4de, - 0x3, 0x2, 0x2, 0x2, 0x4e3, 0x4e2, 0x3, 0x2, 0x2, 0x2, 0x4e4, 0x87, 0x3, - 0x2, 0x2, 0x2, 0x4e5, 0x4e6, 0x7, 0x74, 0x2, 0x2, 0x4e6, 0x4ef, 0x5, - 0xa6, 0x54, 0x2, 0x4e7, 0x4e8, 0x7, 0xaa, 0x2, 0x2, 0x4e8, 0x4e9, 0x7, - 0xcd, 0x2, 0x2, 0x4e9, 0x4ea, 0x5, 0xa6, 0x54, 0x2, 0x4ea, 0x4eb, 0x7, - 0xd7, 0x2, 0x2, 0x4eb, 0x4ef, 0x3, 0x2, 0x2, 0x2, 0x4ec, 0x4ed, 0x7, - 0xaa, 0x2, 0x2, 0x4ed, 0x4ef, 0x5, 0xa6, 0x54, 0x2, 0x4ee, 0x4e5, 0x3, - 0x2, 0x2, 0x2, 0x4ee, 0x4e7, 0x3, 0x2, 0x2, 0x2, 0x4ee, 0x4ec, 0x3, - 0x2, 0x2, 0x2, 0x4ef, 0x89, 0x3, 0x2, 0x2, 0x2, 0x4f0, 0x4f1, 0x7, 0x88, - 0x2, 0x2, 0x4f1, 0x4f4, 0x5, 0x92, 0x4a, 0x2, 0x4f2, 0x4f3, 0x7, 0x73, - 0x2, 0x2, 0x4f3, 0x4f5, 0x5, 0x92, 0x4a, 0x2, 0x4f4, 0x4f2, 0x3, 0x2, - 0x2, 0x2, 0x4f4, 0x4f5, 0x3, 0x2, 0x2, 0x2, 0x4f5, 0x8b, 0x3, 0x2, 0x2, - 0x2, 0x4f6, 0x4f9, 0x5, 0xaa, 0x56, 0x2, 0x4f7, 0x4f8, 0x9, 0xf, 0x2, - 0x2, 0x4f8, 0x4fa, 0x5, 0xaa, 0x56, 0x2, 0x4f9, 0x4f7, 0x3, 0x2, 0x2, - 0x2, 0x4f9, 0x4fa, 0x3, 0x2, 0x2, 0x2, 0x4fa, 0x8d, 0x3, 0x2, 0x2, 0x2, - 0x4fb, 0x500, 0x5, 0x90, 0x49, 0x2, 0x4fc, 0x4fd, 0x7, 0xc2, 0x2, 0x2, - 0x4fd, 0x4ff, 0x5, 0x90, 0x49, 0x2, 0x4fe, 0x4fc, 0x3, 0x2, 0x2, 0x2, - 0x4ff, 0x502, 0x3, 0x2, 0x2, 0x2, 0x500, 0x4fe, 0x3, 0x2, 0x2, 0x2, - 0x500, 0x501, 0x3, 0x2, 0x2, 0x2, 0x501, 0x8f, 0x3, 0x2, 0x2, 0x2, 0x502, - 0x500, 0x3, 0x2, 0x2, 0x2, 0x503, 0x505, 0x5, 0xaa, 0x56, 0x2, 0x504, - 0x506, 0x9, 0x10, 0x2, 0x2, 0x505, 0x504, 0x3, 0x2, 0x2, 0x2, 0x505, - 0x506, 0x3, 0x2, 0x2, 0x2, 0x506, 0x509, 0x3, 0x2, 0x2, 0x2, 0x507, - 0x508, 0x7, 0x72, 0x2, 0x2, 0x508, 0x50a, 0x9, 0x11, 0x2, 0x2, 0x509, - 0x507, 0x3, 0x2, 0x2, 0x2, 0x509, 0x50a, 0x3, 0x2, 0x2, 0x2, 0x50a, - 0x50d, 0x3, 0x2, 0x2, 0x2, 0x50b, 0x50c, 0x7, 0x1a, 0x2, 0x2, 0x50c, - 0x50e, 0x7, 0xbc, 0x2, 0x2, 0x50d, 0x50b, 0x3, 0x2, 0x2, 0x2, 0x50d, - 0x50e, 0x3, 0x2, 0x2, 0x2, 0x50e, 0x91, 0x3, 0x2, 0x2, 0x2, 0x50f, 0x512, - 0x5, 0xc4, 0x63, 0x2, 0x510, 0x511, 0x7, 0xd9, 0x2, 0x2, 0x511, 0x513, - 0x5, 0xc4, 0x63, 0x2, 0x512, 0x510, 0x3, 0x2, 0x2, 0x2, 0x512, 0x513, - 0x3, 0x2, 0x2, 0x2, 0x513, 0x93, 0x3, 0x2, 0x2, 0x2, 0x514, 0x519, 0x5, - 0x96, 0x4c, 0x2, 0x515, 0x516, 0x7, 0xc2, 0x2, 0x2, 0x516, 0x518, 0x5, - 0x96, 0x4c, 0x2, 0x517, 0x515, 0x3, 0x2, 0x2, 0x2, 0x518, 0x51b, 0x3, - 0x2, 0x2, 0x2, 0x519, 0x517, 0x3, 0x2, 0x2, 0x2, 0x519, 0x51a, 0x3, - 0x2, 0x2, 0x2, 0x51a, 0x95, 0x3, 0x2, 0x2, 0x2, 0x51b, 0x519, 0x3, 0x2, - 0x2, 0x2, 0x51c, 0x51d, 0x5, 0xd0, 0x69, 0x2, 0x51d, 0x51e, 0x7, 0xc7, - 0x2, 0x2, 0x51e, 0x51f, 0x5, 0xc6, 0x64, 0x2, 0x51f, 0x97, 0x3, 0x2, - 0x2, 0x2, 0x520, 0x521, 0x7, 0x8d, 0x2, 0x2, 0x521, 0x522, 0x5, 0x94, - 0x4b, 0x2, 0x522, 0x99, 0x3, 0x2, 0x2, 0x2, 0x523, 0x524, 0x7, 0x8f, - 0x2, 0x2, 0x524, 0x525, 0x7, 0x1e, 0x2, 0x2, 0x525, 0x526, 0x7, 0x21, - 0x2, 0x2, 0x526, 0x54e, 0x5, 0xc0, 0x61, 0x2, 0x527, 0x528, 0x7, 0x8f, - 0x2, 0x2, 0x528, 0x529, 0x7, 0x1e, 0x2, 0x2, 0x529, 0x52a, 0x7, 0x2e, - 0x2, 0x2, 0x52a, 0x54e, 0x5, 0xba, 0x5e, 0x2, 0x52b, 0x52c, 0x7, 0x8f, - 0x2, 0x2, 0x52c, 0x52e, 0x7, 0x1e, 0x2, 0x2, 0x52d, 0x52f, 0x7, 0x99, - 0x2, 0x2, 0x52e, 0x52d, 0x3, 0x2, 0x2, 0x2, 0x52e, 0x52f, 0x3, 0x2, - 0x2, 0x2, 0x52f, 0x531, 0x3, 0x2, 0x2, 0x2, 0x530, 0x532, 0x7, 0x97, - 0x2, 0x2, 0x531, 0x530, 0x3, 0x2, 0x2, 0x2, 0x531, 0x532, 0x3, 0x2, - 0x2, 0x2, 0x532, 0x533, 0x3, 0x2, 0x2, 0x2, 0x533, 0x54e, 0x5, 0xba, - 0x5e, 0x2, 0x534, 0x535, 0x7, 0x8f, 0x2, 0x2, 0x535, 0x54e, 0x7, 0x22, - 0x2, 0x2, 0x536, 0x537, 0x7, 0x8f, 0x2, 0x2, 0x537, 0x53a, 0x7, 0x2d, - 0x2, 0x2, 0x538, 0x539, 0x7, 0x42, 0x2, 0x2, 0x539, 0x53b, 0x5, 0xc0, - 0x61, 0x2, 0x53a, 0x538, 0x3, 0x2, 0x2, 0x2, 0x53a, 0x53b, 0x3, 0x2, - 0x2, 0x2, 0x53b, 0x54e, 0x3, 0x2, 0x2, 0x2, 0x53c, 0x53e, 0x7, 0x8f, - 0x2, 0x2, 0x53d, 0x53f, 0x7, 0x99, 0x2, 0x2, 0x53e, 0x53d, 0x3, 0x2, - 0x2, 0x2, 0x53e, 0x53f, 0x3, 0x2, 0x2, 0x2, 0x53f, 0x540, 0x3, 0x2, - 0x2, 0x2, 0x540, 0x543, 0x7, 0x98, 0x2, 0x2, 0x541, 0x542, 0x9, 0x12, - 0x2, 0x2, 0x542, 0x544, 0x5, 0xc0, 0x61, 0x2, 0x543, 0x541, 0x3, 0x2, - 0x2, 0x2, 0x543, 0x544, 0x3, 0x2, 0x2, 0x2, 0x544, 0x548, 0x3, 0x2, - 0x2, 0x2, 0x545, 0x546, 0x7, 0x60, 0x2, 0x2, 0x546, 0x549, 0x7, 0xbc, - 0x2, 0x2, 0x547, 0x549, 0x5, 0x74, 0x3b, 0x2, 0x548, 0x545, 0x3, 0x2, - 0x2, 0x2, 0x548, 0x547, 0x3, 0x2, 0x2, 0x2, 0x548, 0x549, 0x3, 0x2, - 0x2, 0x2, 0x549, 0x54b, 0x3, 0x2, 0x2, 0x2, 0x54a, 0x54c, 0x5, 0x7e, - 0x40, 0x2, 0x54b, 0x54a, 0x3, 0x2, 0x2, 0x2, 0x54b, 0x54c, 0x3, 0x2, - 0x2, 0x2, 0x54c, 0x54e, 0x3, 0x2, 0x2, 0x2, 0x54d, 0x523, 0x3, 0x2, - 0x2, 0x2, 0x54d, 0x527, 0x3, 0x2, 0x2, 0x2, 0x54d, 0x52b, 0x3, 0x2, - 0x2, 0x2, 0x54d, 0x534, 0x3, 0x2, 0x2, 0x2, 0x54d, 0x536, 0x3, 0x2, - 0x2, 0x2, 0x54d, 0x53c, 0x3, 0x2, 0x2, 0x2, 0x54e, 0x9b, 0x3, 0x2, 0x2, - 0x2, 0x54f, 0x550, 0x7, 0x96, 0x2, 0x2, 0x550, 0x551, 0x7, 0x3e, 0x2, - 0x2, 0x551, 0x552, 0x7, 0x31, 0x2, 0x2, 0x552, 0x572, 0x5, 0xba, 0x5e, - 0x2, 0x553, 0x554, 0x7, 0x96, 0x2, 0x2, 0x554, 0x555, 0x7, 0x3e, 0x2, - 0x2, 0x555, 0x572, 0x7, 0x64, 0x2, 0x2, 0x556, 0x557, 0x7, 0x96, 0x2, - 0x2, 0x557, 0x558, 0x7, 0x80, 0x2, 0x2, 0x558, 0x572, 0x7, 0x2d, 0x2, - 0x2, 0x559, 0x55a, 0x7, 0x96, 0x2, 0x2, 0x55a, 0x55b, 0x7, 0x80, 0x2, - 0x2, 0x55b, 0x55c, 0x7, 0x2e, 0x2, 0x2, 0x55c, 0x572, 0x5, 0xba, 0x5e, - 0x2, 0x55d, 0x55e, 0x7, 0x96, 0x2, 0x2, 0x55e, 0x566, 0x9, 0x13, 0x2, - 0x2, 0x55f, 0x560, 0x7, 0x31, 0x2, 0x2, 0x560, 0x567, 0x7, 0x8c, 0x2, - 0x2, 0x561, 0x567, 0x7, 0x3b, 0x2, 0x2, 0x562, 0x564, 0x7, 0xa5, 0x2, - 0x2, 0x563, 0x562, 0x3, 0x2, 0x2, 0x2, 0x563, 0x564, 0x3, 0x2, 0x2, - 0x2, 0x564, 0x565, 0x3, 0x2, 0x2, 0x2, 0x565, 0x567, 0x7, 0x67, 0x2, - 0x2, 0x566, 0x55f, 0x3, 0x2, 0x2, 0x2, 0x566, 0x561, 0x3, 0x2, 0x2, - 0x2, 0x566, 0x563, 0x3, 0x2, 0x2, 0x2, 0x567, 0x568, 0x3, 0x2, 0x2, - 0x2, 0x568, 0x572, 0x5, 0xba, 0x5e, 0x2, 0x569, 0x56a, 0x7, 0x96, 0x2, - 0x2, 0x56a, 0x56b, 0x9, 0x13, 0x2, 0x2, 0x56b, 0x56c, 0x7, 0x85, 0x2, - 0x2, 0x56c, 0x572, 0x7, 0x8c, 0x2, 0x2, 0x56d, 0x56e, 0x7, 0x96, 0x2, - 0x2, 0x56e, 0x56f, 0x7, 0x94, 0x2, 0x2, 0x56f, 0x570, 0x7, 0x84, 0x2, - 0x2, 0x570, 0x572, 0x5, 0xba, 0x5e, 0x2, 0x571, 0x54f, 0x3, 0x2, 0x2, - 0x2, 0x571, 0x553, 0x3, 0x2, 0x2, 0x2, 0x571, 0x556, 0x3, 0x2, 0x2, - 0x2, 0x571, 0x559, 0x3, 0x2, 0x2, 0x2, 0x571, 0x55d, 0x3, 0x2, 0x2, - 0x2, 0x571, 0x569, 0x3, 0x2, 0x2, 0x2, 0x571, 0x56d, 0x3, 0x2, 0x2, - 0x2, 0x572, 0x9d, 0x3, 0x2, 0x2, 0x2, 0x573, 0x575, 0x7, 0xa4, 0x2, - 0x2, 0x574, 0x576, 0x7, 0x99, 0x2, 0x2, 0x575, 0x574, 0x3, 0x2, 0x2, - 0x2, 0x575, 0x576, 0x3, 0x2, 0x2, 0x2, 0x576, 0x578, 0x3, 0x2, 0x2, - 0x2, 0x577, 0x579, 0x7, 0x97, 0x2, 0x2, 0x578, 0x577, 0x3, 0x2, 0x2, - 0x2, 0x578, 0x579, 0x3, 0x2, 0x2, 0x2, 0x579, 0x57c, 0x3, 0x2, 0x2, - 0x2, 0x57a, 0x57b, 0x7, 0x4c, 0x2, 0x2, 0x57b, 0x57d, 0x7, 0x37, 0x2, - 0x2, 0x57c, 0x57a, 0x3, 0x2, 0x2, 0x2, 0x57c, 0x57d, 0x3, 0x2, 0x2, - 0x2, 0x57d, 0x57e, 0x3, 0x2, 0x2, 0x2, 0x57e, 0x580, 0x5, 0xba, 0x5e, - 0x2, 0x57f, 0x581, 0x5, 0x2c, 0x17, 0x2, 0x580, 0x57f, 0x3, 0x2, 0x2, - 0x2, 0x580, 0x581, 0x3, 0x2, 0x2, 0x2, 0x581, 0x9f, 0x3, 0x2, 0x2, 0x2, - 0x582, 0x583, 0x7, 0xa9, 0x2, 0x2, 0x583, 0x584, 0x5, 0xc0, 0x61, 0x2, - 0x584, 0xa1, 0x3, 0x2, 0x2, 0x2, 0x585, 0x586, 0x7, 0xaf, 0x2, 0x2, - 0x586, 0x588, 0x5, 0xba, 0x5e, 0x2, 0x587, 0x589, 0x7, 0x36, 0x2, 0x2, - 0x588, 0x587, 0x3, 0x2, 0x2, 0x2, 0x588, 0x589, 0x3, 0x2, 0x2, 0x2, - 0x589, 0x58c, 0x3, 0x2, 0x2, 0x2, 0x58a, 0x58b, 0x7, 0x61, 0x2, 0x2, - 0x58b, 0x58d, 0x7, 0xba, 0x2, 0x2, 0x58c, 0x58a, 0x3, 0x2, 0x2, 0x2, - 0x58c, 0x58d, 0x3, 0x2, 0x2, 0x2, 0x58d, 0xa3, 0x3, 0x2, 0x2, 0x2, 0x58e, - 0x5be, 0x5, 0xd0, 0x69, 0x2, 0x58f, 0x590, 0x5, 0xd0, 0x69, 0x2, 0x590, - 0x591, 0x7, 0xcd, 0x2, 0x2, 0x591, 0x592, 0x5, 0xd0, 0x69, 0x2, 0x592, - 0x599, 0x5, 0xa4, 0x53, 0x2, 0x593, 0x594, 0x7, 0xc2, 0x2, 0x2, 0x594, - 0x595, 0x5, 0xd0, 0x69, 0x2, 0x595, 0x596, 0x5, 0xa4, 0x53, 0x2, 0x596, - 0x598, 0x3, 0x2, 0x2, 0x2, 0x597, 0x593, 0x3, 0x2, 0x2, 0x2, 0x598, - 0x59b, 0x3, 0x2, 0x2, 0x2, 0x599, 0x597, 0x3, 0x2, 0x2, 0x2, 0x599, - 0x59a, 0x3, 0x2, 0x2, 0x2, 0x59a, 0x59c, 0x3, 0x2, 0x2, 0x2, 0x59b, - 0x599, 0x3, 0x2, 0x2, 0x2, 0x59c, 0x59d, 0x7, 0xd7, 0x2, 0x2, 0x59d, - 0x5be, 0x3, 0x2, 0x2, 0x2, 0x59e, 0x59f, 0x5, 0xd0, 0x69, 0x2, 0x59f, - 0x5a0, 0x7, 0xcd, 0x2, 0x2, 0x5a0, 0x5a5, 0x5, 0xd4, 0x6b, 0x2, 0x5a1, - 0x5a2, 0x7, 0xc2, 0x2, 0x2, 0x5a2, 0x5a4, 0x5, 0xd4, 0x6b, 0x2, 0x5a3, - 0x5a1, 0x3, 0x2, 0x2, 0x2, 0x5a4, 0x5a7, 0x3, 0x2, 0x2, 0x2, 0x5a5, - 0x5a3, 0x3, 0x2, 0x2, 0x2, 0x5a5, 0x5a6, 0x3, 0x2, 0x2, 0x2, 0x5a6, - 0x5a8, 0x3, 0x2, 0x2, 0x2, 0x5a7, 0x5a5, 0x3, 0x2, 0x2, 0x2, 0x5a8, - 0x5a9, 0x7, 0xd7, 0x2, 0x2, 0x5a9, 0x5be, 0x3, 0x2, 0x2, 0x2, 0x5aa, - 0x5ab, 0x5, 0xd0, 0x69, 0x2, 0x5ab, 0x5ac, 0x7, 0xcd, 0x2, 0x2, 0x5ac, - 0x5b1, 0x5, 0xa4, 0x53, 0x2, 0x5ad, 0x5ae, 0x7, 0xc2, 0x2, 0x2, 0x5ae, - 0x5b0, 0x5, 0xa4, 0x53, 0x2, 0x5af, 0x5ad, 0x3, 0x2, 0x2, 0x2, 0x5b0, - 0x5b3, 0x3, 0x2, 0x2, 0x2, 0x5b1, 0x5af, 0x3, 0x2, 0x2, 0x2, 0x5b1, - 0x5b2, 0x3, 0x2, 0x2, 0x2, 0x5b2, 0x5b4, 0x3, 0x2, 0x2, 0x2, 0x5b3, - 0x5b1, 0x3, 0x2, 0x2, 0x2, 0x5b4, 0x5b5, 0x7, 0xd7, 0x2, 0x2, 0x5b5, - 0x5be, 0x3, 0x2, 0x2, 0x2, 0x5b6, 0x5b7, 0x5, 0xd0, 0x69, 0x2, 0x5b7, - 0x5b9, 0x7, 0xcd, 0x2, 0x2, 0x5b8, 0x5ba, 0x5, 0xa6, 0x54, 0x2, 0x5b9, - 0x5b8, 0x3, 0x2, 0x2, 0x2, 0x5b9, 0x5ba, 0x3, 0x2, 0x2, 0x2, 0x5ba, - 0x5bb, 0x3, 0x2, 0x2, 0x2, 0x5bb, 0x5bc, 0x7, 0xd7, 0x2, 0x2, 0x5bc, - 0x5be, 0x3, 0x2, 0x2, 0x2, 0x5bd, 0x58e, 0x3, 0x2, 0x2, 0x2, 0x5bd, - 0x58f, 0x3, 0x2, 0x2, 0x2, 0x5bd, 0x59e, 0x3, 0x2, 0x2, 0x2, 0x5bd, - 0x5aa, 0x3, 0x2, 0x2, 0x2, 0x5bd, 0x5b6, 0x3, 0x2, 0x2, 0x2, 0x5be, - 0xa5, 0x3, 0x2, 0x2, 0x2, 0x5bf, 0x5c4, 0x5, 0xa8, 0x55, 0x2, 0x5c0, - 0x5c1, 0x7, 0xc2, 0x2, 0x2, 0x5c1, 0x5c3, 0x5, 0xa8, 0x55, 0x2, 0x5c2, - 0x5c0, 0x3, 0x2, 0x2, 0x2, 0x5c3, 0x5c6, 0x3, 0x2, 0x2, 0x2, 0x5c4, - 0x5c2, 0x3, 0x2, 0x2, 0x2, 0x5c4, 0x5c5, 0x3, 0x2, 0x2, 0x2, 0x5c5, - 0xa7, 0x3, 0x2, 0x2, 0x2, 0x5c6, 0x5c4, 0x3, 0x2, 0x2, 0x2, 0x5c7, 0x5c8, - 0x5, 0xba, 0x5e, 0x2, 0x5c8, 0x5c9, 0x7, 0xc5, 0x2, 0x2, 0x5c9, 0x5cb, - 0x3, 0x2, 0x2, 0x2, 0x5ca, 0x5c7, 0x3, 0x2, 0x2, 0x2, 0x5ca, 0x5cb, - 0x3, 0x2, 0x2, 0x2, 0x5cb, 0x5cc, 0x3, 0x2, 0x2, 0x2, 0x5cc, 0x5d3, - 0x7, 0xbe, 0x2, 0x2, 0x5cd, 0x5ce, 0x7, 0xcd, 0x2, 0x2, 0x5ce, 0x5cf, - 0x5, 0x64, 0x33, 0x2, 0x5cf, 0x5d0, 0x7, 0xd7, 0x2, 0x2, 0x5d0, 0x5d3, - 0x3, 0x2, 0x2, 0x2, 0x5d1, 0x5d3, 0x5, 0xaa, 0x56, 0x2, 0x5d2, 0x5ca, - 0x3, 0x2, 0x2, 0x2, 0x5d2, 0x5cd, 0x3, 0x2, 0x2, 0x2, 0x5d2, 0x5d1, - 0x3, 0x2, 0x2, 0x2, 0x5d3, 0xa9, 0x3, 0x2, 0x2, 0x2, 0x5d4, 0x5d5, 0x8, - 0x56, 0x1, 0x2, 0x5d5, 0x5d7, 0x7, 0x14, 0x2, 0x2, 0x5d6, 0x5d8, 0x5, - 0xaa, 0x56, 0x2, 0x5d7, 0x5d6, 0x3, 0x2, 0x2, 0x2, 0x5d7, 0x5d8, 0x3, - 0x2, 0x2, 0x2, 0x5d8, 0x5de, 0x3, 0x2, 0x2, 0x2, 0x5d9, 0x5da, 0x7, - 0xb1, 0x2, 0x2, 0x5da, 0x5db, 0x5, 0xaa, 0x56, 0x2, 0x5db, 0x5dc, 0x7, - 0x9b, 0x2, 0x2, 0x5dc, 0x5dd, 0x5, 0xaa, 0x56, 0x2, 0x5dd, 0x5df, 0x3, - 0x2, 0x2, 0x2, 0x5de, 0x5d9, 0x3, 0x2, 0x2, 0x2, 0x5df, 0x5e0, 0x3, - 0x2, 0x2, 0x2, 0x5e0, 0x5de, 0x3, 0x2, 0x2, 0x2, 0x5e0, 0x5e1, 0x3, - 0x2, 0x2, 0x2, 0x5e1, 0x5e4, 0x3, 0x2, 0x2, 0x2, 0x5e2, 0x5e3, 0x7, - 0x33, 0x2, 0x2, 0x5e3, 0x5e5, 0x5, 0xaa, 0x56, 0x2, 0x5e4, 0x5e2, 0x3, - 0x2, 0x2, 0x2, 0x5e4, 0x5e5, 0x3, 0x2, 0x2, 0x2, 0x5e5, 0x5e6, 0x3, - 0x2, 0x2, 0x2, 0x5e6, 0x5e7, 0x7, 0x34, 0x2, 0x2, 0x5e7, 0x640, 0x3, - 0x2, 0x2, 0x2, 0x5e8, 0x5e9, 0x7, 0x15, 0x2, 0x2, 0x5e9, 0x5ea, 0x7, - 0xcd, 0x2, 0x2, 0x5ea, 0x5eb, 0x5, 0xaa, 0x56, 0x2, 0x5eb, 0x5ec, 0x7, - 0xc, 0x2, 0x2, 0x5ec, 0x5ed, 0x5, 0xa4, 0x53, 0x2, 0x5ed, 0x5ee, 0x7, - 0xd7, 0x2, 0x2, 0x5ee, 0x640, 0x3, 0x2, 0x2, 0x2, 0x5ef, 0x5f0, 0x7, - 0x23, 0x2, 0x2, 0x5f0, 0x640, 0x7, 0xbc, 0x2, 0x2, 0x5f1, 0x5f2, 0x7, - 0x3a, 0x2, 0x2, 0x5f2, 0x5f3, 0x7, 0xcd, 0x2, 0x2, 0x5f3, 0x5f4, 0x5, - 0xc8, 0x65, 0x2, 0x5f4, 0x5f5, 0x7, 0x42, 0x2, 0x2, 0x5f5, 0x5f6, 0x5, - 0xaa, 0x56, 0x2, 0x5f6, 0x5f7, 0x7, 0xd7, 0x2, 0x2, 0x5f7, 0x640, 0x3, - 0x2, 0x2, 0x2, 0x5f8, 0x5f9, 0x7, 0x54, 0x2, 0x2, 0x5f9, 0x5fa, 0x5, - 0xaa, 0x56, 0x2, 0x5fa, 0x5fb, 0x5, 0xc8, 0x65, 0x2, 0x5fb, 0x640, 0x3, - 0x2, 0x2, 0x2, 0x5fc, 0x5fd, 0x7, 0x93, 0x2, 0x2, 0x5fd, 0x5fe, 0x7, - 0xcd, 0x2, 0x2, 0x5fe, 0x5ff, 0x5, 0xaa, 0x56, 0x2, 0x5ff, 0x600, 0x7, - 0x42, 0x2, 0x2, 0x600, 0x603, 0x5, 0xaa, 0x56, 0x2, 0x601, 0x602, 0x7, - 0x3f, 0x2, 0x2, 0x602, 0x604, 0x5, 0xaa, 0x56, 0x2, 0x603, 0x601, 0x3, - 0x2, 0x2, 0x2, 0x603, 0x604, 0x3, 0x2, 0x2, 0x2, 0x604, 0x605, 0x3, - 0x2, 0x2, 0x2, 0x605, 0x606, 0x7, 0xd7, 0x2, 0x2, 0x606, 0x640, 0x3, - 0x2, 0x2, 0x2, 0x607, 0x608, 0x7, 0x9e, 0x2, 0x2, 0x608, 0x640, 0x7, - 0xbc, 0x2, 0x2, 0x609, 0x60a, 0x7, 0xa3, 0x2, 0x2, 0x60a, 0x60b, 0x7, - 0xcd, 0x2, 0x2, 0x60b, 0x60c, 0x9, 0x14, 0x2, 0x2, 0x60c, 0x60d, 0x7, - 0xbc, 0x2, 0x2, 0x60d, 0x60e, 0x7, 0x42, 0x2, 0x2, 0x60e, 0x60f, 0x5, - 0xaa, 0x56, 0x2, 0x60f, 0x610, 0x7, 0xd7, 0x2, 0x2, 0x610, 0x640, 0x3, - 0x2, 0x2, 0x2, 0x611, 0x617, 0x5, 0xd0, 0x69, 0x2, 0x612, 0x614, 0x7, - 0xcd, 0x2, 0x2, 0x613, 0x615, 0x5, 0xa6, 0x54, 0x2, 0x614, 0x613, 0x3, - 0x2, 0x2, 0x2, 0x614, 0x615, 0x3, 0x2, 0x2, 0x2, 0x615, 0x616, 0x3, - 0x2, 0x2, 0x2, 0x616, 0x618, 0x7, 0xd7, 0x2, 0x2, 0x617, 0x612, 0x3, - 0x2, 0x2, 0x2, 0x617, 0x618, 0x3, 0x2, 0x2, 0x2, 0x618, 0x619, 0x3, - 0x2, 0x2, 0x2, 0x619, 0x61b, 0x7, 0xcd, 0x2, 0x2, 0x61a, 0x61c, 0x7, - 0x30, 0x2, 0x2, 0x61b, 0x61a, 0x3, 0x2, 0x2, 0x2, 0x61b, 0x61c, 0x3, - 0x2, 0x2, 0x2, 0x61c, 0x61e, 0x3, 0x2, 0x2, 0x2, 0x61d, 0x61f, 0x5, - 0xac, 0x57, 0x2, 0x61e, 0x61d, 0x3, 0x2, 0x2, 0x2, 0x61e, 0x61f, 0x3, - 0x2, 0x2, 0x2, 0x61f, 0x620, 0x3, 0x2, 0x2, 0x2, 0x620, 0x621, 0x7, - 0xd7, 0x2, 0x2, 0x621, 0x640, 0x3, 0x2, 0x2, 0x2, 0x622, 0x640, 0x5, - 0xc6, 0x64, 0x2, 0x623, 0x624, 0x7, 0xc4, 0x2, 0x2, 0x624, 0x640, 0x5, - 0xaa, 0x56, 0x13, 0x625, 0x626, 0x7, 0x70, 0x2, 0x2, 0x626, 0x640, 0x5, - 0xaa, 0x56, 0xe, 0x627, 0x628, 0x5, 0xba, 0x5e, 0x2, 0x628, 0x629, 0x7, - 0xc5, 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, 0x640, 0x7, 0xbe, 0x2, 0x2, 0x62d, 0x62e, 0x7, - 0xcd, 0x2, 0x2, 0x62e, 0x62f, 0x5, 0x64, 0x33, 0x2, 0x62f, 0x630, 0x7, - 0xd7, 0x2, 0x2, 0x630, 0x640, 0x3, 0x2, 0x2, 0x2, 0x631, 0x632, 0x7, - 0xcd, 0x2, 0x2, 0x632, 0x633, 0x5, 0xaa, 0x56, 0x2, 0x633, 0x634, 0x7, - 0xd7, 0x2, 0x2, 0x634, 0x640, 0x3, 0x2, 0x2, 0x2, 0x635, 0x636, 0x7, - 0xcd, 0x2, 0x2, 0x636, 0x637, 0x5, 0xa6, 0x54, 0x2, 0x637, 0x638, 0x7, - 0xd7, 0x2, 0x2, 0x638, 0x640, 0x3, 0x2, 0x2, 0x2, 0x639, 0x63b, 0x7, - 0xcb, 0x2, 0x2, 0x63a, 0x63c, 0x5, 0xa6, 0x54, 0x2, 0x63b, 0x63a, 0x3, - 0x2, 0x2, 0x2, 0x63b, 0x63c, 0x3, 0x2, 0x2, 0x2, 0x63c, 0x63d, 0x3, - 0x2, 0x2, 0x2, 0x63d, 0x640, 0x7, 0xd6, 0x2, 0x2, 0x63e, 0x640, 0x5, - 0xb2, 0x5a, 0x2, 0x63f, 0x5d4, 0x3, 0x2, 0x2, 0x2, 0x63f, 0x5e8, 0x3, - 0x2, 0x2, 0x2, 0x63f, 0x5ef, 0x3, 0x2, 0x2, 0x2, 0x63f, 0x5f1, 0x3, - 0x2, 0x2, 0x2, 0x63f, 0x5f8, 0x3, 0x2, 0x2, 0x2, 0x63f, 0x5fc, 0x3, - 0x2, 0x2, 0x2, 0x63f, 0x607, 0x3, 0x2, 0x2, 0x2, 0x63f, 0x609, 0x3, - 0x2, 0x2, 0x2, 0x63f, 0x611, 0x3, 0x2, 0x2, 0x2, 0x63f, 0x622, 0x3, - 0x2, 0x2, 0x2, 0x63f, 0x623, 0x3, 0x2, 0x2, 0x2, 0x63f, 0x625, 0x3, - 0x2, 0x2, 0x2, 0x63f, 0x62a, 0x3, 0x2, 0x2, 0x2, 0x63f, 0x62d, 0x3, - 0x2, 0x2, 0x2, 0x63f, 0x631, 0x3, 0x2, 0x2, 0x2, 0x63f, 0x635, 0x3, - 0x2, 0x2, 0x2, 0x63f, 0x639, 0x3, 0x2, 0x2, 0x2, 0x63f, 0x63e, 0x3, - 0x2, 0x2, 0x2, 0x640, 0x688, 0x3, 0x2, 0x2, 0x2, 0x641, 0x642, 0xc, - 0x12, 0x2, 0x2, 0x642, 0x643, 0x9, 0x15, 0x2, 0x2, 0x643, 0x687, 0x5, - 0xaa, 0x56, 0x13, 0x644, 0x645, 0xc, 0x11, 0x2, 0x2, 0x645, 0x646, 0x9, - 0x16, 0x2, 0x2, 0x646, 0x687, 0x5, 0xaa, 0x56, 0x12, 0x647, 0x65a, 0xc, - 0x10, 0x2, 0x2, 0x648, 0x65b, 0x7, 0xc6, 0x2, 0x2, 0x649, 0x65b, 0x7, - 0xc7, 0x2, 0x2, 0x64a, 0x65b, 0x7, 0xcf, 0x2, 0x2, 0x64b, 0x65b, 0x7, - 0xcc, 0x2, 0x2, 0x64c, 0x65b, 0x7, 0xc8, 0x2, 0x2, 0x64d, 0x65b, 0x7, - 0xce, 0x2, 0x2, 0x64e, 0x65b, 0x7, 0xc9, 0x2, 0x2, 0x64f, 0x651, 0x7, - 0x45, 0x2, 0x2, 0x650, 0x64f, 0x3, 0x2, 0x2, 0x2, 0x650, 0x651, 0x3, - 0x2, 0x2, 0x2, 0x651, 0x653, 0x3, 0x2, 0x2, 0x2, 0x652, 0x654, 0x7, - 0x70, 0x2, 0x2, 0x653, 0x652, 0x3, 0x2, 0x2, 0x2, 0x653, 0x654, 0x3, - 0x2, 0x2, 0x2, 0x654, 0x655, 0x3, 0x2, 0x2, 0x2, 0x655, 0x65b, 0x7, - 0x4e, 0x2, 0x2, 0x656, 0x658, 0x7, 0x70, 0x2, 0x2, 0x657, 0x656, 0x3, - 0x2, 0x2, 0x2, 0x657, 0x658, 0x3, 0x2, 0x2, 0x2, 0x658, 0x659, 0x3, - 0x2, 0x2, 0x2, 0x659, 0x65b, 0x9, 0x17, 0x2, 0x2, 0x65a, 0x648, 0x3, - 0x2, 0x2, 0x2, 0x65a, 0x649, 0x3, 0x2, 0x2, 0x2, 0x65a, 0x64a, 0x3, - 0x2, 0x2, 0x2, 0x65a, 0x64b, 0x3, 0x2, 0x2, 0x2, 0x65a, 0x64c, 0x3, - 0x2, 0x2, 0x2, 0x65a, 0x64d, 0x3, 0x2, 0x2, 0x2, 0x65a, 0x64e, 0x3, - 0x2, 0x2, 0x2, 0x65a, 0x650, 0x3, 0x2, 0x2, 0x2, 0x65a, 0x657, 0x3, - 0x2, 0x2, 0x2, 0x65b, 0x65c, 0x3, 0x2, 0x2, 0x2, 0x65c, 0x687, 0x5, - 0xaa, 0x56, 0x11, 0x65d, 0x65e, 0xc, 0xd, 0x2, 0x2, 0x65e, 0x65f, 0x7, - 0x8, 0x2, 0x2, 0x65f, 0x687, 0x5, 0xaa, 0x56, 0xe, 0x660, 0x661, 0xc, - 0xc, 0x2, 0x2, 0x661, 0x662, 0x7, 0x76, 0x2, 0x2, 0x662, 0x687, 0x5, - 0xaa, 0x56, 0xd, 0x663, 0x665, 0xc, 0xb, 0x2, 0x2, 0x664, 0x666, 0x7, - 0x70, 0x2, 0x2, 0x665, 0x664, 0x3, 0x2, 0x2, 0x2, 0x665, 0x666, 0x3, - 0x2, 0x2, 0x2, 0x666, 0x667, 0x3, 0x2, 0x2, 0x2, 0x667, 0x668, 0x7, - 0x11, 0x2, 0x2, 0x668, 0x669, 0x5, 0xaa, 0x56, 0x2, 0x669, 0x66a, 0x7, - 0x8, 0x2, 0x2, 0x66a, 0x66b, 0x5, 0xaa, 0x56, 0xc, 0x66b, 0x687, 0x3, - 0x2, 0x2, 0x2, 0x66c, 0x66d, 0xc, 0xa, 0x2, 0x2, 0x66d, 0x66e, 0x7, - 0xd2, 0x2, 0x2, 0x66e, 0x66f, 0x5, 0xaa, 0x56, 0x2, 0x66f, 0x670, 0x7, - 0xc1, 0x2, 0x2, 0x670, 0x671, 0x5, 0xaa, 0x56, 0xa, 0x671, 0x687, 0x3, - 0x2, 0x2, 0x2, 0x672, 0x673, 0xc, 0x15, 0x2, 0x2, 0x673, 0x674, 0x7, - 0xcb, 0x2, 0x2, 0x674, 0x675, 0x5, 0xaa, 0x56, 0x2, 0x675, 0x676, 0x7, - 0xd6, 0x2, 0x2, 0x676, 0x687, 0x3, 0x2, 0x2, 0x2, 0x677, 0x678, 0xc, - 0x14, 0x2, 0x2, 0x678, 0x679, 0x7, 0xc5, 0x2, 0x2, 0x679, 0x687, 0x7, - 0xba, 0x2, 0x2, 0x67a, 0x67b, 0xc, 0xf, 0x2, 0x2, 0x67b, 0x67d, 0x7, - 0x56, 0x2, 0x2, 0x67c, 0x67e, 0x7, 0x70, 0x2, 0x2, 0x67d, 0x67c, 0x3, - 0x2, 0x2, 0x2, 0x67d, 0x67e, 0x3, 0x2, 0x2, 0x2, 0x67e, 0x67f, 0x3, - 0x2, 0x2, 0x2, 0x67f, 0x687, 0x7, 0x71, 0x2, 0x2, 0x680, 0x684, 0xc, - 0x9, 0x2, 0x2, 0x681, 0x685, 0x5, 0xce, 0x68, 0x2, 0x682, 0x683, 0x7, - 0xc, 0x2, 0x2, 0x683, 0x685, 0x5, 0xd0, 0x69, 0x2, 0x684, 0x681, 0x3, - 0x2, 0x2, 0x2, 0x684, 0x682, 0x3, 0x2, 0x2, 0x2, 0x685, 0x687, 0x3, - 0x2, 0x2, 0x2, 0x686, 0x641, 0x3, 0x2, 0x2, 0x2, 0x686, 0x644, 0x3, - 0x2, 0x2, 0x2, 0x686, 0x647, 0x3, 0x2, 0x2, 0x2, 0x686, 0x65d, 0x3, - 0x2, 0x2, 0x2, 0x686, 0x660, 0x3, 0x2, 0x2, 0x2, 0x686, 0x663, 0x3, - 0x2, 0x2, 0x2, 0x686, 0x66c, 0x3, 0x2, 0x2, 0x2, 0x686, 0x672, 0x3, - 0x2, 0x2, 0x2, 0x686, 0x677, 0x3, 0x2, 0x2, 0x2, 0x686, 0x67a, 0x3, - 0x2, 0x2, 0x2, 0x686, 0x680, 0x3, 0x2, 0x2, 0x2, 0x687, 0x68a, 0x3, - 0x2, 0x2, 0x2, 0x688, 0x686, 0x3, 0x2, 0x2, 0x2, 0x688, 0x689, 0x3, - 0x2, 0x2, 0x2, 0x689, 0xab, 0x3, 0x2, 0x2, 0x2, 0x68a, 0x688, 0x3, 0x2, - 0x2, 0x2, 0x68b, 0x690, 0x5, 0xae, 0x58, 0x2, 0x68c, 0x68d, 0x7, 0xc2, - 0x2, 0x2, 0x68d, 0x68f, 0x5, 0xae, 0x58, 0x2, 0x68e, 0x68c, 0x3, 0x2, - 0x2, 0x2, 0x68f, 0x692, 0x3, 0x2, 0x2, 0x2, 0x690, 0x68e, 0x3, 0x2, - 0x2, 0x2, 0x690, 0x691, 0x3, 0x2, 0x2, 0x2, 0x691, 0xad, 0x3, 0x2, 0x2, - 0x2, 0x692, 0x690, 0x3, 0x2, 0x2, 0x2, 0x693, 0x696, 0x5, 0xb0, 0x59, - 0x2, 0x694, 0x696, 0x5, 0xaa, 0x56, 0x2, 0x695, 0x693, 0x3, 0x2, 0x2, - 0x2, 0x695, 0x694, 0x3, 0x2, 0x2, 0x2, 0x696, 0xaf, 0x3, 0x2, 0x2, 0x2, - 0x697, 0x698, 0x7, 0xcd, 0x2, 0x2, 0x698, 0x69d, 0x5, 0xd0, 0x69, 0x2, - 0x699, 0x69a, 0x7, 0xc2, 0x2, 0x2, 0x69a, 0x69c, 0x5, 0xd0, 0x69, 0x2, - 0x69b, 0x699, 0x3, 0x2, 0x2, 0x2, 0x69c, 0x69f, 0x3, 0x2, 0x2, 0x2, - 0x69d, 0x69b, 0x3, 0x2, 0x2, 0x2, 0x69d, 0x69e, 0x3, 0x2, 0x2, 0x2, - 0x69e, 0x6a0, 0x3, 0x2, 0x2, 0x2, 0x69f, 0x69d, 0x3, 0x2, 0x2, 0x2, - 0x6a0, 0x6a1, 0x7, 0xd7, 0x2, 0x2, 0x6a1, 0x6ab, 0x3, 0x2, 0x2, 0x2, - 0x6a2, 0x6a7, 0x5, 0xd0, 0x69, 0x2, 0x6a3, 0x6a4, 0x7, 0xc2, 0x2, 0x2, - 0x6a4, 0x6a6, 0x5, 0xd0, 0x69, 0x2, 0x6a5, 0x6a3, 0x3, 0x2, 0x2, 0x2, - 0x6a6, 0x6a9, 0x3, 0x2, 0x2, 0x2, 0x6a7, 0x6a5, 0x3, 0x2, 0x2, 0x2, - 0x6a7, 0x6a8, 0x3, 0x2, 0x2, 0x2, 0x6a8, 0x6ab, 0x3, 0x2, 0x2, 0x2, - 0x6a9, 0x6a7, 0x3, 0x2, 0x2, 0x2, 0x6aa, 0x697, 0x3, 0x2, 0x2, 0x2, - 0x6aa, 0x6a2, 0x3, 0x2, 0x2, 0x2, 0x6ab, 0x6ac, 0x3, 0x2, 0x2, 0x2, - 0x6ac, 0x6ad, 0x7, 0xbd, 0x2, 0x2, 0x6ad, 0x6ae, 0x5, 0xaa, 0x56, 0x2, - 0x6ae, 0xb1, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x6b0, 0x5, 0xba, 0x5e, 0x2, - 0x6b0, 0x6b1, 0x7, 0xc5, 0x2, 0x2, 0x6b1, 0x6b3, 0x3, 0x2, 0x2, 0x2, - 0x6b2, 0x6af, 0x3, 0x2, 0x2, 0x2, 0x6b2, 0x6b3, 0x3, 0x2, 0x2, 0x2, - 0x6b3, 0x6b4, 0x3, 0x2, 0x2, 0x2, 0x6b4, 0x6b5, 0x5, 0xb4, 0x5b, 0x2, - 0x6b5, 0xb3, 0x3, 0x2, 0x2, 0x2, 0x6b6, 0x6b9, 0x5, 0xd0, 0x69, 0x2, - 0x6b7, 0x6b8, 0x7, 0xc5, 0x2, 0x2, 0x6b8, 0x6ba, 0x5, 0xd0, 0x69, 0x2, - 0x6b9, 0x6b7, 0x3, 0x2, 0x2, 0x2, 0x6b9, 0x6ba, 0x3, 0x2, 0x2, 0x2, - 0x6ba, 0xb5, 0x3, 0x2, 0x2, 0x2, 0x6bb, 0x6bc, 0x8, 0x5c, 0x1, 0x2, - 0x6bc, 0x6c3, 0x5, 0xba, 0x5e, 0x2, 0x6bd, 0x6c3, 0x5, 0xb8, 0x5d, 0x2, - 0x6be, 0x6bf, 0x7, 0xcd, 0x2, 0x2, 0x6bf, 0x6c0, 0x5, 0x64, 0x33, 0x2, - 0x6c0, 0x6c1, 0x7, 0xd7, 0x2, 0x2, 0x6c1, 0x6c3, 0x3, 0x2, 0x2, 0x2, - 0x6c2, 0x6bb, 0x3, 0x2, 0x2, 0x2, 0x6c2, 0x6bd, 0x3, 0x2, 0x2, 0x2, - 0x6c2, 0x6be, 0x3, 0x2, 0x2, 0x2, 0x6c3, 0x6cc, 0x3, 0x2, 0x2, 0x2, - 0x6c4, 0x6c8, 0xc, 0x3, 0x2, 0x2, 0x6c5, 0x6c9, 0x5, 0xce, 0x68, 0x2, - 0x6c6, 0x6c7, 0x7, 0xc, 0x2, 0x2, 0x6c7, 0x6c9, 0x5, 0xd0, 0x69, 0x2, - 0x6c8, 0x6c5, 0x3, 0x2, 0x2, 0x2, 0x6c8, 0x6c6, 0x3, 0x2, 0x2, 0x2, - 0x6c9, 0x6cb, 0x3, 0x2, 0x2, 0x2, 0x6ca, 0x6c4, 0x3, 0x2, 0x2, 0x2, - 0x6cb, 0x6ce, 0x3, 0x2, 0x2, 0x2, 0x6cc, 0x6ca, 0x3, 0x2, 0x2, 0x2, - 0x6cc, 0x6cd, 0x3, 0x2, 0x2, 0x2, 0x6cd, 0xb7, 0x3, 0x2, 0x2, 0x2, 0x6ce, - 0x6cc, 0x3, 0x2, 0x2, 0x2, 0x6cf, 0x6d0, 0x5, 0xd0, 0x69, 0x2, 0x6d0, - 0x6d2, 0x7, 0xcd, 0x2, 0x2, 0x6d1, 0x6d3, 0x5, 0xbc, 0x5f, 0x2, 0x6d2, - 0x6d1, 0x3, 0x2, 0x2, 0x2, 0x6d2, 0x6d3, 0x3, 0x2, 0x2, 0x2, 0x6d3, - 0x6d4, 0x3, 0x2, 0x2, 0x2, 0x6d4, 0x6d5, 0x7, 0xd7, 0x2, 0x2, 0x6d5, - 0xb9, 0x3, 0x2, 0x2, 0x2, 0x6d6, 0x6d7, 0x5, 0xc0, 0x61, 0x2, 0x6d7, - 0x6d8, 0x7, 0xc5, 0x2, 0x2, 0x6d8, 0x6da, 0x3, 0x2, 0x2, 0x2, 0x6d9, - 0x6d6, 0x3, 0x2, 0x2, 0x2, 0x6d9, 0x6da, 0x3, 0x2, 0x2, 0x2, 0x6da, - 0x6db, 0x3, 0x2, 0x2, 0x2, 0x6db, 0x6dc, 0x5, 0xd0, 0x69, 0x2, 0x6dc, - 0xbb, 0x3, 0x2, 0x2, 0x2, 0x6dd, 0x6e2, 0x5, 0xbe, 0x60, 0x2, 0x6de, - 0x6df, 0x7, 0xc2, 0x2, 0x2, 0x6df, 0x6e1, 0x5, 0xbe, 0x60, 0x2, 0x6e0, - 0x6de, 0x3, 0x2, 0x2, 0x2, 0x6e1, 0x6e4, 0x3, 0x2, 0x2, 0x2, 0x6e2, - 0x6e0, 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6e3, 0x3, 0x2, 0x2, 0x2, 0x6e3, - 0xbd, 0x3, 0x2, 0x2, 0x2, 0x6e4, 0x6e2, 0x3, 0x2, 0x2, 0x2, 0x6e5, 0x6e9, - 0x5, 0xba, 0x5e, 0x2, 0x6e6, 0x6e9, 0x5, 0xb8, 0x5d, 0x2, 0x6e7, 0x6e9, - 0x5, 0xc6, 0x64, 0x2, 0x6e8, 0x6e5, 0x3, 0x2, 0x2, 0x2, 0x6e8, 0x6e6, - 0x3, 0x2, 0x2, 0x2, 0x6e8, 0x6e7, 0x3, 0x2, 0x2, 0x2, 0x6e9, 0xbf, 0x3, - 0x2, 0x2, 0x2, 0x6ea, 0x6eb, 0x5, 0xd0, 0x69, 0x2, 0x6eb, 0xc1, 0x3, - 0x2, 0x2, 0x2, 0x6ec, 0x6f5, 0x7, 0xb8, 0x2, 0x2, 0x6ed, 0x6ee, 0x7, - 0xc5, 0x2, 0x2, 0x6ee, 0x6f5, 0x9, 0x18, 0x2, 0x2, 0x6ef, 0x6f0, 0x7, - 0xba, 0x2, 0x2, 0x6f0, 0x6f2, 0x7, 0xc5, 0x2, 0x2, 0x6f1, 0x6f3, 0x9, - 0x18, 0x2, 0x2, 0x6f2, 0x6f1, 0x3, 0x2, 0x2, 0x2, 0x6f2, 0x6f3, 0x3, - 0x2, 0x2, 0x2, 0x6f3, 0x6f5, 0x3, 0x2, 0x2, 0x2, 0x6f4, 0x6ec, 0x3, - 0x2, 0x2, 0x2, 0x6f4, 0x6ed, 0x3, 0x2, 0x2, 0x2, 0x6f4, 0x6ef, 0x3, - 0x2, 0x2, 0x2, 0x6f5, 0xc3, 0x3, 0x2, 0x2, 0x2, 0x6f6, 0x6f8, 0x9, 0x19, - 0x2, 0x2, 0x6f7, 0x6f6, 0x3, 0x2, 0x2, 0x2, 0x6f7, 0x6f8, 0x3, 0x2, - 0x2, 0x2, 0x6f8, 0x6ff, 0x3, 0x2, 0x2, 0x2, 0x6f9, 0x700, 0x5, 0xc2, - 0x62, 0x2, 0x6fa, 0x700, 0x7, 0xb9, 0x2, 0x2, 0x6fb, 0x700, 0x7, 0xba, - 0x2, 0x2, 0x6fc, 0x700, 0x7, 0xbb, 0x2, 0x2, 0x6fd, 0x700, 0x7, 0x50, - 0x2, 0x2, 0x6fe, 0x700, 0x7, 0x6e, 0x2, 0x2, 0x6ff, 0x6f9, 0x3, 0x2, - 0x2, 0x2, 0x6ff, 0x6fa, 0x3, 0x2, 0x2, 0x2, 0x6ff, 0x6fb, 0x3, 0x2, - 0x2, 0x2, 0x6ff, 0x6fc, 0x3, 0x2, 0x2, 0x2, 0x6ff, 0x6fd, 0x3, 0x2, - 0x2, 0x2, 0x6ff, 0x6fe, 0x3, 0x2, 0x2, 0x2, 0x700, 0xc5, 0x3, 0x2, 0x2, - 0x2, 0x701, 0x705, 0x5, 0xc4, 0x63, 0x2, 0x702, 0x705, 0x7, 0xbc, 0x2, - 0x2, 0x703, 0x705, 0x7, 0x71, 0x2, 0x2, 0x704, 0x701, 0x3, 0x2, 0x2, - 0x2, 0x704, 0x702, 0x3, 0x2, 0x2, 0x2, 0x704, 0x703, 0x3, 0x2, 0x2, - 0x2, 0x705, 0xc7, 0x3, 0x2, 0x2, 0x2, 0x706, 0x707, 0x9, 0x1a, 0x2, - 0x2, 0x707, 0xc9, 0x3, 0x2, 0x2, 0x2, 0x708, 0x709, 0x9, 0x1b, 0x2, - 0x2, 0x709, 0xcb, 0x3, 0x2, 0x2, 0x2, 0x70a, 0x70b, 0x9, 0x1c, 0x2, - 0x2, 0x70b, 0xcd, 0x3, 0x2, 0x2, 0x2, 0x70c, 0x70f, 0x7, 0xb7, 0x2, - 0x2, 0x70d, 0x70f, 0x5, 0xcc, 0x67, 0x2, 0x70e, 0x70c, 0x3, 0x2, 0x2, - 0x2, 0x70e, 0x70d, 0x3, 0x2, 0x2, 0x2, 0x70f, 0xcf, 0x3, 0x2, 0x2, 0x2, - 0x710, 0x714, 0x7, 0xb7, 0x2, 0x2, 0x711, 0x714, 0x5, 0xc8, 0x65, 0x2, - 0x712, 0x714, 0x5, 0xca, 0x66, 0x2, 0x713, 0x710, 0x3, 0x2, 0x2, 0x2, - 0x713, 0x711, 0x3, 0x2, 0x2, 0x2, 0x713, 0x712, 0x3, 0x2, 0x2, 0x2, - 0x714, 0xd1, 0x3, 0x2, 0x2, 0x2, 0x715, 0x718, 0x5, 0xd0, 0x69, 0x2, - 0x716, 0x718, 0x7, 0x71, 0x2, 0x2, 0x717, 0x715, 0x3, 0x2, 0x2, 0x2, - 0x717, 0x716, 0x3, 0x2, 0x2, 0x2, 0x718, 0xd3, 0x3, 0x2, 0x2, 0x2, 0x719, - 0x71a, 0x7, 0xbc, 0x2, 0x2, 0x71a, 0x71b, 0x7, 0xc7, 0x2, 0x2, 0x71b, - 0x71c, 0x5, 0xc4, 0x63, 0x2, 0x71c, 0xd5, 0x3, 0x2, 0x2, 0x2, 0xf4, - 0xda, 0xde, 0xe1, 0xe4, 0xf8, 0xfe, 0x105, 0x10d, 0x112, 0x119, 0x11e, - 0x124, 0x12a, 0x12f, 0x135, 0x143, 0x14a, 0x151, 0x157, 0x160, 0x16a, - 0x174, 0x188, 0x190, 0x19f, 0x1a6, 0x1b4, 0x1ba, 0x1c0, 0x1c7, 0x1cb, - 0x1ce, 0x1d5, 0x1d9, 0x1dc, 0x1e7, 0x1eb, 0x1ee, 0x1f3, 0x1f5, 0x1f8, - 0x1fb, 0x205, 0x209, 0x20c, 0x20f, 0x214, 0x216, 0x21c, 0x222, 0x226, - 0x229, 0x22c, 0x22f, 0x232, 0x237, 0x23d, 0x241, 0x244, 0x247, 0x24b, - 0x253, 0x26d, 0x26f, 0x273, 0x289, 0x28b, 0x296, 0x299, 0x2a2, 0x2b3, - 0x2be, 0x2d0, 0x2dd, 0x2ee, 0x2f7, 0x312, 0x314, 0x329, 0x32e, 0x333, - 0x336, 0x340, 0x345, 0x349, 0x34c, 0x350, 0x354, 0x359, 0x35c, 0x360, - 0x362, 0x375, 0x37d, 0x380, 0x38a, 0x38e, 0x396, 0x39a, 0x39f, 0x3a3, - 0x3a7, 0x3ab, 0x3af, 0x3b1, 0x3b9, 0x3bd, 0x3c0, 0x3c9, 0x3ce, 0x3d1, - 0x3db, 0x3e5, 0x3e9, 0x3ee, 0x3f2, 0x3f8, 0x3fb, 0x3fe, 0x401, 0x40f, - 0x413, 0x41b, 0x423, 0x426, 0x42a, 0x42d, 0x431, 0x434, 0x437, 0x43a, - 0x43d, 0x441, 0x445, 0x448, 0x44b, 0x44e, 0x451, 0x454, 0x45d, 0x463, - 0x477, 0x489, 0x491, 0x494, 0x49a, 0x4a2, 0x4a5, 0x4ab, 0x4ad, 0x4b1, - 0x4b6, 0x4b9, 0x4bc, 0x4c0, 0x4c4, 0x4c7, 0x4c9, 0x4cc, 0x4d0, 0x4d4, - 0x4d7, 0x4d9, 0x4db, 0x4de, 0x4e3, 0x4ee, 0x4f4, 0x4f9, 0x500, 0x505, - 0x509, 0x50d, 0x512, 0x519, 0x52e, 0x531, 0x53a, 0x53e, 0x543, 0x548, - 0x54b, 0x54d, 0x563, 0x566, 0x571, 0x575, 0x578, 0x57c, 0x580, 0x588, - 0x58c, 0x599, 0x5a5, 0x5b1, 0x5b9, 0x5bd, 0x5c4, 0x5ca, 0x5d2, 0x5d7, - 0x5e0, 0x5e4, 0x603, 0x614, 0x617, 0x61b, 0x61e, 0x62a, 0x63b, 0x63f, - 0x650, 0x653, 0x657, 0x65a, 0x665, 0x67d, 0x684, 0x686, 0x688, 0x690, - 0x695, 0x69d, 0x6a7, 0x6aa, 0x6b2, 0x6b9, 0x6c2, 0x6c8, 0x6cc, 0x6d2, - 0x6d9, 0x6e2, 0x6e8, 0x6f2, 0x6f4, 0x6f7, 0x6ff, 0x704, 0x70e, 0x713, - 0x717, + 0x2, 0x44c, 0x63, 0x3, 0x2, 0x2, 0x2, 0x44d, 0x44e, 0x7, 0x84, 0x2, + 0x2, 0x44e, 0x44f, 0x7, 0x99, 0x2, 0x2, 0x44f, 0x450, 0x5, 0xc0, 0x61, + 0x2, 0x450, 0x451, 0x7, 0xa1, 0x2, 0x2, 0x451, 0x459, 0x5, 0xc0, 0x61, + 0x2, 0x452, 0x453, 0x7, 0xc4, 0x2, 0x2, 0x453, 0x454, 0x5, 0xc0, 0x61, + 0x2, 0x454, 0x455, 0x7, 0xa1, 0x2, 0x2, 0x455, 0x456, 0x5, 0xc0, 0x61, + 0x2, 0x456, 0x458, 0x3, 0x2, 0x2, 0x2, 0x457, 0x452, 0x3, 0x2, 0x2, + 0x2, 0x458, 0x45b, 0x3, 0x2, 0x2, 0x2, 0x459, 0x457, 0x3, 0x2, 0x2, + 0x2, 0x459, 0x45a, 0x3, 0x2, 0x2, 0x2, 0x45a, 0x45d, 0x3, 0x2, 0x2, + 0x2, 0x45b, 0x459, 0x3, 0x2, 0x2, 0x2, 0x45c, 0x45e, 0x5, 0x2c, 0x17, + 0x2, 0x45d, 0x45c, 0x3, 0x2, 0x2, 0x2, 0x45d, 0x45e, 0x3, 0x2, 0x2, + 0x2, 0x45e, 0x65, 0x3, 0x2, 0x2, 0x2, 0x45f, 0x461, 0x7, 0xcf, 0x2, + 0x2, 0x460, 0x462, 0x5, 0x6e, 0x38, 0x2, 0x461, 0x460, 0x3, 0x2, 0x2, + 0x2, 0x461, 0x462, 0x3, 0x2, 0x2, 0x2, 0x462, 0x463, 0x3, 0x2, 0x2, + 0x2, 0x463, 0x464, 0x7, 0x8c, 0x2, 0x2, 0x464, 0x466, 0x5, 0xac, 0x57, + 0x2, 0x465, 0x467, 0x5, 0x7a, 0x3e, 0x2, 0x466, 0x465, 0x3, 0x2, 0x2, + 0x2, 0x466, 0x467, 0x3, 0x2, 0x2, 0x2, 0x467, 0x469, 0x3, 0x2, 0x2, + 0x2, 0x468, 0x46a, 0x5, 0x80, 0x41, 0x2, 0x469, 0x468, 0x3, 0x2, 0x2, + 0x2, 0x469, 0x46a, 0x3, 0x2, 0x2, 0x2, 0x46a, 0x46b, 0x3, 0x2, 0x2, + 0x2, 0x46b, 0x46c, 0x7, 0xd9, 0x2, 0x2, 0x46c, 0x67, 0x3, 0x2, 0x2, + 0x2, 0x46d, 0x473, 0x5, 0x6a, 0x36, 0x2, 0x46e, 0x46f, 0x7, 0xa9, 0x2, + 0x2, 0x46f, 0x470, 0x7, 0x6, 0x2, 0x2, 0x470, 0x472, 0x5, 0x6a, 0x36, + 0x2, 0x471, 0x46e, 0x3, 0x2, 0x2, 0x2, 0x472, 0x475, 0x3, 0x2, 0x2, + 0x2, 0x473, 0x471, 0x3, 0x2, 0x2, 0x2, 0x473, 0x474, 0x3, 0x2, 0x2, + 0x2, 0x474, 0x69, 0x3, 0x2, 0x2, 0x2, 0x475, 0x473, 0x3, 0x2, 0x2, 0x2, + 0x476, 0x47c, 0x5, 0x6c, 0x37, 0x2, 0x477, 0x478, 0x7, 0xcf, 0x2, 0x2, + 0x478, 0x479, 0x5, 0x68, 0x35, 0x2, 0x479, 0x47a, 0x7, 0xd9, 0x2, 0x2, + 0x47a, 0x47c, 0x3, 0x2, 0x2, 0x2, 0x47b, 0x476, 0x3, 0x2, 0x2, 0x2, + 0x47b, 0x477, 0x3, 0x2, 0x2, 0x2, 0x47c, 0x6b, 0x3, 0x2, 0x2, 0x2, 0x47d, + 0x47f, 0x5, 0x6e, 0x38, 0x2, 0x47e, 0x47d, 0x3, 0x2, 0x2, 0x2, 0x47e, + 0x47f, 0x3, 0x2, 0x2, 0x2, 0x47f, 0x480, 0x3, 0x2, 0x2, 0x2, 0x480, + 0x482, 0x7, 0x8c, 0x2, 0x2, 0x481, 0x483, 0x7, 0x30, 0x2, 0x2, 0x482, + 0x481, 0x3, 0x2, 0x2, 0x2, 0x482, 0x483, 0x3, 0x2, 0x2, 0x2, 0x483, + 0x485, 0x3, 0x2, 0x2, 0x2, 0x484, 0x486, 0x5, 0x70, 0x39, 0x2, 0x485, + 0x484, 0x3, 0x2, 0x2, 0x2, 0x485, 0x486, 0x3, 0x2, 0x2, 0x2, 0x486, + 0x487, 0x3, 0x2, 0x2, 0x2, 0x487, 0x489, 0x5, 0xac, 0x57, 0x2, 0x488, + 0x48a, 0x5, 0x72, 0x3a, 0x2, 0x489, 0x488, 0x3, 0x2, 0x2, 0x2, 0x489, + 0x48a, 0x3, 0x2, 0x2, 0x2, 0x48a, 0x48c, 0x3, 0x2, 0x2, 0x2, 0x48b, + 0x48d, 0x5, 0x74, 0x3b, 0x2, 0x48c, 0x48b, 0x3, 0x2, 0x2, 0x2, 0x48c, + 0x48d, 0x3, 0x2, 0x2, 0x2, 0x48d, 0x48f, 0x3, 0x2, 0x2, 0x2, 0x48e, + 0x490, 0x5, 0x76, 0x3c, 0x2, 0x48f, 0x48e, 0x3, 0x2, 0x2, 0x2, 0x48f, + 0x490, 0x3, 0x2, 0x2, 0x2, 0x490, 0x492, 0x3, 0x2, 0x2, 0x2, 0x491, + 0x493, 0x5, 0x78, 0x3d, 0x2, 0x492, 0x491, 0x3, 0x2, 0x2, 0x2, 0x492, + 0x493, 0x3, 0x2, 0x2, 0x2, 0x493, 0x495, 0x3, 0x2, 0x2, 0x2, 0x494, + 0x496, 0x5, 0x7a, 0x3e, 0x2, 0x495, 0x494, 0x3, 0x2, 0x2, 0x2, 0x495, + 0x496, 0x3, 0x2, 0x2, 0x2, 0x496, 0x499, 0x3, 0x2, 0x2, 0x2, 0x497, + 0x498, 0x7, 0xb5, 0x2, 0x2, 0x498, 0x49a, 0x9, 0x8, 0x2, 0x2, 0x499, + 0x497, 0x3, 0x2, 0x2, 0x2, 0x499, 0x49a, 0x3, 0x2, 0x2, 0x2, 0x49a, + 0x49d, 0x3, 0x2, 0x2, 0x2, 0x49b, 0x49c, 0x7, 0xb5, 0x2, 0x2, 0x49c, + 0x49e, 0x7, 0xa3, 0x2, 0x2, 0x49d, 0x49b, 0x3, 0x2, 0x2, 0x2, 0x49d, + 0x49e, 0x3, 0x2, 0x2, 0x2, 0x49e, 0x4a0, 0x3, 0x2, 0x2, 0x2, 0x49f, + 0x4a1, 0x5, 0x7c, 0x3f, 0x2, 0x4a0, 0x49f, 0x3, 0x2, 0x2, 0x2, 0x4a0, + 0x4a1, 0x3, 0x2, 0x2, 0x2, 0x4a1, 0x4a3, 0x3, 0x2, 0x2, 0x2, 0x4a2, + 0x4a4, 0x5, 0x7e, 0x40, 0x2, 0x4a3, 0x4a2, 0x3, 0x2, 0x2, 0x2, 0x4a3, + 0x4a4, 0x3, 0x2, 0x2, 0x2, 0x4a4, 0x4a6, 0x3, 0x2, 0x2, 0x2, 0x4a5, + 0x4a7, 0x5, 0x82, 0x42, 0x2, 0x4a6, 0x4a5, 0x3, 0x2, 0x2, 0x2, 0x4a6, + 0x4a7, 0x3, 0x2, 0x2, 0x2, 0x4a7, 0x4a9, 0x3, 0x2, 0x2, 0x2, 0x4a8, + 0x4aa, 0x5, 0x84, 0x43, 0x2, 0x4a9, 0x4a8, 0x3, 0x2, 0x2, 0x2, 0x4a9, + 0x4aa, 0x3, 0x2, 0x2, 0x2, 0x4aa, 0x4ac, 0x3, 0x2, 0x2, 0x2, 0x4ab, + 0x4ad, 0x5, 0x86, 0x44, 0x2, 0x4ac, 0x4ab, 0x3, 0x2, 0x2, 0x2, 0x4ac, + 0x4ad, 0x3, 0x2, 0x2, 0x2, 0x4ad, 0x6d, 0x3, 0x2, 0x2, 0x2, 0x4ae, 0x4af, + 0x7, 0xb5, 0x2, 0x2, 0x4af, 0x4b0, 0x5, 0xac, 0x57, 0x2, 0x4b0, 0x6f, + 0x3, 0x2, 0x2, 0x2, 0x4b1, 0x4b2, 0x7, 0xa2, 0x2, 0x2, 0x4b2, 0x4b5, + 0x7, 0xbc, 0x2, 0x2, 0x4b3, 0x4b4, 0x7, 0xb5, 0x2, 0x2, 0x4b4, 0x4b6, + 0x7, 0x9e, 0x2, 0x2, 0x4b5, 0x4b3, 0x3, 0x2, 0x2, 0x2, 0x4b5, 0x4b6, + 0x3, 0x2, 0x2, 0x2, 0x4b6, 0x71, 0x3, 0x2, 0x2, 0x2, 0x4b7, 0x4b8, 0x7, + 0x42, 0x2, 0x2, 0x4b8, 0x4b9, 0x5, 0x88, 0x45, 0x2, 0x4b9, 0x73, 0x3, + 0x2, 0x2, 0x2, 0x4ba, 0x4bc, 0x9, 0x9, 0x2, 0x2, 0x4bb, 0x4ba, 0x3, + 0x2, 0x2, 0x2, 0x4bb, 0x4bc, 0x3, 0x2, 0x2, 0x2, 0x4bc, 0x4bd, 0x3, + 0x2, 0x2, 0x2, 0x4bd, 0x4be, 0x7, 0xb, 0x2, 0x2, 0x4be, 0x4bf, 0x7, + 0x58, 0x2, 0x2, 0x4bf, 0x4c0, 0x5, 0xac, 0x57, 0x2, 0x4c0, 0x75, 0x3, + 0x2, 0x2, 0x2, 0x4c1, 0x4c2, 0x7, 0x7d, 0x2, 0x2, 0x4c2, 0x4c3, 0x5, + 0xb0, 0x59, 0x2, 0x4c3, 0x77, 0x3, 0x2, 0x2, 0x2, 0x4c4, 0x4c5, 0x7, + 0xb4, 0x2, 0x2, 0x4c5, 0x4c6, 0x5, 0xb0, 0x59, 0x2, 0x4c6, 0x79, 0x3, + 0x2, 0x2, 0x2, 0x4c7, 0x4c8, 0x7, 0x47, 0x2, 0x2, 0x4c8, 0x4cf, 0x7, + 0x13, 0x2, 0x2, 0x4c9, 0x4ca, 0x9, 0x8, 0x2, 0x2, 0x4ca, 0x4cb, 0x7, + 0xcf, 0x2, 0x2, 0x4cb, 0x4cc, 0x5, 0xac, 0x57, 0x2, 0x4cc, 0x4cd, 0x7, + 0xd9, 0x2, 0x2, 0x4cd, 0x4d0, 0x3, 0x2, 0x2, 0x2, 0x4ce, 0x4d0, 0x5, + 0xac, 0x57, 0x2, 0x4cf, 0x4c9, 0x3, 0x2, 0x2, 0x2, 0x4cf, 0x4ce, 0x3, + 0x2, 0x2, 0x2, 0x4d0, 0x7b, 0x3, 0x2, 0x2, 0x2, 0x4d1, 0x4d2, 0x7, 0x48, + 0x2, 0x2, 0x4d2, 0x4d3, 0x5, 0xb0, 0x59, 0x2, 0x4d3, 0x7d, 0x3, 0x2, + 0x2, 0x2, 0x4d4, 0x4d5, 0x7, 0x78, 0x2, 0x2, 0x4d5, 0x4d6, 0x7, 0x13, + 0x2, 0x2, 0x4d6, 0x4d7, 0x5, 0x94, 0x4b, 0x2, 0x4d7, 0x7f, 0x3, 0x2, + 0x2, 0x2, 0x4d8, 0x4d9, 0x7, 0x78, 0x2, 0x2, 0x4d9, 0x4da, 0x7, 0x13, + 0x2, 0x2, 0x4da, 0x4db, 0x5, 0xac, 0x57, 0x2, 0x4db, 0x81, 0x3, 0x2, + 0x2, 0x2, 0x4dc, 0x4dd, 0x7, 0x61, 0x2, 0x2, 0x4dd, 0x4de, 0x5, 0x92, + 0x4a, 0x2, 0x4de, 0x4df, 0x7, 0x13, 0x2, 0x2, 0x4df, 0x4e0, 0x5, 0xac, + 0x57, 0x2, 0x4e0, 0x83, 0x3, 0x2, 0x2, 0x2, 0x4e1, 0x4e2, 0x7, 0x61, + 0x2, 0x2, 0x4e2, 0x4e5, 0x5, 0x92, 0x4a, 0x2, 0x4e3, 0x4e4, 0x7, 0xb5, + 0x2, 0x2, 0x4e4, 0x4e6, 0x7, 0x9e, 0x2, 0x2, 0x4e5, 0x4e3, 0x3, 0x2, + 0x2, 0x2, 0x4e5, 0x4e6, 0x3, 0x2, 0x2, 0x2, 0x4e6, 0x85, 0x3, 0x2, 0x2, + 0x2, 0x4e7, 0x4e8, 0x7, 0x90, 0x2, 0x2, 0x4e8, 0x4e9, 0x5, 0x9a, 0x4e, + 0x2, 0x4e9, 0x87, 0x3, 0x2, 0x2, 0x2, 0x4ea, 0x4eb, 0x8, 0x45, 0x1, + 0x2, 0x4eb, 0x4ed, 0x5, 0xbc, 0x5f, 0x2, 0x4ec, 0x4ee, 0x7, 0x3c, 0x2, + 0x2, 0x4ed, 0x4ec, 0x3, 0x2, 0x2, 0x2, 0x4ed, 0x4ee, 0x3, 0x2, 0x2, + 0x2, 0x4ee, 0x4f0, 0x3, 0x2, 0x2, 0x2, 0x4ef, 0x4f1, 0x5, 0x90, 0x49, + 0x2, 0x4f0, 0x4ef, 0x3, 0x2, 0x2, 0x2, 0x4f0, 0x4f1, 0x3, 0x2, 0x2, + 0x2, 0x4f1, 0x4f7, 0x3, 0x2, 0x2, 0x2, 0x4f2, 0x4f3, 0x7, 0xcf, 0x2, + 0x2, 0x4f3, 0x4f4, 0x5, 0x88, 0x45, 0x2, 0x4f4, 0x4f5, 0x7, 0xd9, 0x2, + 0x2, 0x4f5, 0x4f7, 0x3, 0x2, 0x2, 0x2, 0x4f6, 0x4ea, 0x3, 0x2, 0x2, + 0x2, 0x4f6, 0x4f2, 0x3, 0x2, 0x2, 0x2, 0x4f7, 0x509, 0x3, 0x2, 0x2, + 0x2, 0x4f8, 0x4f9, 0xc, 0x5, 0x2, 0x2, 0x4f9, 0x4fa, 0x5, 0x8c, 0x47, + 0x2, 0x4fa, 0x4fb, 0x5, 0x88, 0x45, 0x6, 0x4fb, 0x508, 0x3, 0x2, 0x2, + 0x2, 0x4fc, 0x4fe, 0xc, 0x6, 0x2, 0x2, 0x4fd, 0x4ff, 0x9, 0xa, 0x2, + 0x2, 0x4fe, 0x4fd, 0x3, 0x2, 0x2, 0x2, 0x4fe, 0x4ff, 0x3, 0x2, 0x2, + 0x2, 0x4ff, 0x501, 0x3, 0x2, 0x2, 0x2, 0x500, 0x502, 0x5, 0x8a, 0x46, + 0x2, 0x501, 0x500, 0x3, 0x2, 0x2, 0x2, 0x501, 0x502, 0x3, 0x2, 0x2, + 0x2, 0x502, 0x503, 0x3, 0x2, 0x2, 0x2, 0x503, 0x504, 0x7, 0x58, 0x2, + 0x2, 0x504, 0x505, 0x5, 0x88, 0x45, 0x2, 0x505, 0x506, 0x5, 0x8e, 0x48, + 0x2, 0x506, 0x508, 0x3, 0x2, 0x2, 0x2, 0x507, 0x4f8, 0x3, 0x2, 0x2, + 0x2, 0x507, 0x4fc, 0x3, 0x2, 0x2, 0x2, 0x508, 0x50b, 0x3, 0x2, 0x2, + 0x2, 0x509, 0x507, 0x3, 0x2, 0x2, 0x2, 0x509, 0x50a, 0x3, 0x2, 0x2, + 0x2, 0x50a, 0x89, 0x3, 0x2, 0x2, 0x2, 0x50b, 0x509, 0x3, 0x2, 0x2, 0x2, + 0x50c, 0x50e, 0x9, 0xb, 0x2, 0x2, 0x50d, 0x50c, 0x3, 0x2, 0x2, 0x2, + 0x50d, 0x50e, 0x3, 0x2, 0x2, 0x2, 0x50e, 0x50f, 0x3, 0x2, 0x2, 0x2, + 0x50f, 0x516, 0x7, 0x52, 0x2, 0x2, 0x510, 0x512, 0x7, 0x52, 0x2, 0x2, + 0x511, 0x513, 0x9, 0xb, 0x2, 0x2, 0x512, 0x511, 0x3, 0x2, 0x2, 0x2, + 0x512, 0x513, 0x3, 0x2, 0x2, 0x2, 0x513, 0x516, 0x3, 0x2, 0x2, 0x2, + 0x514, 0x516, 0x9, 0xb, 0x2, 0x2, 0x515, 0x50d, 0x3, 0x2, 0x2, 0x2, + 0x515, 0x510, 0x3, 0x2, 0x2, 0x2, 0x515, 0x514, 0x3, 0x2, 0x2, 0x2, + 0x516, 0x538, 0x3, 0x2, 0x2, 0x2, 0x517, 0x519, 0x9, 0xc, 0x2, 0x2, + 0x518, 0x517, 0x3, 0x2, 0x2, 0x2, 0x518, 0x519, 0x3, 0x2, 0x2, 0x2, + 0x519, 0x51a, 0x3, 0x2, 0x2, 0x2, 0x51a, 0x51c, 0x9, 0xd, 0x2, 0x2, + 0x51b, 0x51d, 0x7, 0x79, 0x2, 0x2, 0x51c, 0x51b, 0x3, 0x2, 0x2, 0x2, + 0x51c, 0x51d, 0x3, 0x2, 0x2, 0x2, 0x51d, 0x526, 0x3, 0x2, 0x2, 0x2, + 0x51e, 0x520, 0x9, 0xd, 0x2, 0x2, 0x51f, 0x521, 0x7, 0x79, 0x2, 0x2, + 0x520, 0x51f, 0x3, 0x2, 0x2, 0x2, 0x520, 0x521, 0x3, 0x2, 0x2, 0x2, + 0x521, 0x523, 0x3, 0x2, 0x2, 0x2, 0x522, 0x524, 0x9, 0xc, 0x2, 0x2, + 0x523, 0x522, 0x3, 0x2, 0x2, 0x2, 0x523, 0x524, 0x3, 0x2, 0x2, 0x2, + 0x524, 0x526, 0x3, 0x2, 0x2, 0x2, 0x525, 0x518, 0x3, 0x2, 0x2, 0x2, + 0x525, 0x51e, 0x3, 0x2, 0x2, 0x2, 0x526, 0x538, 0x3, 0x2, 0x2, 0x2, + 0x527, 0x529, 0x9, 0xe, 0x2, 0x2, 0x528, 0x527, 0x3, 0x2, 0x2, 0x2, + 0x528, 0x529, 0x3, 0x2, 0x2, 0x2, 0x529, 0x52a, 0x3, 0x2, 0x2, 0x2, + 0x52a, 0x52c, 0x7, 0x43, 0x2, 0x2, 0x52b, 0x52d, 0x7, 0x79, 0x2, 0x2, + 0x52c, 0x52b, 0x3, 0x2, 0x2, 0x2, 0x52c, 0x52d, 0x3, 0x2, 0x2, 0x2, + 0x52d, 0x536, 0x3, 0x2, 0x2, 0x2, 0x52e, 0x530, 0x7, 0x43, 0x2, 0x2, + 0x52f, 0x531, 0x7, 0x79, 0x2, 0x2, 0x530, 0x52f, 0x3, 0x2, 0x2, 0x2, + 0x530, 0x531, 0x3, 0x2, 0x2, 0x2, 0x531, 0x533, 0x3, 0x2, 0x2, 0x2, + 0x532, 0x534, 0x9, 0xe, 0x2, 0x2, 0x533, 0x532, 0x3, 0x2, 0x2, 0x2, + 0x533, 0x534, 0x3, 0x2, 0x2, 0x2, 0x534, 0x536, 0x3, 0x2, 0x2, 0x2, + 0x535, 0x528, 0x3, 0x2, 0x2, 0x2, 0x535, 0x52e, 0x3, 0x2, 0x2, 0x2, + 0x536, 0x538, 0x3, 0x2, 0x2, 0x2, 0x537, 0x515, 0x3, 0x2, 0x2, 0x2, + 0x537, 0x525, 0x3, 0x2, 0x2, 0x2, 0x537, 0x535, 0x3, 0x2, 0x2, 0x2, + 0x538, 0x8b, 0x3, 0x2, 0x2, 0x2, 0x539, 0x53b, 0x9, 0xa, 0x2, 0x2, 0x53a, + 0x539, 0x3, 0x2, 0x2, 0x2, 0x53a, 0x53b, 0x3, 0x2, 0x2, 0x2, 0x53b, + 0x53c, 0x3, 0x2, 0x2, 0x2, 0x53c, 0x53d, 0x7, 0x1f, 0x2, 0x2, 0x53d, + 0x540, 0x7, 0x58, 0x2, 0x2, 0x53e, 0x540, 0x7, 0xc4, 0x2, 0x2, 0x53f, + 0x53a, 0x3, 0x2, 0x2, 0x2, 0x53f, 0x53e, 0x3, 0x2, 0x2, 0x2, 0x540, + 0x8d, 0x3, 0x2, 0x2, 0x2, 0x541, 0x542, 0x7, 0x75, 0x2, 0x2, 0x542, + 0x54b, 0x5, 0xac, 0x57, 0x2, 0x543, 0x544, 0x7, 0xac, 0x2, 0x2, 0x544, + 0x545, 0x7, 0xcf, 0x2, 0x2, 0x545, 0x546, 0x5, 0xac, 0x57, 0x2, 0x546, + 0x547, 0x7, 0xd9, 0x2, 0x2, 0x547, 0x54b, 0x3, 0x2, 0x2, 0x2, 0x548, + 0x549, 0x7, 0xac, 0x2, 0x2, 0x549, 0x54b, 0x5, 0xac, 0x57, 0x2, 0x54a, + 0x541, 0x3, 0x2, 0x2, 0x2, 0x54a, 0x543, 0x3, 0x2, 0x2, 0x2, 0x54a, + 0x548, 0x3, 0x2, 0x2, 0x2, 0x54b, 0x8f, 0x3, 0x2, 0x2, 0x2, 0x54c, 0x54d, + 0x7, 0x8a, 0x2, 0x2, 0x54d, 0x550, 0x5, 0x98, 0x4d, 0x2, 0x54e, 0x54f, + 0x7, 0x74, 0x2, 0x2, 0x54f, 0x551, 0x5, 0x98, 0x4d, 0x2, 0x550, 0x54e, + 0x3, 0x2, 0x2, 0x2, 0x550, 0x551, 0x3, 0x2, 0x2, 0x2, 0x551, 0x91, 0x3, + 0x2, 0x2, 0x2, 0x552, 0x555, 0x5, 0xb0, 0x59, 0x2, 0x553, 0x554, 0x9, + 0xf, 0x2, 0x2, 0x554, 0x556, 0x5, 0xb0, 0x59, 0x2, 0x555, 0x553, 0x3, + 0x2, 0x2, 0x2, 0x555, 0x556, 0x3, 0x2, 0x2, 0x2, 0x556, 0x93, 0x3, 0x2, + 0x2, 0x2, 0x557, 0x55c, 0x5, 0x96, 0x4c, 0x2, 0x558, 0x559, 0x7, 0xc4, + 0x2, 0x2, 0x559, 0x55b, 0x5, 0x96, 0x4c, 0x2, 0x55a, 0x558, 0x3, 0x2, + 0x2, 0x2, 0x55b, 0x55e, 0x3, 0x2, 0x2, 0x2, 0x55c, 0x55a, 0x3, 0x2, + 0x2, 0x2, 0x55c, 0x55d, 0x3, 0x2, 0x2, 0x2, 0x55d, 0x95, 0x3, 0x2, 0x2, + 0x2, 0x55e, 0x55c, 0x3, 0x2, 0x2, 0x2, 0x55f, 0x561, 0x5, 0xb0, 0x59, + 0x2, 0x560, 0x562, 0x9, 0x10, 0x2, 0x2, 0x561, 0x560, 0x3, 0x2, 0x2, + 0x2, 0x561, 0x562, 0x3, 0x2, 0x2, 0x2, 0x562, 0x565, 0x3, 0x2, 0x2, + 0x2, 0x563, 0x564, 0x7, 0x73, 0x2, 0x2, 0x564, 0x566, 0x9, 0x11, 0x2, + 0x2, 0x565, 0x563, 0x3, 0x2, 0x2, 0x2, 0x565, 0x566, 0x3, 0x2, 0x2, + 0x2, 0x566, 0x569, 0x3, 0x2, 0x2, 0x2, 0x567, 0x568, 0x7, 0x1a, 0x2, + 0x2, 0x568, 0x56a, 0x7, 0xbe, 0x2, 0x2, 0x569, 0x567, 0x3, 0x2, 0x2, + 0x2, 0x569, 0x56a, 0x3, 0x2, 0x2, 0x2, 0x56a, 0x97, 0x3, 0x2, 0x2, 0x2, + 0x56b, 0x56e, 0x5, 0xca, 0x66, 0x2, 0x56c, 0x56d, 0x7, 0xdb, 0x2, 0x2, + 0x56d, 0x56f, 0x5, 0xca, 0x66, 0x2, 0x56e, 0x56c, 0x3, 0x2, 0x2, 0x2, + 0x56e, 0x56f, 0x3, 0x2, 0x2, 0x2, 0x56f, 0x99, 0x3, 0x2, 0x2, 0x2, 0x570, + 0x575, 0x5, 0x9c, 0x4f, 0x2, 0x571, 0x572, 0x7, 0xc4, 0x2, 0x2, 0x572, + 0x574, 0x5, 0x9c, 0x4f, 0x2, 0x573, 0x571, 0x3, 0x2, 0x2, 0x2, 0x574, + 0x577, 0x3, 0x2, 0x2, 0x2, 0x575, 0x573, 0x3, 0x2, 0x2, 0x2, 0x575, + 0x576, 0x3, 0x2, 0x2, 0x2, 0x576, 0x9b, 0x3, 0x2, 0x2, 0x2, 0x577, 0x575, + 0x3, 0x2, 0x2, 0x2, 0x578, 0x579, 0x5, 0xd6, 0x6c, 0x2, 0x579, 0x57a, + 0x7, 0xc9, 0x2, 0x2, 0x57a, 0x57b, 0x5, 0xcc, 0x67, 0x2, 0x57b, 0x9d, + 0x3, 0x2, 0x2, 0x2, 0x57c, 0x57d, 0x7, 0x8f, 0x2, 0x2, 0x57d, 0x57e, + 0x5, 0x9a, 0x4e, 0x2, 0x57e, 0x9f, 0x3, 0x2, 0x2, 0x2, 0x57f, 0x580, + 0x7, 0x91, 0x2, 0x2, 0x580, 0x581, 0x7, 0x1e, 0x2, 0x2, 0x581, 0x582, + 0x7, 0x21, 0x2, 0x2, 0x582, 0x5aa, 0x5, 0xc6, 0x64, 0x2, 0x583, 0x584, + 0x7, 0x91, 0x2, 0x2, 0x584, 0x585, 0x7, 0x1e, 0x2, 0x2, 0x585, 0x586, + 0x7, 0x2e, 0x2, 0x2, 0x586, 0x5aa, 0x5, 0xc0, 0x61, 0x2, 0x587, 0x588, + 0x7, 0x91, 0x2, 0x2, 0x588, 0x58a, 0x7, 0x1e, 0x2, 0x2, 0x589, 0x58b, + 0x7, 0x9b, 0x2, 0x2, 0x58a, 0x589, 0x3, 0x2, 0x2, 0x2, 0x58a, 0x58b, + 0x3, 0x2, 0x2, 0x2, 0x58b, 0x58d, 0x3, 0x2, 0x2, 0x2, 0x58c, 0x58e, + 0x7, 0x99, 0x2, 0x2, 0x58d, 0x58c, 0x3, 0x2, 0x2, 0x2, 0x58d, 0x58e, + 0x3, 0x2, 0x2, 0x2, 0x58e, 0x58f, 0x3, 0x2, 0x2, 0x2, 0x58f, 0x5aa, + 0x5, 0xc0, 0x61, 0x2, 0x590, 0x591, 0x7, 0x91, 0x2, 0x2, 0x591, 0x5aa, + 0x7, 0x22, 0x2, 0x2, 0x592, 0x593, 0x7, 0x91, 0x2, 0x2, 0x593, 0x596, + 0x7, 0x2d, 0x2, 0x2, 0x594, 0x595, 0x7, 0x42, 0x2, 0x2, 0x595, 0x597, + 0x5, 0xc6, 0x64, 0x2, 0x596, 0x594, 0x3, 0x2, 0x2, 0x2, 0x596, 0x597, + 0x3, 0x2, 0x2, 0x2, 0x597, 0x5aa, 0x3, 0x2, 0x2, 0x2, 0x598, 0x59a, + 0x7, 0x91, 0x2, 0x2, 0x599, 0x59b, 0x7, 0x9b, 0x2, 0x2, 0x59a, 0x599, + 0x3, 0x2, 0x2, 0x2, 0x59a, 0x59b, 0x3, 0x2, 0x2, 0x2, 0x59b, 0x59c, + 0x3, 0x2, 0x2, 0x2, 0x59c, 0x59f, 0x7, 0x9a, 0x2, 0x2, 0x59d, 0x59e, + 0x9, 0x12, 0x2, 0x2, 0x59e, 0x5a0, 0x5, 0xc6, 0x64, 0x2, 0x59f, 0x59d, + 0x3, 0x2, 0x2, 0x2, 0x59f, 0x5a0, 0x3, 0x2, 0x2, 0x2, 0x5a0, 0x5a4, + 0x3, 0x2, 0x2, 0x2, 0x5a1, 0x5a2, 0x7, 0x60, 0x2, 0x2, 0x5a2, 0x5a5, + 0x7, 0xbe, 0x2, 0x2, 0x5a3, 0x5a5, 0x5, 0x78, 0x3d, 0x2, 0x5a4, 0x5a1, + 0x3, 0x2, 0x2, 0x2, 0x5a4, 0x5a3, 0x3, 0x2, 0x2, 0x2, 0x5a4, 0x5a5, + 0x3, 0x2, 0x2, 0x2, 0x5a5, 0x5a7, 0x3, 0x2, 0x2, 0x2, 0x5a6, 0x5a8, + 0x5, 0x84, 0x43, 0x2, 0x5a7, 0x5a6, 0x3, 0x2, 0x2, 0x2, 0x5a7, 0x5a8, + 0x3, 0x2, 0x2, 0x2, 0x5a8, 0x5aa, 0x3, 0x2, 0x2, 0x2, 0x5a9, 0x57f, + 0x3, 0x2, 0x2, 0x2, 0x5a9, 0x583, 0x3, 0x2, 0x2, 0x2, 0x5a9, 0x587, + 0x3, 0x2, 0x2, 0x2, 0x5a9, 0x590, 0x3, 0x2, 0x2, 0x2, 0x5a9, 0x592, + 0x3, 0x2, 0x2, 0x2, 0x5a9, 0x598, 0x3, 0x2, 0x2, 0x2, 0x5aa, 0xa1, 0x3, + 0x2, 0x2, 0x2, 0x5ab, 0x5ac, 0x7, 0x98, 0x2, 0x2, 0x5ac, 0x5ad, 0x7, + 0x3e, 0x2, 0x2, 0x5ad, 0x5ae, 0x7, 0x31, 0x2, 0x2, 0x5ae, 0x5ce, 0x5, + 0xc0, 0x61, 0x2, 0x5af, 0x5b0, 0x7, 0x98, 0x2, 0x2, 0x5b0, 0x5b1, 0x7, + 0x3e, 0x2, 0x2, 0x5b1, 0x5ce, 0x7, 0x64, 0x2, 0x2, 0x5b2, 0x5b3, 0x7, + 0x98, 0x2, 0x2, 0x5b3, 0x5b4, 0x7, 0x82, 0x2, 0x2, 0x5b4, 0x5ce, 0x7, + 0x2d, 0x2, 0x2, 0x5b5, 0x5b6, 0x7, 0x98, 0x2, 0x2, 0x5b6, 0x5b7, 0x7, + 0x82, 0x2, 0x2, 0x5b7, 0x5b8, 0x7, 0x2e, 0x2, 0x2, 0x5b8, 0x5ce, 0x5, + 0xc0, 0x61, 0x2, 0x5b9, 0x5ba, 0x7, 0x98, 0x2, 0x2, 0x5ba, 0x5c2, 0x9, + 0x13, 0x2, 0x2, 0x5bb, 0x5bc, 0x7, 0x31, 0x2, 0x2, 0x5bc, 0x5c3, 0x7, + 0x8e, 0x2, 0x2, 0x5bd, 0x5c3, 0x7, 0x3b, 0x2, 0x2, 0x5be, 0x5c0, 0x7, + 0xa7, 0x2, 0x2, 0x5bf, 0x5be, 0x3, 0x2, 0x2, 0x2, 0x5bf, 0x5c0, 0x3, + 0x2, 0x2, 0x2, 0x5c0, 0x5c1, 0x3, 0x2, 0x2, 0x2, 0x5c1, 0x5c3, 0x7, + 0x68, 0x2, 0x2, 0x5c2, 0x5bb, 0x3, 0x2, 0x2, 0x2, 0x5c2, 0x5bd, 0x3, + 0x2, 0x2, 0x2, 0x5c2, 0x5bf, 0x3, 0x2, 0x2, 0x2, 0x5c3, 0x5c4, 0x3, + 0x2, 0x2, 0x2, 0x5c4, 0x5ce, 0x5, 0xc0, 0x61, 0x2, 0x5c5, 0x5c6, 0x7, + 0x98, 0x2, 0x2, 0x5c6, 0x5c7, 0x9, 0x13, 0x2, 0x2, 0x5c7, 0x5c8, 0x7, + 0x87, 0x2, 0x2, 0x5c8, 0x5ce, 0x7, 0x8e, 0x2, 0x2, 0x5c9, 0x5ca, 0x7, + 0x98, 0x2, 0x2, 0x5ca, 0x5cb, 0x7, 0x96, 0x2, 0x2, 0x5cb, 0x5cc, 0x7, + 0x86, 0x2, 0x2, 0x5cc, 0x5ce, 0x5, 0xc0, 0x61, 0x2, 0x5cd, 0x5ab, 0x3, + 0x2, 0x2, 0x2, 0x5cd, 0x5af, 0x3, 0x2, 0x2, 0x2, 0x5cd, 0x5b2, 0x3, + 0x2, 0x2, 0x2, 0x5cd, 0x5b5, 0x3, 0x2, 0x2, 0x2, 0x5cd, 0x5b9, 0x3, + 0x2, 0x2, 0x2, 0x5cd, 0x5c5, 0x3, 0x2, 0x2, 0x2, 0x5cd, 0x5c9, 0x3, + 0x2, 0x2, 0x2, 0x5ce, 0xa3, 0x3, 0x2, 0x2, 0x2, 0x5cf, 0x5d1, 0x7, 0xa6, + 0x2, 0x2, 0x5d0, 0x5d2, 0x7, 0x9b, 0x2, 0x2, 0x5d1, 0x5d0, 0x3, 0x2, + 0x2, 0x2, 0x5d1, 0x5d2, 0x3, 0x2, 0x2, 0x2, 0x5d2, 0x5d4, 0x3, 0x2, + 0x2, 0x2, 0x5d3, 0x5d5, 0x7, 0x99, 0x2, 0x2, 0x5d4, 0x5d3, 0x3, 0x2, + 0x2, 0x2, 0x5d4, 0x5d5, 0x3, 0x2, 0x2, 0x2, 0x5d5, 0x5d8, 0x3, 0x2, + 0x2, 0x2, 0x5d6, 0x5d7, 0x7, 0x4c, 0x2, 0x2, 0x5d7, 0x5d9, 0x7, 0x37, + 0x2, 0x2, 0x5d8, 0x5d6, 0x3, 0x2, 0x2, 0x2, 0x5d8, 0x5d9, 0x3, 0x2, + 0x2, 0x2, 0x5d9, 0x5da, 0x3, 0x2, 0x2, 0x2, 0x5da, 0x5dc, 0x5, 0xc0, + 0x61, 0x2, 0x5db, 0x5dd, 0x5, 0x2c, 0x17, 0x2, 0x5dc, 0x5db, 0x3, 0x2, + 0x2, 0x2, 0x5dc, 0x5dd, 0x3, 0x2, 0x2, 0x2, 0x5dd, 0xa5, 0x3, 0x2, 0x2, + 0x2, 0x5de, 0x5df, 0x7, 0xab, 0x2, 0x2, 0x5df, 0x5e0, 0x5, 0xc6, 0x64, + 0x2, 0x5e0, 0xa7, 0x3, 0x2, 0x2, 0x2, 0x5e1, 0x5e2, 0x7, 0xb1, 0x2, + 0x2, 0x5e2, 0x5e4, 0x5, 0xc0, 0x61, 0x2, 0x5e3, 0x5e5, 0x7, 0x36, 0x2, + 0x2, 0x5e4, 0x5e3, 0x3, 0x2, 0x2, 0x2, 0x5e4, 0x5e5, 0x3, 0x2, 0x2, + 0x2, 0x5e5, 0x5e8, 0x3, 0x2, 0x2, 0x2, 0x5e6, 0x5e7, 0x7, 0x61, 0x2, + 0x2, 0x5e7, 0x5e9, 0x7, 0xbc, 0x2, 0x2, 0x5e8, 0x5e6, 0x3, 0x2, 0x2, + 0x2, 0x5e8, 0x5e9, 0x3, 0x2, 0x2, 0x2, 0x5e9, 0xa9, 0x3, 0x2, 0x2, 0x2, + 0x5ea, 0x61a, 0x5, 0xd6, 0x6c, 0x2, 0x5eb, 0x5ec, 0x5, 0xd6, 0x6c, 0x2, + 0x5ec, 0x5ed, 0x7, 0xcf, 0x2, 0x2, 0x5ed, 0x5ee, 0x5, 0xd6, 0x6c, 0x2, + 0x5ee, 0x5f5, 0x5, 0xaa, 0x56, 0x2, 0x5ef, 0x5f0, 0x7, 0xc4, 0x2, 0x2, + 0x5f0, 0x5f1, 0x5, 0xd6, 0x6c, 0x2, 0x5f1, 0x5f2, 0x5, 0xaa, 0x56, 0x2, + 0x5f2, 0x5f4, 0x3, 0x2, 0x2, 0x2, 0x5f3, 0x5ef, 0x3, 0x2, 0x2, 0x2, + 0x5f4, 0x5f7, 0x3, 0x2, 0x2, 0x2, 0x5f5, 0x5f3, 0x3, 0x2, 0x2, 0x2, + 0x5f5, 0x5f6, 0x3, 0x2, 0x2, 0x2, 0x5f6, 0x5f8, 0x3, 0x2, 0x2, 0x2, + 0x5f7, 0x5f5, 0x3, 0x2, 0x2, 0x2, 0x5f8, 0x5f9, 0x7, 0xd9, 0x2, 0x2, + 0x5f9, 0x61a, 0x3, 0x2, 0x2, 0x2, 0x5fa, 0x5fb, 0x5, 0xd6, 0x6c, 0x2, + 0x5fb, 0x5fc, 0x7, 0xcf, 0x2, 0x2, 0x5fc, 0x601, 0x5, 0xda, 0x6e, 0x2, + 0x5fd, 0x5fe, 0x7, 0xc4, 0x2, 0x2, 0x5fe, 0x600, 0x5, 0xda, 0x6e, 0x2, + 0x5ff, 0x5fd, 0x3, 0x2, 0x2, 0x2, 0x600, 0x603, 0x3, 0x2, 0x2, 0x2, + 0x601, 0x5ff, 0x3, 0x2, 0x2, 0x2, 0x601, 0x602, 0x3, 0x2, 0x2, 0x2, + 0x602, 0x604, 0x3, 0x2, 0x2, 0x2, 0x603, 0x601, 0x3, 0x2, 0x2, 0x2, + 0x604, 0x605, 0x7, 0xd9, 0x2, 0x2, 0x605, 0x61a, 0x3, 0x2, 0x2, 0x2, + 0x606, 0x607, 0x5, 0xd6, 0x6c, 0x2, 0x607, 0x608, 0x7, 0xcf, 0x2, 0x2, + 0x608, 0x60d, 0x5, 0xaa, 0x56, 0x2, 0x609, 0x60a, 0x7, 0xc4, 0x2, 0x2, + 0x60a, 0x60c, 0x5, 0xaa, 0x56, 0x2, 0x60b, 0x609, 0x3, 0x2, 0x2, 0x2, + 0x60c, 0x60f, 0x3, 0x2, 0x2, 0x2, 0x60d, 0x60b, 0x3, 0x2, 0x2, 0x2, + 0x60d, 0x60e, 0x3, 0x2, 0x2, 0x2, 0x60e, 0x610, 0x3, 0x2, 0x2, 0x2, + 0x60f, 0x60d, 0x3, 0x2, 0x2, 0x2, 0x610, 0x611, 0x7, 0xd9, 0x2, 0x2, + 0x611, 0x61a, 0x3, 0x2, 0x2, 0x2, 0x612, 0x613, 0x5, 0xd6, 0x6c, 0x2, + 0x613, 0x615, 0x7, 0xcf, 0x2, 0x2, 0x614, 0x616, 0x5, 0xac, 0x57, 0x2, + 0x615, 0x614, 0x3, 0x2, 0x2, 0x2, 0x615, 0x616, 0x3, 0x2, 0x2, 0x2, + 0x616, 0x617, 0x3, 0x2, 0x2, 0x2, 0x617, 0x618, 0x7, 0xd9, 0x2, 0x2, + 0x618, 0x61a, 0x3, 0x2, 0x2, 0x2, 0x619, 0x5ea, 0x3, 0x2, 0x2, 0x2, + 0x619, 0x5eb, 0x3, 0x2, 0x2, 0x2, 0x619, 0x5fa, 0x3, 0x2, 0x2, 0x2, + 0x619, 0x606, 0x3, 0x2, 0x2, 0x2, 0x619, 0x612, 0x3, 0x2, 0x2, 0x2, + 0x61a, 0xab, 0x3, 0x2, 0x2, 0x2, 0x61b, 0x620, 0x5, 0xae, 0x58, 0x2, + 0x61c, 0x61d, 0x7, 0xc4, 0x2, 0x2, 0x61d, 0x61f, 0x5, 0xae, 0x58, 0x2, + 0x61e, 0x61c, 0x3, 0x2, 0x2, 0x2, 0x61f, 0x622, 0x3, 0x2, 0x2, 0x2, + 0x620, 0x61e, 0x3, 0x2, 0x2, 0x2, 0x620, 0x621, 0x3, 0x2, 0x2, 0x2, + 0x621, 0xad, 0x3, 0x2, 0x2, 0x2, 0x622, 0x620, 0x3, 0x2, 0x2, 0x2, 0x623, + 0x624, 0x5, 0xc0, 0x61, 0x2, 0x624, 0x625, 0x7, 0xc7, 0x2, 0x2, 0x625, + 0x627, 0x3, 0x2, 0x2, 0x2, 0x626, 0x623, 0x3, 0x2, 0x2, 0x2, 0x626, + 0x627, 0x3, 0x2, 0x2, 0x2, 0x627, 0x628, 0x3, 0x2, 0x2, 0x2, 0x628, + 0x62f, 0x7, 0xc0, 0x2, 0x2, 0x629, 0x62a, 0x7, 0xcf, 0x2, 0x2, 0x62a, + 0x62b, 0x5, 0x68, 0x35, 0x2, 0x62b, 0x62c, 0x7, 0xd9, 0x2, 0x2, 0x62c, + 0x62f, 0x3, 0x2, 0x2, 0x2, 0x62d, 0x62f, 0x5, 0xb0, 0x59, 0x2, 0x62e, + 0x626, 0x3, 0x2, 0x2, 0x2, 0x62e, 0x629, 0x3, 0x2, 0x2, 0x2, 0x62e, + 0x62d, 0x3, 0x2, 0x2, 0x2, 0x62f, 0xaf, 0x3, 0x2, 0x2, 0x2, 0x630, 0x631, + 0x8, 0x59, 0x1, 0x2, 0x631, 0x633, 0x7, 0x14, 0x2, 0x2, 0x632, 0x634, + 0x5, 0xb0, 0x59, 0x2, 0x633, 0x632, 0x3, 0x2, 0x2, 0x2, 0x633, 0x634, + 0x3, 0x2, 0x2, 0x2, 0x634, 0x63a, 0x3, 0x2, 0x2, 0x2, 0x635, 0x636, + 0x7, 0xb3, 0x2, 0x2, 0x636, 0x637, 0x5, 0xb0, 0x59, 0x2, 0x637, 0x638, + 0x7, 0x9d, 0x2, 0x2, 0x638, 0x639, 0x5, 0xb0, 0x59, 0x2, 0x639, 0x63b, + 0x3, 0x2, 0x2, 0x2, 0x63a, 0x635, 0x3, 0x2, 0x2, 0x2, 0x63b, 0x63c, + 0x3, 0x2, 0x2, 0x2, 0x63c, 0x63a, 0x3, 0x2, 0x2, 0x2, 0x63c, 0x63d, + 0x3, 0x2, 0x2, 0x2, 0x63d, 0x640, 0x3, 0x2, 0x2, 0x2, 0x63e, 0x63f, + 0x7, 0x33, 0x2, 0x2, 0x63f, 0x641, 0x5, 0xb0, 0x59, 0x2, 0x640, 0x63e, + 0x3, 0x2, 0x2, 0x2, 0x640, 0x641, 0x3, 0x2, 0x2, 0x2, 0x641, 0x642, + 0x3, 0x2, 0x2, 0x2, 0x642, 0x643, 0x7, 0x34, 0x2, 0x2, 0x643, 0x69c, + 0x3, 0x2, 0x2, 0x2, 0x644, 0x645, 0x7, 0x15, 0x2, 0x2, 0x645, 0x646, + 0x7, 0xcf, 0x2, 0x2, 0x646, 0x647, 0x5, 0xb0, 0x59, 0x2, 0x647, 0x648, + 0x7, 0xc, 0x2, 0x2, 0x648, 0x649, 0x5, 0xaa, 0x56, 0x2, 0x649, 0x64a, + 0x7, 0xd9, 0x2, 0x2, 0x64a, 0x69c, 0x3, 0x2, 0x2, 0x2, 0x64b, 0x64c, + 0x7, 0x23, 0x2, 0x2, 0x64c, 0x69c, 0x7, 0xbe, 0x2, 0x2, 0x64d, 0x64e, + 0x7, 0x3a, 0x2, 0x2, 0x64e, 0x64f, 0x7, 0xcf, 0x2, 0x2, 0x64f, 0x650, + 0x5, 0xce, 0x68, 0x2, 0x650, 0x651, 0x7, 0x42, 0x2, 0x2, 0x651, 0x652, + 0x5, 0xb0, 0x59, 0x2, 0x652, 0x653, 0x7, 0xd9, 0x2, 0x2, 0x653, 0x69c, + 0x3, 0x2, 0x2, 0x2, 0x654, 0x655, 0x7, 0x54, 0x2, 0x2, 0x655, 0x656, + 0x5, 0xb0, 0x59, 0x2, 0x656, 0x657, 0x5, 0xce, 0x68, 0x2, 0x657, 0x69c, + 0x3, 0x2, 0x2, 0x2, 0x658, 0x659, 0x7, 0x95, 0x2, 0x2, 0x659, 0x65a, + 0x7, 0xcf, 0x2, 0x2, 0x65a, 0x65b, 0x5, 0xb0, 0x59, 0x2, 0x65b, 0x65c, + 0x7, 0x42, 0x2, 0x2, 0x65c, 0x65f, 0x5, 0xb0, 0x59, 0x2, 0x65d, 0x65e, + 0x7, 0x3f, 0x2, 0x2, 0x65e, 0x660, 0x5, 0xb0, 0x59, 0x2, 0x65f, 0x65d, + 0x3, 0x2, 0x2, 0x2, 0x65f, 0x660, 0x3, 0x2, 0x2, 0x2, 0x660, 0x661, + 0x3, 0x2, 0x2, 0x2, 0x661, 0x662, 0x7, 0xd9, 0x2, 0x2, 0x662, 0x69c, + 0x3, 0x2, 0x2, 0x2, 0x663, 0x664, 0x7, 0xa0, 0x2, 0x2, 0x664, 0x69c, + 0x7, 0xbe, 0x2, 0x2, 0x665, 0x666, 0x7, 0xa5, 0x2, 0x2, 0x666, 0x667, + 0x7, 0xcf, 0x2, 0x2, 0x667, 0x668, 0x9, 0x14, 0x2, 0x2, 0x668, 0x669, + 0x7, 0xbe, 0x2, 0x2, 0x669, 0x66a, 0x7, 0x42, 0x2, 0x2, 0x66a, 0x66b, + 0x5, 0xb0, 0x59, 0x2, 0x66b, 0x66c, 0x7, 0xd9, 0x2, 0x2, 0x66c, 0x69c, + 0x3, 0x2, 0x2, 0x2, 0x66d, 0x673, 0x5, 0xd6, 0x6c, 0x2, 0x66e, 0x670, + 0x7, 0xcf, 0x2, 0x2, 0x66f, 0x671, 0x5, 0xac, 0x57, 0x2, 0x670, 0x66f, + 0x3, 0x2, 0x2, 0x2, 0x670, 0x671, 0x3, 0x2, 0x2, 0x2, 0x671, 0x672, + 0x3, 0x2, 0x2, 0x2, 0x672, 0x674, 0x7, 0xd9, 0x2, 0x2, 0x673, 0x66e, + 0x3, 0x2, 0x2, 0x2, 0x673, 0x674, 0x3, 0x2, 0x2, 0x2, 0x674, 0x675, + 0x3, 0x2, 0x2, 0x2, 0x675, 0x677, 0x7, 0xcf, 0x2, 0x2, 0x676, 0x678, + 0x7, 0x30, 0x2, 0x2, 0x677, 0x676, 0x3, 0x2, 0x2, 0x2, 0x677, 0x678, + 0x3, 0x2, 0x2, 0x2, 0x678, 0x67a, 0x3, 0x2, 0x2, 0x2, 0x679, 0x67b, + 0x5, 0xb2, 0x5a, 0x2, 0x67a, 0x679, 0x3, 0x2, 0x2, 0x2, 0x67a, 0x67b, + 0x3, 0x2, 0x2, 0x2, 0x67b, 0x67c, 0x3, 0x2, 0x2, 0x2, 0x67c, 0x67d, + 0x7, 0xd9, 0x2, 0x2, 0x67d, 0x69c, 0x3, 0x2, 0x2, 0x2, 0x67e, 0x69c, + 0x5, 0xcc, 0x67, 0x2, 0x67f, 0x680, 0x7, 0xc6, 0x2, 0x2, 0x680, 0x69c, + 0x5, 0xb0, 0x59, 0x13, 0x681, 0x682, 0x7, 0x71, 0x2, 0x2, 0x682, 0x69c, + 0x5, 0xb0, 0x59, 0xe, 0x683, 0x684, 0x5, 0xc0, 0x61, 0x2, 0x684, 0x685, + 0x7, 0xc7, 0x2, 0x2, 0x685, 0x687, 0x3, 0x2, 0x2, 0x2, 0x686, 0x683, + 0x3, 0x2, 0x2, 0x2, 0x686, 0x687, 0x3, 0x2, 0x2, 0x2, 0x687, 0x688, + 0x3, 0x2, 0x2, 0x2, 0x688, 0x69c, 0x7, 0xc0, 0x2, 0x2, 0x689, 0x68a, + 0x7, 0xcf, 0x2, 0x2, 0x68a, 0x68b, 0x5, 0x68, 0x35, 0x2, 0x68b, 0x68c, + 0x7, 0xd9, 0x2, 0x2, 0x68c, 0x69c, 0x3, 0x2, 0x2, 0x2, 0x68d, 0x68e, + 0x7, 0xcf, 0x2, 0x2, 0x68e, 0x68f, 0x5, 0xb0, 0x59, 0x2, 0x68f, 0x690, + 0x7, 0xd9, 0x2, 0x2, 0x690, 0x69c, 0x3, 0x2, 0x2, 0x2, 0x691, 0x692, + 0x7, 0xcf, 0x2, 0x2, 0x692, 0x693, 0x5, 0xac, 0x57, 0x2, 0x693, 0x694, + 0x7, 0xd9, 0x2, 0x2, 0x694, 0x69c, 0x3, 0x2, 0x2, 0x2, 0x695, 0x697, + 0x7, 0xcd, 0x2, 0x2, 0x696, 0x698, 0x5, 0xac, 0x57, 0x2, 0x697, 0x696, + 0x3, 0x2, 0x2, 0x2, 0x697, 0x698, 0x3, 0x2, 0x2, 0x2, 0x698, 0x699, + 0x3, 0x2, 0x2, 0x2, 0x699, 0x69c, 0x7, 0xd8, 0x2, 0x2, 0x69a, 0x69c, + 0x5, 0xb8, 0x5d, 0x2, 0x69b, 0x630, 0x3, 0x2, 0x2, 0x2, 0x69b, 0x644, + 0x3, 0x2, 0x2, 0x2, 0x69b, 0x64b, 0x3, 0x2, 0x2, 0x2, 0x69b, 0x64d, + 0x3, 0x2, 0x2, 0x2, 0x69b, 0x654, 0x3, 0x2, 0x2, 0x2, 0x69b, 0x658, + 0x3, 0x2, 0x2, 0x2, 0x69b, 0x663, 0x3, 0x2, 0x2, 0x2, 0x69b, 0x665, + 0x3, 0x2, 0x2, 0x2, 0x69b, 0x66d, 0x3, 0x2, 0x2, 0x2, 0x69b, 0x67e, + 0x3, 0x2, 0x2, 0x2, 0x69b, 0x67f, 0x3, 0x2, 0x2, 0x2, 0x69b, 0x681, + 0x3, 0x2, 0x2, 0x2, 0x69b, 0x686, 0x3, 0x2, 0x2, 0x2, 0x69b, 0x689, + 0x3, 0x2, 0x2, 0x2, 0x69b, 0x68d, 0x3, 0x2, 0x2, 0x2, 0x69b, 0x691, + 0x3, 0x2, 0x2, 0x2, 0x69b, 0x695, 0x3, 0x2, 0x2, 0x2, 0x69b, 0x69a, + 0x3, 0x2, 0x2, 0x2, 0x69c, 0x6e4, 0x3, 0x2, 0x2, 0x2, 0x69d, 0x69e, + 0xc, 0x12, 0x2, 0x2, 0x69e, 0x69f, 0x9, 0x15, 0x2, 0x2, 0x69f, 0x6e3, + 0x5, 0xb0, 0x59, 0x13, 0x6a0, 0x6a1, 0xc, 0x11, 0x2, 0x2, 0x6a1, 0x6a2, + 0x9, 0x16, 0x2, 0x2, 0x6a2, 0x6e3, 0x5, 0xb0, 0x59, 0x12, 0x6a3, 0x6b6, + 0xc, 0x10, 0x2, 0x2, 0x6a4, 0x6b7, 0x7, 0xc8, 0x2, 0x2, 0x6a5, 0x6b7, + 0x7, 0xc9, 0x2, 0x2, 0x6a6, 0x6b7, 0x7, 0xd1, 0x2, 0x2, 0x6a7, 0x6b7, + 0x7, 0xce, 0x2, 0x2, 0x6a8, 0x6b7, 0x7, 0xca, 0x2, 0x2, 0x6a9, 0x6b7, + 0x7, 0xd0, 0x2, 0x2, 0x6aa, 0x6b7, 0x7, 0xcb, 0x2, 0x2, 0x6ab, 0x6ad, + 0x7, 0x45, 0x2, 0x2, 0x6ac, 0x6ab, 0x3, 0x2, 0x2, 0x2, 0x6ac, 0x6ad, + 0x3, 0x2, 0x2, 0x2, 0x6ad, 0x6af, 0x3, 0x2, 0x2, 0x2, 0x6ae, 0x6b0, + 0x7, 0x71, 0x2, 0x2, 0x6af, 0x6ae, 0x3, 0x2, 0x2, 0x2, 0x6af, 0x6b0, + 0x3, 0x2, 0x2, 0x2, 0x6b0, 0x6b1, 0x3, 0x2, 0x2, 0x2, 0x6b1, 0x6b7, + 0x7, 0x4e, 0x2, 0x2, 0x6b2, 0x6b4, 0x7, 0x71, 0x2, 0x2, 0x6b3, 0x6b2, + 0x3, 0x2, 0x2, 0x2, 0x6b3, 0x6b4, 0x3, 0x2, 0x2, 0x2, 0x6b4, 0x6b5, + 0x3, 0x2, 0x2, 0x2, 0x6b5, 0x6b7, 0x9, 0x17, 0x2, 0x2, 0x6b6, 0x6a4, + 0x3, 0x2, 0x2, 0x2, 0x6b6, 0x6a5, 0x3, 0x2, 0x2, 0x2, 0x6b6, 0x6a6, + 0x3, 0x2, 0x2, 0x2, 0x6b6, 0x6a7, 0x3, 0x2, 0x2, 0x2, 0x6b6, 0x6a8, + 0x3, 0x2, 0x2, 0x2, 0x6b6, 0x6a9, 0x3, 0x2, 0x2, 0x2, 0x6b6, 0x6aa, + 0x3, 0x2, 0x2, 0x2, 0x6b6, 0x6ac, 0x3, 0x2, 0x2, 0x2, 0x6b6, 0x6b3, + 0x3, 0x2, 0x2, 0x2, 0x6b7, 0x6b8, 0x3, 0x2, 0x2, 0x2, 0x6b8, 0x6e3, + 0x5, 0xb0, 0x59, 0x11, 0x6b9, 0x6ba, 0xc, 0xd, 0x2, 0x2, 0x6ba, 0x6bb, + 0x7, 0x8, 0x2, 0x2, 0x6bb, 0x6e3, 0x5, 0xb0, 0x59, 0xe, 0x6bc, 0x6bd, + 0xc, 0xc, 0x2, 0x2, 0x6bd, 0x6be, 0x7, 0x77, 0x2, 0x2, 0x6be, 0x6e3, + 0x5, 0xb0, 0x59, 0xd, 0x6bf, 0x6c1, 0xc, 0xb, 0x2, 0x2, 0x6c0, 0x6c2, + 0x7, 0x71, 0x2, 0x2, 0x6c1, 0x6c0, 0x3, 0x2, 0x2, 0x2, 0x6c1, 0x6c2, + 0x3, 0x2, 0x2, 0x2, 0x6c2, 0x6c3, 0x3, 0x2, 0x2, 0x2, 0x6c3, 0x6c4, + 0x7, 0x11, 0x2, 0x2, 0x6c4, 0x6c5, 0x5, 0xb0, 0x59, 0x2, 0x6c5, 0x6c6, + 0x7, 0x8, 0x2, 0x2, 0x6c6, 0x6c7, 0x5, 0xb0, 0x59, 0xc, 0x6c7, 0x6e3, + 0x3, 0x2, 0x2, 0x2, 0x6c8, 0x6c9, 0xc, 0xa, 0x2, 0x2, 0x6c9, 0x6ca, + 0x7, 0xd4, 0x2, 0x2, 0x6ca, 0x6cb, 0x5, 0xb0, 0x59, 0x2, 0x6cb, 0x6cc, + 0x7, 0xc3, 0x2, 0x2, 0x6cc, 0x6cd, 0x5, 0xb0, 0x59, 0xa, 0x6cd, 0x6e3, + 0x3, 0x2, 0x2, 0x2, 0x6ce, 0x6cf, 0xc, 0x15, 0x2, 0x2, 0x6cf, 0x6d0, + 0x7, 0xcd, 0x2, 0x2, 0x6d0, 0x6d1, 0x5, 0xb0, 0x59, 0x2, 0x6d1, 0x6d2, + 0x7, 0xd8, 0x2, 0x2, 0x6d2, 0x6e3, 0x3, 0x2, 0x2, 0x2, 0x6d3, 0x6d4, + 0xc, 0x14, 0x2, 0x2, 0x6d4, 0x6d5, 0x7, 0xc7, 0x2, 0x2, 0x6d5, 0x6e3, + 0x7, 0xbc, 0x2, 0x2, 0x6d6, 0x6d7, 0xc, 0xf, 0x2, 0x2, 0x6d7, 0x6d9, + 0x7, 0x56, 0x2, 0x2, 0x6d8, 0x6da, 0x7, 0x71, 0x2, 0x2, 0x6d9, 0x6d8, + 0x3, 0x2, 0x2, 0x2, 0x6d9, 0x6da, 0x3, 0x2, 0x2, 0x2, 0x6da, 0x6db, + 0x3, 0x2, 0x2, 0x2, 0x6db, 0x6e3, 0x7, 0x72, 0x2, 0x2, 0x6dc, 0x6e0, + 0xc, 0x9, 0x2, 0x2, 0x6dd, 0x6e1, 0x5, 0xd4, 0x6b, 0x2, 0x6de, 0x6df, + 0x7, 0xc, 0x2, 0x2, 0x6df, 0x6e1, 0x5, 0xd6, 0x6c, 0x2, 0x6e0, 0x6dd, + 0x3, 0x2, 0x2, 0x2, 0x6e0, 0x6de, 0x3, 0x2, 0x2, 0x2, 0x6e1, 0x6e3, + 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x69d, 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6a0, + 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6a3, 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6b9, + 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6bc, 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6bf, + 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6c8, 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6ce, + 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6d3, 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6d6, + 0x3, 0x2, 0x2, 0x2, 0x6e2, 0x6dc, 0x3, 0x2, 0x2, 0x2, 0x6e3, 0x6e6, + 0x3, 0x2, 0x2, 0x2, 0x6e4, 0x6e2, 0x3, 0x2, 0x2, 0x2, 0x6e4, 0x6e5, + 0x3, 0x2, 0x2, 0x2, 0x6e5, 0xb1, 0x3, 0x2, 0x2, 0x2, 0x6e6, 0x6e4, 0x3, + 0x2, 0x2, 0x2, 0x6e7, 0x6ec, 0x5, 0xb4, 0x5b, 0x2, 0x6e8, 0x6e9, 0x7, + 0xc4, 0x2, 0x2, 0x6e9, 0x6eb, 0x5, 0xb4, 0x5b, 0x2, 0x6ea, 0x6e8, 0x3, + 0x2, 0x2, 0x2, 0x6eb, 0x6ee, 0x3, 0x2, 0x2, 0x2, 0x6ec, 0x6ea, 0x3, + 0x2, 0x2, 0x2, 0x6ec, 0x6ed, 0x3, 0x2, 0x2, 0x2, 0x6ed, 0xb3, 0x3, 0x2, + 0x2, 0x2, 0x6ee, 0x6ec, 0x3, 0x2, 0x2, 0x2, 0x6ef, 0x6f2, 0x5, 0xb6, + 0x5c, 0x2, 0x6f0, 0x6f2, 0x5, 0xb0, 0x59, 0x2, 0x6f1, 0x6ef, 0x3, 0x2, + 0x2, 0x2, 0x6f1, 0x6f0, 0x3, 0x2, 0x2, 0x2, 0x6f2, 0xb5, 0x3, 0x2, 0x2, + 0x2, 0x6f3, 0x6f4, 0x7, 0xcf, 0x2, 0x2, 0x6f4, 0x6f9, 0x5, 0xd6, 0x6c, + 0x2, 0x6f5, 0x6f6, 0x7, 0xc4, 0x2, 0x2, 0x6f6, 0x6f8, 0x5, 0xd6, 0x6c, + 0x2, 0x6f7, 0x6f5, 0x3, 0x2, 0x2, 0x2, 0x6f8, 0x6fb, 0x3, 0x2, 0x2, + 0x2, 0x6f9, 0x6f7, 0x3, 0x2, 0x2, 0x2, 0x6f9, 0x6fa, 0x3, 0x2, 0x2, + 0x2, 0x6fa, 0x6fc, 0x3, 0x2, 0x2, 0x2, 0x6fb, 0x6f9, 0x3, 0x2, 0x2, + 0x2, 0x6fc, 0x6fd, 0x7, 0xd9, 0x2, 0x2, 0x6fd, 0x707, 0x3, 0x2, 0x2, + 0x2, 0x6fe, 0x703, 0x5, 0xd6, 0x6c, 0x2, 0x6ff, 0x700, 0x7, 0xc4, 0x2, + 0x2, 0x700, 0x702, 0x5, 0xd6, 0x6c, 0x2, 0x701, 0x6ff, 0x3, 0x2, 0x2, + 0x2, 0x702, 0x705, 0x3, 0x2, 0x2, 0x2, 0x703, 0x701, 0x3, 0x2, 0x2, + 0x2, 0x703, 0x704, 0x3, 0x2, 0x2, 0x2, 0x704, 0x707, 0x3, 0x2, 0x2, + 0x2, 0x705, 0x703, 0x3, 0x2, 0x2, 0x2, 0x706, 0x6f3, 0x3, 0x2, 0x2, + 0x2, 0x706, 0x6fe, 0x3, 0x2, 0x2, 0x2, 0x707, 0x708, 0x3, 0x2, 0x2, + 0x2, 0x708, 0x709, 0x7, 0xbf, 0x2, 0x2, 0x709, 0x70a, 0x5, 0xb0, 0x59, + 0x2, 0x70a, 0xb7, 0x3, 0x2, 0x2, 0x2, 0x70b, 0x70c, 0x5, 0xc0, 0x61, + 0x2, 0x70c, 0x70d, 0x7, 0xc7, 0x2, 0x2, 0x70d, 0x70f, 0x3, 0x2, 0x2, + 0x2, 0x70e, 0x70b, 0x3, 0x2, 0x2, 0x2, 0x70e, 0x70f, 0x3, 0x2, 0x2, + 0x2, 0x70f, 0x710, 0x3, 0x2, 0x2, 0x2, 0x710, 0x711, 0x5, 0xba, 0x5e, + 0x2, 0x711, 0xb9, 0x3, 0x2, 0x2, 0x2, 0x712, 0x715, 0x5, 0xd6, 0x6c, + 0x2, 0x713, 0x714, 0x7, 0xc7, 0x2, 0x2, 0x714, 0x716, 0x5, 0xd6, 0x6c, + 0x2, 0x715, 0x713, 0x3, 0x2, 0x2, 0x2, 0x715, 0x716, 0x3, 0x2, 0x2, + 0x2, 0x716, 0xbb, 0x3, 0x2, 0x2, 0x2, 0x717, 0x718, 0x8, 0x5f, 0x1, + 0x2, 0x718, 0x71f, 0x5, 0xc0, 0x61, 0x2, 0x719, 0x71f, 0x5, 0xbe, 0x60, + 0x2, 0x71a, 0x71b, 0x7, 0xcf, 0x2, 0x2, 0x71b, 0x71c, 0x5, 0x68, 0x35, + 0x2, 0x71c, 0x71d, 0x7, 0xd9, 0x2, 0x2, 0x71d, 0x71f, 0x3, 0x2, 0x2, + 0x2, 0x71e, 0x717, 0x3, 0x2, 0x2, 0x2, 0x71e, 0x719, 0x3, 0x2, 0x2, + 0x2, 0x71e, 0x71a, 0x3, 0x2, 0x2, 0x2, 0x71f, 0x728, 0x3, 0x2, 0x2, + 0x2, 0x720, 0x724, 0xc, 0x3, 0x2, 0x2, 0x721, 0x725, 0x5, 0xd4, 0x6b, + 0x2, 0x722, 0x723, 0x7, 0xc, 0x2, 0x2, 0x723, 0x725, 0x5, 0xd6, 0x6c, + 0x2, 0x724, 0x721, 0x3, 0x2, 0x2, 0x2, 0x724, 0x722, 0x3, 0x2, 0x2, + 0x2, 0x725, 0x727, 0x3, 0x2, 0x2, 0x2, 0x726, 0x720, 0x3, 0x2, 0x2, + 0x2, 0x727, 0x72a, 0x3, 0x2, 0x2, 0x2, 0x728, 0x726, 0x3, 0x2, 0x2, + 0x2, 0x728, 0x729, 0x3, 0x2, 0x2, 0x2, 0x729, 0xbd, 0x3, 0x2, 0x2, 0x2, + 0x72a, 0x728, 0x3, 0x2, 0x2, 0x2, 0x72b, 0x72c, 0x5, 0xd6, 0x6c, 0x2, + 0x72c, 0x72e, 0x7, 0xcf, 0x2, 0x2, 0x72d, 0x72f, 0x5, 0xc2, 0x62, 0x2, + 0x72e, 0x72d, 0x3, 0x2, 0x2, 0x2, 0x72e, 0x72f, 0x3, 0x2, 0x2, 0x2, + 0x72f, 0x730, 0x3, 0x2, 0x2, 0x2, 0x730, 0x731, 0x7, 0xd9, 0x2, 0x2, + 0x731, 0xbf, 0x3, 0x2, 0x2, 0x2, 0x732, 0x733, 0x5, 0xc6, 0x64, 0x2, + 0x733, 0x734, 0x7, 0xc7, 0x2, 0x2, 0x734, 0x736, 0x3, 0x2, 0x2, 0x2, + 0x735, 0x732, 0x3, 0x2, 0x2, 0x2, 0x735, 0x736, 0x3, 0x2, 0x2, 0x2, + 0x736, 0x737, 0x3, 0x2, 0x2, 0x2, 0x737, 0x738, 0x5, 0xd6, 0x6c, 0x2, + 0x738, 0xc1, 0x3, 0x2, 0x2, 0x2, 0x739, 0x73e, 0x5, 0xc4, 0x63, 0x2, + 0x73a, 0x73b, 0x7, 0xc4, 0x2, 0x2, 0x73b, 0x73d, 0x5, 0xc4, 0x63, 0x2, + 0x73c, 0x73a, 0x3, 0x2, 0x2, 0x2, 0x73d, 0x740, 0x3, 0x2, 0x2, 0x2, + 0x73e, 0x73c, 0x3, 0x2, 0x2, 0x2, 0x73e, 0x73f, 0x3, 0x2, 0x2, 0x2, + 0x73f, 0xc3, 0x3, 0x2, 0x2, 0x2, 0x740, 0x73e, 0x3, 0x2, 0x2, 0x2, 0x741, + 0x745, 0x5, 0xc0, 0x61, 0x2, 0x742, 0x745, 0x5, 0xbe, 0x60, 0x2, 0x743, + 0x745, 0x5, 0xcc, 0x67, 0x2, 0x744, 0x741, 0x3, 0x2, 0x2, 0x2, 0x744, + 0x742, 0x3, 0x2, 0x2, 0x2, 0x744, 0x743, 0x3, 0x2, 0x2, 0x2, 0x745, + 0xc5, 0x3, 0x2, 0x2, 0x2, 0x746, 0x747, 0x5, 0xd6, 0x6c, 0x2, 0x747, + 0xc7, 0x3, 0x2, 0x2, 0x2, 0x748, 0x751, 0x7, 0xba, 0x2, 0x2, 0x749, + 0x74a, 0x7, 0xc7, 0x2, 0x2, 0x74a, 0x751, 0x9, 0x18, 0x2, 0x2, 0x74b, + 0x74c, 0x7, 0xbc, 0x2, 0x2, 0x74c, 0x74e, 0x7, 0xc7, 0x2, 0x2, 0x74d, + 0x74f, 0x9, 0x18, 0x2, 0x2, 0x74e, 0x74d, 0x3, 0x2, 0x2, 0x2, 0x74e, + 0x74f, 0x3, 0x2, 0x2, 0x2, 0x74f, 0x751, 0x3, 0x2, 0x2, 0x2, 0x750, + 0x748, 0x3, 0x2, 0x2, 0x2, 0x750, 0x749, 0x3, 0x2, 0x2, 0x2, 0x750, + 0x74b, 0x3, 0x2, 0x2, 0x2, 0x751, 0xc9, 0x3, 0x2, 0x2, 0x2, 0x752, 0x754, + 0x9, 0x19, 0x2, 0x2, 0x753, 0x752, 0x3, 0x2, 0x2, 0x2, 0x753, 0x754, + 0x3, 0x2, 0x2, 0x2, 0x754, 0x75b, 0x3, 0x2, 0x2, 0x2, 0x755, 0x75c, + 0x5, 0xc8, 0x65, 0x2, 0x756, 0x75c, 0x7, 0xbb, 0x2, 0x2, 0x757, 0x75c, + 0x7, 0xbc, 0x2, 0x2, 0x758, 0x75c, 0x7, 0xbd, 0x2, 0x2, 0x759, 0x75c, + 0x7, 0x50, 0x2, 0x2, 0x75a, 0x75c, 0x7, 0x6f, 0x2, 0x2, 0x75b, 0x755, + 0x3, 0x2, 0x2, 0x2, 0x75b, 0x756, 0x3, 0x2, 0x2, 0x2, 0x75b, 0x757, + 0x3, 0x2, 0x2, 0x2, 0x75b, 0x758, 0x3, 0x2, 0x2, 0x2, 0x75b, 0x759, + 0x3, 0x2, 0x2, 0x2, 0x75b, 0x75a, 0x3, 0x2, 0x2, 0x2, 0x75c, 0xcb, 0x3, + 0x2, 0x2, 0x2, 0x75d, 0x761, 0x5, 0xca, 0x66, 0x2, 0x75e, 0x761, 0x7, + 0xbe, 0x2, 0x2, 0x75f, 0x761, 0x7, 0x72, 0x2, 0x2, 0x760, 0x75d, 0x3, + 0x2, 0x2, 0x2, 0x760, 0x75e, 0x3, 0x2, 0x2, 0x2, 0x760, 0x75f, 0x3, + 0x2, 0x2, 0x2, 0x761, 0xcd, 0x3, 0x2, 0x2, 0x2, 0x762, 0x763, 0x9, 0x1a, + 0x2, 0x2, 0x763, 0xcf, 0x3, 0x2, 0x2, 0x2, 0x764, 0x765, 0x9, 0x1b, + 0x2, 0x2, 0x765, 0xd1, 0x3, 0x2, 0x2, 0x2, 0x766, 0x767, 0x9, 0x1c, + 0x2, 0x2, 0x767, 0xd3, 0x3, 0x2, 0x2, 0x2, 0x768, 0x76b, 0x7, 0xb9, + 0x2, 0x2, 0x769, 0x76b, 0x5, 0xd2, 0x6a, 0x2, 0x76a, 0x768, 0x3, 0x2, + 0x2, 0x2, 0x76a, 0x769, 0x3, 0x2, 0x2, 0x2, 0x76b, 0xd5, 0x3, 0x2, 0x2, + 0x2, 0x76c, 0x770, 0x7, 0xb9, 0x2, 0x2, 0x76d, 0x770, 0x5, 0xce, 0x68, + 0x2, 0x76e, 0x770, 0x5, 0xd0, 0x69, 0x2, 0x76f, 0x76c, 0x3, 0x2, 0x2, + 0x2, 0x76f, 0x76d, 0x3, 0x2, 0x2, 0x2, 0x76f, 0x76e, 0x3, 0x2, 0x2, + 0x2, 0x770, 0xd7, 0x3, 0x2, 0x2, 0x2, 0x771, 0x774, 0x5, 0xd6, 0x6c, + 0x2, 0x772, 0x774, 0x7, 0x72, 0x2, 0x2, 0x773, 0x771, 0x3, 0x2, 0x2, + 0x2, 0x773, 0x772, 0x3, 0x2, 0x2, 0x2, 0x774, 0xd9, 0x3, 0x2, 0x2, 0x2, + 0x775, 0x776, 0x7, 0xbe, 0x2, 0x2, 0x776, 0x777, 0x7, 0xc9, 0x2, 0x2, + 0x777, 0x778, 0x5, 0xca, 0x66, 0x2, 0x778, 0xdb, 0x3, 0x2, 0x2, 0x2, + 0x102, 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, 0x413, + 0x418, 0x41b, 0x425, 0x42f, 0x433, 0x438, 0x43c, 0x442, 0x445, 0x448, + 0x44b, 0x459, 0x45d, 0x461, 0x466, 0x469, 0x473, 0x47b, 0x47e, 0x482, + 0x485, 0x489, 0x48c, 0x48f, 0x492, 0x495, 0x499, 0x49d, 0x4a0, 0x4a3, + 0x4a6, 0x4a9, 0x4ac, 0x4b5, 0x4bb, 0x4cf, 0x4e5, 0x4ed, 0x4f0, 0x4f6, + 0x4fe, 0x501, 0x507, 0x509, 0x50d, 0x512, 0x515, 0x518, 0x51c, 0x520, + 0x523, 0x525, 0x528, 0x52c, 0x530, 0x533, 0x535, 0x537, 0x53a, 0x53f, + 0x54a, 0x550, 0x555, 0x55c, 0x561, 0x565, 0x569, 0x56e, 0x575, 0x58a, + 0x58d, 0x596, 0x59a, 0x59f, 0x5a4, 0x5a7, 0x5a9, 0x5bf, 0x5c2, 0x5cd, + 0x5d1, 0x5d4, 0x5d8, 0x5dc, 0x5e4, 0x5e8, 0x5f5, 0x601, 0x60d, 0x615, + 0x619, 0x620, 0x626, 0x62e, 0x633, 0x63c, 0x640, 0x65f, 0x670, 0x673, + 0x677, 0x67a, 0x686, 0x697, 0x69b, 0x6ac, 0x6af, 0x6b3, 0x6b6, 0x6c1, + 0x6d9, 0x6e0, 0x6e2, 0x6e4, 0x6ec, 0x6f1, 0x6f9, 0x703, 0x706, 0x70e, + 0x715, 0x71e, 0x724, 0x728, 0x72e, 0x735, 0x73e, 0x744, 0x74e, 0x750, + 0x753, 0x75b, 0x760, 0x76a, 0x76f, 0x773, }; atn::ATNDeserializer deserializer; diff --git a/src/Parsers/New/ClickHouseParser.g4 b/src/Parsers/New/ClickHouseParser.g4 index 722eaf969c0..8c237f96a5d 100644 --- a/src/Parsers/New/ClickHouseParser.g4 +++ b/src/Parsers/New/ClickHouseParser.g4 @@ -35,31 +35,37 @@ alterStmt ; alterTableClause - : ADD COLUMN (IF NOT EXISTS)? tableColumnDfnt (AFTER nestedIdentifier)? # AlterTableClauseAddColumn - | ADD INDEX (IF NOT EXISTS)? tableIndexDfnt (AFTER nestedIdentifier)? # AlterTableClauseAddIndex - | ATTACH partitionClause (FROM tableIdentifier)? # AlterTableClauseAttach - | CLEAR COLUMN (IF EXISTS)? nestedIdentifier (IN partitionClause)? # AlterTableClauseClear - | COMMENT COLUMN (IF EXISTS)? nestedIdentifier STRING_LITERAL # AlterTableClauseComment - | DELETE WHERE columnExpr # AlterTableClauseDelete - | DETACH partitionClause # AlterTableClauseDetach - | DROP COLUMN (IF EXISTS)? nestedIdentifier # AlterTableClauseDropColumn - | DROP INDEX (IF EXISTS)? nestedIdentifier # AlterTableClauseDropIndex - | DROP partitionClause # AlterTableClauseDropPartition - | FREEZE partitionClause? # AlterTableClauseFreezePartition - | MODIFY COLUMN (IF EXISTS)? nestedIdentifier codecExpr # AlterTableClauseModifyCodec - | MODIFY COLUMN (IF EXISTS)? nestedIdentifier COMMENT STRING_LITERAL # AlterTableClauseModifyComment - | MODIFY COLUMN (IF EXISTS)? nestedIdentifier REMOVE tableColumnPropertyType # AlterTableClauseModifyRemove - | MODIFY COLUMN (IF EXISTS)? tableColumnDfnt # AlterTableClauseModify - | MODIFY ORDER BY columnExpr # AlterTableClauseModifyOrderBy - | MODIFY ttlClause # AlterTableClauseModifyTTL + : ADD COLUMN (IF NOT EXISTS)? tableColumnDfnt (AFTER nestedIdentifier)? # AlterTableClauseAddColumn + | ADD INDEX (IF NOT EXISTS)? tableIndexDfnt (AFTER nestedIdentifier)? # AlterTableClauseAddIndex + | ADD PROJECTION (IF NOT EXISTS)? tableProjectionDfnt (AFTER nestedIdentifier)? # AlterTableClauseAddProjection + | ATTACH partitionClause (FROM tableIdentifier)? # AlterTableClauseAttach + | CLEAR COLUMN (IF EXISTS)? nestedIdentifier (IN partitionClause)? # AlterTableClauseClearColumn + | CLEAR INDEX (IF EXISTS)? nestedIdentifier (IN partitionClause)? # AlterTableClauseClearIndex + | CLEAR PROJECTION (IF EXISTS)? nestedIdentifier (IN partitionClause)? # AlterTableClauseClearProjection + | COMMENT COLUMN (IF EXISTS)? nestedIdentifier STRING_LITERAL # AlterTableClauseComment + | DELETE WHERE columnExpr # AlterTableClauseDelete + | DETACH partitionClause # AlterTableClauseDetach + | DROP COLUMN (IF EXISTS)? nestedIdentifier # AlterTableClauseDropColumn + | DROP INDEX (IF EXISTS)? nestedIdentifier # AlterTableClauseDropIndex + | DROP PROJECTION (IF EXISTS)? nestedIdentifier # AlterTableClauseDropProjection + | DROP partitionClause # AlterTableClauseDropPartition + | FREEZE partitionClause? # AlterTableClauseFreezePartition + | MATERIALIZE INDEX (IF EXISTS)? nestedIdentifier (IN partitionClause)? # AlterTableClauseMaterializeIndex + | MATERIALIZE PROJECTION (IF EXISTS)? nestedIdentifier (IN partitionClause)? # AlterTableClauseMaterializeProjection + | MODIFY COLUMN (IF EXISTS)? nestedIdentifier codecExpr # AlterTableClauseModifyCodec + | MODIFY COLUMN (IF EXISTS)? nestedIdentifier COMMENT STRING_LITERAL # AlterTableClauseModifyComment + | MODIFY COLUMN (IF EXISTS)? nestedIdentifier REMOVE tableColumnPropertyType # AlterTableClauseModifyRemove + | MODIFY COLUMN (IF EXISTS)? tableColumnDfnt # AlterTableClauseModify + | MODIFY ORDER BY columnExpr # AlterTableClauseModifyOrderBy + | MODIFY ttlClause # AlterTableClauseModifyTTL | MOVE partitionClause ( TO DISK STRING_LITERAL | TO VOLUME STRING_LITERAL | TO TABLE tableIdentifier - ) # AlterTableClauseMovePartition - | REMOVE TTL # AlterTableClauseRemoveTTL - | RENAME COLUMN (IF EXISTS)? nestedIdentifier TO nestedIdentifier # AlterTableClauseRename - | REPLACE partitionClause FROM tableIdentifier # AlterTableClauseReplace - | UPDATE assignmentExprList whereClause # AlterTableClauseUpdate + ) # AlterTableClauseMovePartition + | REMOVE TTL # AlterTableClauseRemoveTTL + | RENAME COLUMN (IF EXISTS)? nestedIdentifier TO nestedIdentifier # AlterTableClauseRename + | REPLACE partitionClause FROM tableIdentifier # AlterTableClauseReplace + | UPDATE assignmentExprList whereClause # AlterTableClauseUpdate ; assignmentExprList: assignmentExpr (COMMA assignmentExpr)*; @@ -154,6 +160,7 @@ tableElementExpr : tableColumnDfnt # TableElementExprColumn | CONSTRAINT identifier CHECK columnExpr # TableElementExprConstraint | INDEX tableIndexDfnt # TableElementExprIndex + | PROJECTION tableProjectionDfnt # TableElementExprProjection ; tableColumnDfnt : nestedIdentifier columnTypeExpr tableColumnPropertyExpr? (COMMENT STRING_LITERAL)? codecExpr? (TTL columnExpr)? @@ -161,6 +168,7 @@ tableColumnDfnt ; tableColumnPropertyExpr: (DEFAULT | MATERIALIZED | ALIAS) columnExpr; tableIndexDfnt: nestedIdentifier columnExpr TYPE columnTypeExpr GRANULARITY DECIMAL_LITERAL; +tableProjectionDfnt: nestedIdentifier projectionSelectStmt; codecExpr: CODEC LPAREN codecArgExpr (COMMA codecArgExpr)* RPAREN; codecArgExpr: identifier (LPAREN columnExprList? RPAREN)?; ttlExpr: columnExpr (DELETE | TO DISK STRING_LITERAL | TO VOLUME STRING_LITERAL)?; @@ -212,6 +220,17 @@ optimizeStmt: OPTIMIZE TABLE tableIdentifier clusterClause? partitionClause? FIN renameStmt: RENAME TABLE tableIdentifier TO tableIdentifier (COMMA tableIdentifier TO tableIdentifier)* clusterClause?; +// PROJECTION SELECT statement + +projectionSelectStmt: + LPAREN + withClause? + SELECT columnExprList + groupByClause? + projectionOrderByClause? + RPAREN + ; + // SELECT statement selectUnionStmt: selectStmtWithParens (UNION ALL selectStmtWithParens)*; @@ -240,6 +259,7 @@ whereClause: WHERE columnExpr; groupByClause: GROUP BY ((CUBE | ROLLUP) LPAREN columnExprList RPAREN | columnExprList); havingClause: HAVING columnExpr; orderByClause: ORDER BY orderExprList; +projectionOrderByClause: ORDER BY columnExprList; limitByClause: LIMIT limitExpr BY columnExprList; limitClause: LIMIT limitExpr (WITH TIES)?; settingsClause: SETTINGS settingExprList; @@ -436,7 +456,7 @@ keyword | DISTRIBUTED | DROP | ELSE | END | ENGINE | EVENTS | EXISTS | EXPLAIN | EXPRESSION | EXTRACT | FETCHES | FINAL | FIRST | FLUSH | FOR | FORMAT | FREEZE | FROM | FULL | FUNCTION | GLOBAL | GRANULARITY | GROUP | HAVING | HIERARCHICAL | ID | IF | ILIKE | IN | INDEX | INJECTIVE | INNER | INSERT | INTERVAL | INTO | IS | IS_OBJECT_ID | JOIN | JSON_FALSE | JSON_TRUE | KEY | KILL | LAST | LAYOUT - | LEADING | LEFT | LIFETIME | LIKE | LIMIT | LIVE | LOCAL | LOGS | MATERIALIZED | MAX | MERGES | MIN | MODIFY | MOVE | MUTATION | NO + | LEADING | LEFT | LIFETIME | LIKE | LIMIT | LIVE | LOCAL | LOGS | MATERIALIZE | MATERIALIZED | MAX | MERGES | MIN | MODIFY | MOVE | MUTATION | NO | NOT | NULLS | OFFSET | ON | OPTIMIZE | OR | ORDER | OUTER | OUTFILE | PARTITION | POPULATE | PREWHERE | PRIMARY | RANGE | RELOAD | REMOVE | RENAME | REPLACE | REPLICA | REPLICATED | RIGHT | ROLLUP | SAMPLE | SELECT | SEMI | SENDS | SET | SETTINGS | SHOW | SOURCE | START | STOP | SUBSTRING | SYNC | SYNTAX | SYSTEM | TABLE | TABLES | TEMPORARY | TEST | THEN | TIES | TIMEOUT | TIMESTAMP | TOTALS diff --git a/src/Parsers/New/ClickHouseParser.h b/src/Parsers/New/ClickHouseParser.h index 35e8d81d7f8..ecaff1a5add 100644 --- a/src/Parsers/New/ClickHouseParser.h +++ b/src/Parsers/New/ClickHouseParser.h @@ -30,30 +30,30 @@ public: INSERT = 81, INTERVAL = 82, INTO = 83, IS = 84, IS_OBJECT_ID = 85, JOIN = 86, KEY = 87, KILL = 88, LAST = 89, LAYOUT = 90, LEADING = 91, LEFT = 92, LIFETIME = 93, LIKE = 94, LIMIT = 95, LIVE = 96, LOCAL = 97, LOGS = 98, - MATERIALIZED = 99, MAX = 100, MERGES = 101, MIN = 102, MINUTE = 103, - MODIFY = 104, MONTH = 105, MOVE = 106, MUTATION = 107, NAN_SQL = 108, - NO = 109, NOT = 110, NULL_SQL = 111, NULLS = 112, OFFSET = 113, ON = 114, - OPTIMIZE = 115, OR = 116, ORDER = 117, OUTER = 118, OUTFILE = 119, PARTITION = 120, - POPULATE = 121, PREWHERE = 122, PRIMARY = 123, QUARTER = 124, RANGE = 125, - RELOAD = 126, REMOVE = 127, RENAME = 128, REPLACE = 129, REPLICA = 130, - REPLICATED = 131, RIGHT = 132, ROLLUP = 133, SAMPLE = 134, SECOND = 135, - SELECT = 136, SEMI = 137, SENDS = 138, SET = 139, SETTINGS = 140, SHOW = 141, - SOURCE = 142, START = 143, STOP = 144, SUBSTRING = 145, SYNC = 146, - SYNTAX = 147, SYSTEM = 148, TABLE = 149, TABLES = 150, TEMPORARY = 151, - TEST = 152, THEN = 153, TIES = 154, TIMEOUT = 155, TIMESTAMP = 156, - TO = 157, TOP = 158, TOTALS = 159, TRAILING = 160, TRIM = 161, TRUNCATE = 162, - TTL = 163, TYPE = 164, UNION = 165, UPDATE = 166, USE = 167, USING = 168, - UUID = 169, VALUES = 170, VIEW = 171, VOLUME = 172, WATCH = 173, WEEK = 174, - WHEN = 175, WHERE = 176, WITH = 177, YEAR = 178, JSON_FALSE = 179, JSON_TRUE = 180, - IDENTIFIER = 181, FLOATING_LITERAL = 182, OCTAL_LITERAL = 183, DECIMAL_LITERAL = 184, - HEXADECIMAL_LITERAL = 185, STRING_LITERAL = 186, ARROW = 187, ASTERISK = 188, - BACKQUOTE = 189, BACKSLASH = 190, COLON = 191, COMMA = 192, CONCAT = 193, - DASH = 194, DOT = 195, EQ_DOUBLE = 196, EQ_SINGLE = 197, GE = 198, GT = 199, - LBRACE = 200, LBRACKET = 201, LE = 202, LPAREN = 203, LT = 204, NOT_EQ = 205, - PERCENT = 206, PLUS = 207, QUERY = 208, QUOTE_DOUBLE = 209, QUOTE_SINGLE = 210, - RBRACE = 211, RBRACKET = 212, RPAREN = 213, SEMICOLON = 214, SLASH = 215, - UNDERSCORE = 216, MULTI_LINE_COMMENT = 217, SINGLE_LINE_COMMENT = 218, - WHITESPACE = 219 + MATERIALIZED = 99, MATERIALIZE = 100, MAX = 101, MERGES = 102, MIN = 103, + MINUTE = 104, MODIFY = 105, MONTH = 106, MOVE = 107, MUTATION = 108, + NAN_SQL = 109, NO = 110, NOT = 111, NULL_SQL = 112, NULLS = 113, OFFSET = 114, + ON = 115, OPTIMIZE = 116, OR = 117, ORDER = 118, OUTER = 119, OUTFILE = 120, + PARTITION = 121, POPULATE = 122, PREWHERE = 123, PRIMARY = 124, PROJECTION = 125, + QUARTER = 126, RANGE = 127, RELOAD = 128, REMOVE = 129, RENAME = 130, + REPLACE = 131, REPLICA = 132, REPLICATED = 133, RIGHT = 134, ROLLUP = 135, + SAMPLE = 136, SECOND = 137, SELECT = 138, SEMI = 139, SENDS = 140, SET = 141, + SETTINGS = 142, SHOW = 143, SOURCE = 144, START = 145, STOP = 146, SUBSTRING = 147, + SYNC = 148, SYNTAX = 149, SYSTEM = 150, TABLE = 151, TABLES = 152, TEMPORARY = 153, + TEST = 154, THEN = 155, TIES = 156, TIMEOUT = 157, TIMESTAMP = 158, + TO = 159, TOP = 160, TOTALS = 161, TRAILING = 162, TRIM = 163, TRUNCATE = 164, + TTL = 165, TYPE = 166, UNION = 167, UPDATE = 168, USE = 169, USING = 170, + UUID = 171, VALUES = 172, VIEW = 173, VOLUME = 174, WATCH = 175, WEEK = 176, + WHEN = 177, WHERE = 178, WITH = 179, YEAR = 180, JSON_FALSE = 181, JSON_TRUE = 182, + IDENTIFIER = 183, FLOATING_LITERAL = 184, OCTAL_LITERAL = 185, DECIMAL_LITERAL = 186, + HEXADECIMAL_LITERAL = 187, STRING_LITERAL = 188, ARROW = 189, ASTERISK = 190, + BACKQUOTE = 191, BACKSLASH = 192, COLON = 193, COMMA = 194, CONCAT = 195, + DASH = 196, DOT = 197, EQ_DOUBLE = 198, EQ_SINGLE = 199, GE = 200, GT = 201, + LBRACE = 202, LBRACKET = 203, LE = 204, LPAREN = 205, LT = 206, NOT_EQ = 207, + PERCENT = 208, PLUS = 209, QUERY = 210, QUOTE_DOUBLE = 211, QUOTE_SINGLE = 212, + RBRACE = 213, RBRACKET = 214, RPAREN = 215, SEMICOLON = 216, SLASH = 217, + UNDERSCORE = 218, MULTI_LINE_COMMENT = 219, SINGLE_LINE_COMMENT = 220, + WHITESPACE = 221 }; enum { @@ -68,30 +68,31 @@ public: RuleEngineClause = 26, RulePartitionByClause = 27, RulePrimaryKeyClause = 28, RuleSampleByClause = 29, RuleTtlClause = 30, RuleEngineExpr = 31, RuleTableElementExpr = 32, RuleTableColumnDfnt = 33, RuleTableColumnPropertyExpr = 34, RuleTableIndexDfnt = 35, - RuleCodecExpr = 36, RuleCodecArgExpr = 37, RuleTtlExpr = 38, RuleDescribeStmt = 39, - RuleDropStmt = 40, RuleExistsStmt = 41, RuleExplainStmt = 42, RuleInsertStmt = 43, - RuleColumnsClause = 44, RuleDataClause = 45, RuleKillStmt = 46, RuleOptimizeStmt = 47, - RuleRenameStmt = 48, RuleSelectUnionStmt = 49, RuleSelectStmtWithParens = 50, - RuleSelectStmt = 51, RuleWithClause = 52, RuleTopClause = 53, RuleFromClause = 54, - RuleArrayJoinClause = 55, RulePrewhereClause = 56, RuleWhereClause = 57, - RuleGroupByClause = 58, RuleHavingClause = 59, RuleOrderByClause = 60, - RuleLimitByClause = 61, RuleLimitClause = 62, RuleSettingsClause = 63, - RuleJoinExpr = 64, RuleJoinOp = 65, RuleJoinOpCross = 66, RuleJoinConstraintClause = 67, - RuleSampleClause = 68, RuleLimitExpr = 69, RuleOrderExprList = 70, RuleOrderExpr = 71, - RuleRatioExpr = 72, RuleSettingExprList = 73, RuleSettingExpr = 74, - RuleSetStmt = 75, RuleShowStmt = 76, RuleSystemStmt = 77, RuleTruncateStmt = 78, - RuleUseStmt = 79, RuleWatchStmt = 80, RuleColumnTypeExpr = 81, RuleColumnExprList = 82, - RuleColumnsExpr = 83, RuleColumnExpr = 84, RuleColumnArgList = 85, RuleColumnArgExpr = 86, - RuleColumnLambdaExpr = 87, RuleColumnIdentifier = 88, RuleNestedIdentifier = 89, - RuleTableExpr = 90, RuleTableFunctionExpr = 91, RuleTableIdentifier = 92, - RuleTableArgList = 93, RuleTableArgExpr = 94, RuleDatabaseIdentifier = 95, - RuleFloatingLiteral = 96, RuleNumberLiteral = 97, RuleLiteral = 98, - RuleInterval = 99, RuleKeyword = 100, RuleKeywordForAlias = 101, RuleAlias = 102, - RuleIdentifier = 103, RuleIdentifierOrNull = 104, RuleEnumValue = 105 + RuleTableProjectionDfnt = 36, RuleCodecExpr = 37, RuleCodecArgExpr = 38, + RuleTtlExpr = 39, RuleDescribeStmt = 40, RuleDropStmt = 41, RuleExistsStmt = 42, + RuleExplainStmt = 43, RuleInsertStmt = 44, RuleColumnsClause = 45, RuleDataClause = 46, + RuleKillStmt = 47, RuleOptimizeStmt = 48, RuleRenameStmt = 49, RuleProjectionSelectStmt = 50, + RuleSelectUnionStmt = 51, RuleSelectStmtWithParens = 52, RuleSelectStmt = 53, + RuleWithClause = 54, RuleTopClause = 55, RuleFromClause = 56, RuleArrayJoinClause = 57, + RulePrewhereClause = 58, RuleWhereClause = 59, RuleGroupByClause = 60, + RuleHavingClause = 61, RuleOrderByClause = 62, RuleProjectionOrderByClause = 63, + RuleLimitByClause = 64, RuleLimitClause = 65, RuleSettingsClause = 66, + RuleJoinExpr = 67, RuleJoinOp = 68, RuleJoinOpCross = 69, RuleJoinConstraintClause = 70, + RuleSampleClause = 71, RuleLimitExpr = 72, RuleOrderExprList = 73, RuleOrderExpr = 74, + RuleRatioExpr = 75, RuleSettingExprList = 76, RuleSettingExpr = 77, + RuleSetStmt = 78, RuleShowStmt = 79, RuleSystemStmt = 80, RuleTruncateStmt = 81, + RuleUseStmt = 82, RuleWatchStmt = 83, RuleColumnTypeExpr = 84, RuleColumnExprList = 85, + RuleColumnsExpr = 86, RuleColumnExpr = 87, RuleColumnArgList = 88, RuleColumnArgExpr = 89, + RuleColumnLambdaExpr = 90, RuleColumnIdentifier = 91, RuleNestedIdentifier = 92, + RuleTableExpr = 93, RuleTableFunctionExpr = 94, RuleTableIdentifier = 95, + RuleTableArgList = 96, RuleTableArgExpr = 97, RuleDatabaseIdentifier = 98, + RuleFloatingLiteral = 99, RuleNumberLiteral = 100, RuleLiteral = 101, + RuleInterval = 102, RuleKeyword = 103, RuleKeywordForAlias = 104, RuleAlias = 105, + RuleIdentifier = 106, RuleIdentifierOrNull = 107, RuleEnumValue = 108 }; ClickHouseParser(antlr4::TokenStream *input); - ~ClickHouseParser() override; + ~ClickHouseParser(); virtual std::string getGrammarFileName() const override; virtual const antlr4::atn::ATN& getATN() const override { return _atn; }; @@ -136,6 +137,7 @@ public: class TableColumnDfntContext; class TableColumnPropertyExprContext; class TableIndexDfntContext; + class TableProjectionDfntContext; class CodecExprContext; class CodecArgExprContext; class TtlExprContext; @@ -149,6 +151,7 @@ public: class KillStmtContext; class OptimizeStmtContext; class RenameStmtContext; + class ProjectionSelectStmtContext; class SelectUnionStmtContext; class SelectStmtWithParensContext; class SelectStmtContext; @@ -161,6 +164,7 @@ public: class GroupByClauseContext; class HavingClauseContext; class OrderByClauseContext; + class ProjectionOrderByClauseContext; class LimitByClauseContext; class LimitClauseContext; class SettingsClauseContext; @@ -313,44 +317,6 @@ public: virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; - class AlterTableClauseRenameContext : public AlterTableClauseContext { - public: - AlterTableClauseRenameContext(AlterTableClauseContext *ctx); - - antlr4::tree::TerminalNode *RENAME(); - antlr4::tree::TerminalNode *COLUMN(); - std::vector nestedIdentifier(); - NestedIdentifierContext* nestedIdentifier(size_t i); - antlr4::tree::TerminalNode *TO(); - antlr4::tree::TerminalNode *IF(); - antlr4::tree::TerminalNode *EXISTS(); - - virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; - }; - - class AlterTableClauseFreezePartitionContext : public AlterTableClauseContext { - public: - AlterTableClauseFreezePartitionContext(AlterTableClauseContext *ctx); - - antlr4::tree::TerminalNode *FREEZE(); - PartitionClauseContext *partitionClause(); - - virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; - }; - - class AlterTableClauseModifyContext : public AlterTableClauseContext { - public: - AlterTableClauseModifyContext(AlterTableClauseContext *ctx); - - antlr4::tree::TerminalNode *MODIFY(); - antlr4::tree::TerminalNode *COLUMN(); - TableColumnDfntContext *tableColumnDfnt(); - antlr4::tree::TerminalNode *IF(); - antlr4::tree::TerminalNode *EXISTS(); - - virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; - }; - class AlterTableClauseModifyOrderByContext : public AlterTableClauseContext { public: AlterTableClauseModifyOrderByContext(AlterTableClauseContext *ctx); @@ -363,16 +329,6 @@ public: virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; - class AlterTableClauseRemoveTTLContext : public AlterTableClauseContext { - public: - AlterTableClauseRemoveTTLContext(AlterTableClauseContext *ctx); - - antlr4::tree::TerminalNode *REMOVE(); - antlr4::tree::TerminalNode *TTL(); - - virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; - }; - class AlterTableClauseUpdateContext : public AlterTableClauseContext { public: AlterTableClauseUpdateContext(AlterTableClauseContext *ctx); @@ -384,6 +340,21 @@ public: virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; + class AlterTableClauseClearProjectionContext : public AlterTableClauseContext { + public: + AlterTableClauseClearProjectionContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *CLEAR(); + antlr4::tree::TerminalNode *PROJECTION(); + NestedIdentifierContext *nestedIdentifier(); + antlr4::tree::TerminalNode *IF(); + antlr4::tree::TerminalNode *EXISTS(); + antlr4::tree::TerminalNode *IN(); + PartitionClauseContext *partitionClause(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + class AlterTableClauseModifyRemoveContext : public AlterTableClauseContext { public: AlterTableClauseModifyRemoveContext(AlterTableClauseContext *ctx); @@ -410,20 +381,6 @@ public: virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; - class AlterTableClauseModifyCodecContext : public AlterTableClauseContext { - public: - AlterTableClauseModifyCodecContext(AlterTableClauseContext *ctx); - - antlr4::tree::TerminalNode *MODIFY(); - antlr4::tree::TerminalNode *COLUMN(); - NestedIdentifierContext *nestedIdentifier(); - CodecExprContext *codecExpr(); - antlr4::tree::TerminalNode *IF(); - antlr4::tree::TerminalNode *EXISTS(); - - virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; - }; - class AlterTableClauseCommentContext : public AlterTableClauseContext { public: AlterTableClauseCommentContext(AlterTableClauseContext *ctx); @@ -438,18 +395,6 @@ public: virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; - class AlterTableClauseAttachContext : public AlterTableClauseContext { - public: - AlterTableClauseAttachContext(AlterTableClauseContext *ctx); - - antlr4::tree::TerminalNode *ATTACH(); - PartitionClauseContext *partitionClause(); - antlr4::tree::TerminalNode *FROM(); - TableIdentifierContext *tableIdentifier(); - - virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; - }; - class AlterTableClauseDropColumnContext : public AlterTableClauseContext { public: AlterTableClauseDropColumnContext(AlterTableClauseContext *ctx); @@ -463,21 +408,6 @@ public: virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; - class AlterTableClauseClearContext : public AlterTableClauseContext { - public: - AlterTableClauseClearContext(AlterTableClauseContext *ctx); - - antlr4::tree::TerminalNode *CLEAR(); - antlr4::tree::TerminalNode *COLUMN(); - NestedIdentifierContext *nestedIdentifier(); - antlr4::tree::TerminalNode *IF(); - antlr4::tree::TerminalNode *EXISTS(); - antlr4::tree::TerminalNode *IN(); - PartitionClauseContext *partitionClause(); - - virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; - }; - class AlterTableClauseDetachContext : public AlterTableClauseContext { public: AlterTableClauseDetachContext(AlterTableClauseContext *ctx); @@ -488,19 +418,6 @@ public: virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; - class AlterTableClauseDropIndexContext : public AlterTableClauseContext { - public: - AlterTableClauseDropIndexContext(AlterTableClauseContext *ctx); - - antlr4::tree::TerminalNode *DROP(); - antlr4::tree::TerminalNode *INDEX(); - NestedIdentifierContext *nestedIdentifier(); - antlr4::tree::TerminalNode *IF(); - antlr4::tree::TerminalNode *EXISTS(); - - virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; - }; - class AlterTableClauseAddIndexContext : public AlterTableClauseContext { public: AlterTableClauseAddIndexContext(AlterTableClauseContext *ctx); @@ -527,6 +444,182 @@ public: virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; + class AlterTableClauseMaterializeIndexContext : public AlterTableClauseContext { + public: + AlterTableClauseMaterializeIndexContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *MATERIALIZE(); + antlr4::tree::TerminalNode *INDEX(); + NestedIdentifierContext *nestedIdentifier(); + antlr4::tree::TerminalNode *IF(); + antlr4::tree::TerminalNode *EXISTS(); + antlr4::tree::TerminalNode *IN(); + PartitionClauseContext *partitionClause(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseMaterializeProjectionContext : public AlterTableClauseContext { + public: + AlterTableClauseMaterializeProjectionContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *MATERIALIZE(); + antlr4::tree::TerminalNode *PROJECTION(); + NestedIdentifierContext *nestedIdentifier(); + antlr4::tree::TerminalNode *IF(); + antlr4::tree::TerminalNode *EXISTS(); + antlr4::tree::TerminalNode *IN(); + PartitionClauseContext *partitionClause(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseMovePartitionContext : public AlterTableClauseContext { + public: + AlterTableClauseMovePartitionContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *MOVE(); + PartitionClauseContext *partitionClause(); + antlr4::tree::TerminalNode *TO(); + antlr4::tree::TerminalNode *DISK(); + antlr4::tree::TerminalNode *STRING_LITERAL(); + antlr4::tree::TerminalNode *VOLUME(); + antlr4::tree::TerminalNode *TABLE(); + TableIdentifierContext *tableIdentifier(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseRenameContext : public AlterTableClauseContext { + public: + AlterTableClauseRenameContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *RENAME(); + antlr4::tree::TerminalNode *COLUMN(); + std::vector nestedIdentifier(); + NestedIdentifierContext* nestedIdentifier(size_t i); + antlr4::tree::TerminalNode *TO(); + antlr4::tree::TerminalNode *IF(); + antlr4::tree::TerminalNode *EXISTS(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseFreezePartitionContext : public AlterTableClauseContext { + public: + AlterTableClauseFreezePartitionContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *FREEZE(); + PartitionClauseContext *partitionClause(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseClearColumnContext : public AlterTableClauseContext { + public: + AlterTableClauseClearColumnContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *CLEAR(); + antlr4::tree::TerminalNode *COLUMN(); + NestedIdentifierContext *nestedIdentifier(); + antlr4::tree::TerminalNode *IF(); + antlr4::tree::TerminalNode *EXISTS(); + antlr4::tree::TerminalNode *IN(); + PartitionClauseContext *partitionClause(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseModifyContext : public AlterTableClauseContext { + public: + AlterTableClauseModifyContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *MODIFY(); + antlr4::tree::TerminalNode *COLUMN(); + TableColumnDfntContext *tableColumnDfnt(); + antlr4::tree::TerminalNode *IF(); + antlr4::tree::TerminalNode *EXISTS(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseClearIndexContext : public AlterTableClauseContext { + public: + AlterTableClauseClearIndexContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *CLEAR(); + antlr4::tree::TerminalNode *INDEX(); + NestedIdentifierContext *nestedIdentifier(); + antlr4::tree::TerminalNode *IF(); + antlr4::tree::TerminalNode *EXISTS(); + antlr4::tree::TerminalNode *IN(); + PartitionClauseContext *partitionClause(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseRemoveTTLContext : public AlterTableClauseContext { + public: + AlterTableClauseRemoveTTLContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *REMOVE(); + antlr4::tree::TerminalNode *TTL(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseModifyCodecContext : public AlterTableClauseContext { + public: + AlterTableClauseModifyCodecContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *MODIFY(); + antlr4::tree::TerminalNode *COLUMN(); + NestedIdentifierContext *nestedIdentifier(); + CodecExprContext *codecExpr(); + antlr4::tree::TerminalNode *IF(); + antlr4::tree::TerminalNode *EXISTS(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseAttachContext : public AlterTableClauseContext { + public: + AlterTableClauseAttachContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *ATTACH(); + PartitionClauseContext *partitionClause(); + antlr4::tree::TerminalNode *FROM(); + TableIdentifierContext *tableIdentifier(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseDropProjectionContext : public AlterTableClauseContext { + public: + AlterTableClauseDropProjectionContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *DROP(); + antlr4::tree::TerminalNode *PROJECTION(); + NestedIdentifierContext *nestedIdentifier(); + antlr4::tree::TerminalNode *IF(); + antlr4::tree::TerminalNode *EXISTS(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + + class AlterTableClauseDropIndexContext : public AlterTableClauseContext { + public: + AlterTableClauseDropIndexContext(AlterTableClauseContext *ctx); + + antlr4::tree::TerminalNode *DROP(); + antlr4::tree::TerminalNode *INDEX(); + NestedIdentifierContext *nestedIdentifier(); + antlr4::tree::TerminalNode *IF(); + antlr4::tree::TerminalNode *EXISTS(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + class AlterTableClauseModifyCommentContext : public AlterTableClauseContext { public: AlterTableClauseModifyCommentContext(AlterTableClauseContext *ctx); @@ -552,18 +645,18 @@ public: virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; - class AlterTableClauseMovePartitionContext : public AlterTableClauseContext { + class AlterTableClauseAddProjectionContext : public AlterTableClauseContext { public: - AlterTableClauseMovePartitionContext(AlterTableClauseContext *ctx); + AlterTableClauseAddProjectionContext(AlterTableClauseContext *ctx); - antlr4::tree::TerminalNode *MOVE(); - PartitionClauseContext *partitionClause(); - antlr4::tree::TerminalNode *TO(); - antlr4::tree::TerminalNode *DISK(); - antlr4::tree::TerminalNode *STRING_LITERAL(); - antlr4::tree::TerminalNode *VOLUME(); - antlr4::tree::TerminalNode *TABLE(); - TableIdentifierContext *tableIdentifier(); + antlr4::tree::TerminalNode *ADD(); + antlr4::tree::TerminalNode *PROJECTION(); + TableProjectionDfntContext *tableProjectionDfnt(); + antlr4::tree::TerminalNode *IF(); + antlr4::tree::TerminalNode *NOT(); + antlr4::tree::TerminalNode *EXISTS(); + antlr4::tree::TerminalNode *AFTER(); + NestedIdentifierContext *nestedIdentifier(); virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; }; @@ -1257,6 +1350,16 @@ public: }; + class TableElementExprProjectionContext : public TableElementExprContext { + public: + TableElementExprProjectionContext(TableElementExprContext *ctx); + + antlr4::tree::TerminalNode *PROJECTION(); + TableProjectionDfntContext *tableProjectionDfnt(); + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + }; + class TableElementExprConstraintContext : public TableElementExprContext { public: TableElementExprConstraintContext(TableElementExprContext *ctx); @@ -1344,6 +1447,20 @@ public: TableIndexDfntContext* tableIndexDfnt(); + class TableProjectionDfntContext : public antlr4::ParserRuleContext { + public: + TableProjectionDfntContext(antlr4::ParserRuleContext *parent, size_t invokingState); + virtual size_t getRuleIndex() const override; + NestedIdentifierContext *nestedIdentifier(); + ProjectionSelectStmtContext *projectionSelectStmt(); + + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + + }; + + TableProjectionDfntContext* tableProjectionDfnt(); + class CodecExprContext : public antlr4::ParserRuleContext { public: CodecExprContext(antlr4::ParserRuleContext *parent, size_t invokingState); @@ -1671,6 +1788,25 @@ public: RenameStmtContext* renameStmt(); + class ProjectionSelectStmtContext : public antlr4::ParserRuleContext { + public: + ProjectionSelectStmtContext(antlr4::ParserRuleContext *parent, size_t invokingState); + virtual size_t getRuleIndex() const override; + antlr4::tree::TerminalNode *LPAREN(); + antlr4::tree::TerminalNode *SELECT(); + ColumnExprListContext *columnExprList(); + antlr4::tree::TerminalNode *RPAREN(); + WithClauseContext *withClause(); + GroupByClauseContext *groupByClause(); + ProjectionOrderByClauseContext *projectionOrderByClause(); + + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + + }; + + ProjectionSelectStmtContext* projectionSelectStmt(); + class SelectUnionStmtContext : public antlr4::ParserRuleContext { public: SelectUnionStmtContext(antlr4::ParserRuleContext *parent, size_t invokingState); @@ -1874,6 +2010,21 @@ public: OrderByClauseContext* orderByClause(); + class ProjectionOrderByClauseContext : public antlr4::ParserRuleContext { + public: + ProjectionOrderByClauseContext(antlr4::ParserRuleContext *parent, size_t invokingState); + virtual size_t getRuleIndex() const override; + antlr4::tree::TerminalNode *ORDER(); + antlr4::tree::TerminalNode *BY(); + ColumnExprListContext *columnExprList(); + + + virtual antlrcpp::Any accept(antlr4::tree::ParseTreeVisitor *visitor) override; + + }; + + ProjectionOrderByClauseContext* projectionOrderByClause(); + class LimitByClauseContext : public antlr4::ParserRuleContext { public: LimitByClauseContext(antlr4::ParserRuleContext *parent, size_t invokingState); @@ -3273,6 +3424,7 @@ public: antlr4::tree::TerminalNode *LIVE(); antlr4::tree::TerminalNode *LOCAL(); antlr4::tree::TerminalNode *LOGS(); + antlr4::tree::TerminalNode *MATERIALIZE(); antlr4::tree::TerminalNode *MATERIALIZED(); antlr4::tree::TerminalNode *MAX(); antlr4::tree::TerminalNode *MERGES(); diff --git a/src/Parsers/New/ClickHouseParserVisitor.h b/src/Parsers/New/ClickHouseParserVisitor.h index fbe50af7e9a..0cb1876f556 100644 --- a/src/Parsers/New/ClickHouseParserVisitor.h +++ b/src/Parsers/New/ClickHouseParserVisitor.h @@ -30,9 +30,15 @@ public: virtual antlrcpp::Any visitAlterTableClauseAddIndex(ClickHouseParser::AlterTableClauseAddIndexContext *context) = 0; + virtual antlrcpp::Any visitAlterTableClauseAddProjection(ClickHouseParser::AlterTableClauseAddProjectionContext *context) = 0; + virtual antlrcpp::Any visitAlterTableClauseAttach(ClickHouseParser::AlterTableClauseAttachContext *context) = 0; - virtual antlrcpp::Any visitAlterTableClauseClear(ClickHouseParser::AlterTableClauseClearContext *context) = 0; + virtual antlrcpp::Any visitAlterTableClauseClearColumn(ClickHouseParser::AlterTableClauseClearColumnContext *context) = 0; + + virtual antlrcpp::Any visitAlterTableClauseClearIndex(ClickHouseParser::AlterTableClauseClearIndexContext *context) = 0; + + virtual antlrcpp::Any visitAlterTableClauseClearProjection(ClickHouseParser::AlterTableClauseClearProjectionContext *context) = 0; virtual antlrcpp::Any visitAlterTableClauseComment(ClickHouseParser::AlterTableClauseCommentContext *context) = 0; @@ -44,10 +50,16 @@ public: virtual antlrcpp::Any visitAlterTableClauseDropIndex(ClickHouseParser::AlterTableClauseDropIndexContext *context) = 0; + virtual antlrcpp::Any visitAlterTableClauseDropProjection(ClickHouseParser::AlterTableClauseDropProjectionContext *context) = 0; + virtual antlrcpp::Any visitAlterTableClauseDropPartition(ClickHouseParser::AlterTableClauseDropPartitionContext *context) = 0; virtual antlrcpp::Any visitAlterTableClauseFreezePartition(ClickHouseParser::AlterTableClauseFreezePartitionContext *context) = 0; + virtual antlrcpp::Any visitAlterTableClauseMaterializeIndex(ClickHouseParser::AlterTableClauseMaterializeIndexContext *context) = 0; + + virtual antlrcpp::Any visitAlterTableClauseMaterializeProjection(ClickHouseParser::AlterTableClauseMaterializeProjectionContext *context) = 0; + virtual antlrcpp::Any visitAlterTableClauseModifyCodec(ClickHouseParser::AlterTableClauseModifyCodecContext *context) = 0; virtual antlrcpp::Any visitAlterTableClauseModifyComment(ClickHouseParser::AlterTableClauseModifyCommentContext *context) = 0; @@ -146,12 +158,16 @@ public: virtual antlrcpp::Any visitTableElementExprIndex(ClickHouseParser::TableElementExprIndexContext *context) = 0; + virtual antlrcpp::Any visitTableElementExprProjection(ClickHouseParser::TableElementExprProjectionContext *context) = 0; + virtual antlrcpp::Any visitTableColumnDfnt(ClickHouseParser::TableColumnDfntContext *context) = 0; virtual antlrcpp::Any visitTableColumnPropertyExpr(ClickHouseParser::TableColumnPropertyExprContext *context) = 0; virtual antlrcpp::Any visitTableIndexDfnt(ClickHouseParser::TableIndexDfntContext *context) = 0; + virtual antlrcpp::Any visitTableProjectionDfnt(ClickHouseParser::TableProjectionDfntContext *context) = 0; + virtual antlrcpp::Any visitCodecExpr(ClickHouseParser::CodecExprContext *context) = 0; virtual antlrcpp::Any visitCodecArgExpr(ClickHouseParser::CodecArgExprContext *context) = 0; @@ -186,6 +202,8 @@ public: virtual antlrcpp::Any visitRenameStmt(ClickHouseParser::RenameStmtContext *context) = 0; + virtual antlrcpp::Any visitProjectionSelectStmt(ClickHouseParser::ProjectionSelectStmtContext *context) = 0; + virtual antlrcpp::Any visitSelectUnionStmt(ClickHouseParser::SelectUnionStmtContext *context) = 0; virtual antlrcpp::Any visitSelectStmtWithParens(ClickHouseParser::SelectStmtWithParensContext *context) = 0; @@ -210,6 +228,8 @@ public: virtual antlrcpp::Any visitOrderByClause(ClickHouseParser::OrderByClauseContext *context) = 0; + virtual antlrcpp::Any visitProjectionOrderByClause(ClickHouseParser::ProjectionOrderByClauseContext *context) = 0; + virtual antlrcpp::Any visitLimitByClause(ClickHouseParser::LimitByClauseContext *context) = 0; virtual antlrcpp::Any visitLimitClause(ClickHouseParser::LimitClauseContext *context) = 0; diff --git a/src/Parsers/New/ParseTreeVisitor.h b/src/Parsers/New/ParseTreeVisitor.h index 2c204ead972..3b07c1beed8 100644 --- a/src/Parsers/New/ParseTreeVisitor.h +++ b/src/Parsers/New/ParseTreeVisitor.h @@ -19,15 +19,21 @@ public: // AlterTableQuery antlrcpp::Any visitAlterTableClauseAddColumn(ClickHouseParser::AlterTableClauseAddColumnContext * ctx) override; antlrcpp::Any visitAlterTableClauseAddIndex(ClickHouseParser::AlterTableClauseAddIndexContext * ctx) override; + antlrcpp::Any visitAlterTableClauseAddProjection(ClickHouseParser::AlterTableClauseAddProjectionContext * ctx) override; antlrcpp::Any visitAlterTableClauseAttach(ClickHouseParser::AlterTableClauseAttachContext * ctx) override; - antlrcpp::Any visitAlterTableClauseClear(ClickHouseParser::AlterTableClauseClearContext * ctx) override; + antlrcpp::Any visitAlterTableClauseClearColumn(ClickHouseParser::AlterTableClauseClearColumnContext * ctx) override; + antlrcpp::Any visitAlterTableClauseClearIndex(ClickHouseParser::AlterTableClauseClearIndexContext * ctx) override; + antlrcpp::Any visitAlterTableClauseClearProjection(ClickHouseParser::AlterTableClauseClearProjectionContext * ctx) override; antlrcpp::Any visitAlterTableClauseComment(ClickHouseParser::AlterTableClauseCommentContext * ctx) override; antlrcpp::Any visitAlterTableClauseDelete(ClickHouseParser::AlterTableClauseDeleteContext * ctx) override; antlrcpp::Any visitAlterTableClauseDetach(ClickHouseParser::AlterTableClauseDetachContext * ctx) override; antlrcpp::Any visitAlterTableClauseDropColumn(ClickHouseParser::AlterTableClauseDropColumnContext * ctx) override; antlrcpp::Any visitAlterTableClauseDropIndex(ClickHouseParser::AlterTableClauseDropIndexContext * ctx) override; + antlrcpp::Any visitAlterTableClauseDropProjection(ClickHouseParser::AlterTableClauseDropProjectionContext * ctx) override; antlrcpp::Any visitAlterTableClauseDropPartition(ClickHouseParser::AlterTableClauseDropPartitionContext * ctx) override; antlrcpp::Any visitAlterTableClauseFreezePartition(ClickHouseParser::AlterTableClauseFreezePartitionContext * ctx) override; + antlrcpp::Any visitAlterTableClauseMaterializeIndex(ClickHouseParser::AlterTableClauseMaterializeIndexContext * ctx) override; + antlrcpp::Any visitAlterTableClauseMaterializeProjection(ClickHouseParser::AlterTableClauseMaterializeProjectionContext * ctx) override; antlrcpp::Any visitAlterTableClauseModify(ClickHouseParser::AlterTableClauseModifyContext * ctx) override; antlrcpp::Any visitAlterTableClauseModifyCodec(ClickHouseParser::AlterTableClauseModifyCodecContext * ctx) override; antlrcpp::Any visitAlterTableClauseModifyComment(ClickHouseParser::AlterTableClauseModifyCommentContext * ctx) override; @@ -162,6 +168,7 @@ public: antlrcpp::Any visitRenameStmt(ClickHouseParser::RenameStmtContext * ctx) override; // SelectUnionQuery + antlrcpp::Any visitProjectionSelectStmt(ClickHouseParser::ProjectionSelectStmtContext * ctx) override; antlrcpp::Any visitSelectStmt(ClickHouseParser::SelectStmtContext * ctx) override; antlrcpp::Any visitSelectStmtWithParens(ClickHouseParser::SelectStmtWithParensContext * ctx) override; antlrcpp::Any visitSelectUnionStmt(ClickHouseParser::SelectUnionStmtContext * ctx) override; @@ -190,7 +197,9 @@ public: antlrcpp::Any visitTableElementExprColumn(ClickHouseParser::TableElementExprColumnContext * ctx) override; antlrcpp::Any visitTableElementExprConstraint(ClickHouseParser::TableElementExprConstraintContext * ctx) override; antlrcpp::Any visitTableElementExprIndex(ClickHouseParser::TableElementExprIndexContext * ctx) override; + antlrcpp::Any visitTableElementExprProjection(ClickHouseParser::TableElementExprProjectionContext * ctx) override; antlrcpp::Any visitTableIndexDfnt(ClickHouseParser::TableIndexDfntContext * ctx) override; + antlrcpp::Any visitTableProjectionDfnt(ClickHouseParser::TableProjectionDfntContext * ctx) override; // TableExpr antlrcpp::Any visitTableArgExpr(ClickHouseParser::TableArgExprContext * ctx) override; @@ -236,6 +245,7 @@ public: antlrcpp::Any visitGroupByClause(ClickHouseParser::GroupByClauseContext *ctx) override; antlrcpp::Any visitHavingClause(ClickHouseParser::HavingClauseContext *ctx) override; antlrcpp::Any visitOrderByClause(ClickHouseParser::OrderByClauseContext *ctx) override; + antlrcpp::Any visitProjectionOrderByClause(ClickHouseParser::ProjectionOrderByClauseContext *ctx) override; antlrcpp::Any visitLimitByClause(ClickHouseParser::LimitByClauseContext *ctx) override; antlrcpp::Any visitLimitClause(ClickHouseParser::LimitClauseContext *ctx) override; antlrcpp::Any visitSettingsClause(ClickHouseParser::SettingsClauseContext *ctx) override; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 4bbe9cc0848..b8ac6ed09eb 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -177,8 +177,7 @@ bool ParserProjectionDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & auto projection = std::make_shared(); projection->name = name->as().name(); - projection->query = query; - projection->children.emplace_back(projection->query); + projection->set(projection->query, query); node = projection; return true; diff --git a/src/Parsers/ParserProjectionSelectQuery.cpp b/src/Parsers/ParserProjectionSelectQuery.cpp index d115acb0b4c..0467f84de2a 100644 --- a/src/Parsers/ParserProjectionSelectQuery.cpp +++ b/src/Parsers/ParserProjectionSelectQuery.cpp @@ -17,20 +17,15 @@ bool ParserProjectionSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ParserKeyword s_with("WITH"); ParserKeyword s_select("SELECT"); - ParserKeyword s_distinct("DISTINCT"); - ParserKeyword s_where("WHERE"); ParserKeyword s_group_by("GROUP BY"); ParserKeyword s_order_by("ORDER BY"); - ParserNotEmptyExpressionList exp_list(false); ParserNotEmptyExpressionList exp_list_for_with_clause(false); ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword. - ParserExpressionWithOptionalAlias exp_elem(false); ParserExpression order_expression_p; ASTPtr with_expression_list; ASTPtr select_expression_list; - ASTPtr where_expression; ASTPtr group_expression_list; ASTPtr order_expression; @@ -48,21 +43,10 @@ bool ParserProjectionSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (!s_select.ignore(pos, expected)) return false; - if (s_distinct.ignore(pos, expected)) - select_query->distinct = true; - if (!exp_list_for_select_clause.parse(pos, select_expression_list, expected)) return false; } - // TODO: wait for condition normalizer to land - /// WHERE expr - // if (s_where.ignore(pos, expected)) - // { - // if (!exp_elem.parse(pos, where_expression, expected)) - // return false; - // } - // If group by is specified, AggregatingMergeTree engine is used, and the group by keys are implied to be order by keys if (s_group_by.ignore(pos, expected)) { @@ -70,6 +54,7 @@ bool ParserProjectionSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & .parse(pos, group_expression_list, expected)) return false; } + if (s_order_by.ignore(pos, expected)) { ASTPtr expr_list; @@ -92,7 +77,6 @@ bool ParserProjectionSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & select_query->setExpression(ASTProjectionSelectQuery::Expression::WITH, std::move(with_expression_list)); select_query->setExpression(ASTProjectionSelectQuery::Expression::SELECT, std::move(select_expression_list)); - // select_query->setExpression(ASTProjectionSelectQuery::Expression::WHERE, std::move(where_expression)); select_query->setExpression(ASTProjectionSelectQuery::Expression::GROUP_BY, std::move(group_expression_list)); select_query->setExpression(ASTProjectionSelectQuery::Expression::ORDER_BY, std::move(order_expression)); return true; diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index a9a41b1ac1f..d3fec057c2a 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -397,9 +397,6 @@ "01475_read_subcolumns_storages", "01674_clickhouse_client_query_param_cte", "01666_merge_tree_max_query_limit", - "01710_projections", - "01710_normal_projections", - "01710_aggregate_projections", "01786_explain_merge_tree", "01666_merge_tree_max_query_limit", "01802_test_postgresql_protocol_with_row_policy", /// It cannot parse DROP ROW POLICY From 58f118daf045d8d37469e1f652b22a45f41b9c50 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 2 Jun 2021 15:55:20 +0000 Subject: [PATCH 154/154] done --- tests/queries/0_stateless/01504_rocksdb.sql | 59 ++++++++++----------- tests/queries/0_stateless/01686_rocksdb.sql | 29 +++++----- 2 files changed, 43 insertions(+), 45 deletions(-) diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index 0e1917a95f2..9e30263c6b3 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -1,47 +1,46 @@ -DROP TABLE IF EXISTS test; +DROP TABLE IF EXISTS 01504_test; -CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB; -- { serverError 36 } -CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key2); -- { serverError 47 } -CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key, value); -- { serverError 36 } -CREATE TABLE test (key Tuple(String, UInt32), value UInt64) Engine=EmbeddedRocksDB PRIMARY KEY(key); +CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB; -- { serverError 36 } +CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key2); -- { serverError 47 } +CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key, value); -- { serverError 36 } +CREATE TABLE 01504_test (key Tuple(String, UInt32), value UInt64) Engine=EmbeddedRocksDB PRIMARY KEY(key); -DROP TABLE IF EXISTS test; -CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key); +DROP TABLE IF EXISTS 01504_test; +CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key); -INSERT INTO test SELECT '1_1', number FROM numbers(10000); -SELECT COUNT(1) == 1 FROM test; +INSERT INTO 01504_test SELECT '1_1', number FROM numbers(10000); +SELECT COUNT(1) == 1 FROM 01504_test; -INSERT INTO test SELECT concat(toString(number), '_1'), number FROM numbers(10000); -SELECT COUNT(1) == 10000 FROM test; -SELECT uniqExact(key) == 32 FROM (SELECT * FROM test LIMIT 32 SETTINGS max_block_size = 1); -SELECT SUM(value) == 1 + 99 + 900 FROM test WHERE key IN ('1_1', '99_1', '900_1'); +INSERT INTO 01504_test SELECT concat(toString(number), '_1'), number FROM numbers(10000); +SELECT COUNT(1) == 10000 FROM 01504_test; +SELECT uniqExact(key) == 32 FROM (SELECT * FROM 01504_test LIMIT 32 SETTINGS max_block_size = 1); +SELECT SUM(value) == 1 + 99 + 900 FROM 01504_test WHERE key IN ('1_1', '99_1', '900_1'); -DROP TABLE IF EXISTS test; -DROP TABLE IF EXISTS test_memory; +DROP TABLE IF EXISTS 01504_test; +DROP TABLE IF EXISTS 01504_test_memory; -CREATE TABLE test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksDB PRIMARY KEY(k); -CREATE TABLE test_memory AS test Engine = Memory; +CREATE TABLE 01504_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksDB PRIMARY KEY(k); +CREATE TABLE 01504_test_memory AS 01504_test Engine = Memory; -INSERT INTO test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; +INSERT INTO 01504_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; -INSERT INTO test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; +INSERT INTO 01504_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; -SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test_memory) B USING a ORDER BY a; +SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 01504_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 01504_test_memory) B USING a ORDER BY a; CREATE TEMPORARY TABLE keys AS SELECT * FROM numbers(1000); SET max_rows_to_read = 2; -SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3) OR k IN (1) OR k IN (3, 1) OR k IN [1] OR k IN [1, 3] ; -SELECT k == 4 FROM test WHERE k = 4 OR k IN [4] OR k in (4, 10000001, 10000002) AND value > 0; -SELECT k == 4 FROM test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4); -SELECT k, value FROM test WHERE k = 0 OR value > 0; -- { serverError 158 } -SELECT k, value FROM test WHERE k = 0 AND k IN (1, 3) OR k > 8; -- { serverError 158 } +SELECT dummy == (1,1.2) FROM 01504_test WHERE k IN (1, 3) OR k IN (1) OR k IN (3, 1) OR k IN [1] OR k IN [1, 3] ; +SELECT k == 4 FROM 01504_test WHERE k = 4 OR k IN [4] OR k in (4, 10000001, 10000002) AND value > 0; +SELECT k == 4 FROM 01504_test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4); +SELECT k, value FROM 01504_test WHERE k = 0 OR value > 0; -- { serverError 158 } +SELECT k, value FROM 01504_test WHERE k = 0 AND k IN (1, 3) OR k > 8; -- { serverError 158 } -TRUNCATE TABLE test; -SELECT 0 == COUNT(1) FROM test; - -DROP TABLE IF EXISTS test; -DROP TABLE IF EXISTS test_memory; +TRUNCATE TABLE 01504_test; +SELECT 0 == COUNT(1) FROM 01504_test; +DROP TABLE IF EXISTS 01504_test; +DROP TABLE IF EXISTS 01504_test_memory; diff --git a/tests/queries/0_stateless/01686_rocksdb.sql b/tests/queries/0_stateless/01686_rocksdb.sql index c9b133acff3..9a8662453c1 100644 --- a/tests/queries/0_stateless/01686_rocksdb.sql +++ b/tests/queries/0_stateless/01686_rocksdb.sql @@ -1,27 +1,26 @@ -DROP TABLE IF EXISTS test; +DROP TABLE IF EXISTS 01686_test; -CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); +CREATE TABLE 01686_test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); -INSERT INTO test SELECT number, format('Hello, world ({})', toString(number)) FROM numbers(10000); +INSERT INTO 01686_test SELECT number, format('Hello, world ({})', toString(number)) FROM numbers(10000); -SELECT * FROM test WHERE key = 123; +SELECT * FROM 01686_test WHERE key = 123; SELECT '--'; -SELECT * FROM test WHERE key = -123; +SELECT * FROM 01686_test WHERE key = -123; SELECT '--'; -SELECT * FROM test WHERE key = 123 OR key = 4567 ORDER BY key; +SELECT * FROM 01686_test WHERE key = 123 OR key = 4567 ORDER BY key; SELECT '--'; -SELECT * FROM test WHERE key = NULL; +SELECT * FROM 01686_test WHERE key = NULL; SELECT '--'; -SELECT * FROM test WHERE key = NULL OR key = 0; +SELECT * FROM 01686_test WHERE key = NULL OR key = 0; SELECT '--'; -SELECT * FROM test WHERE key IN (123, 456, -123) ORDER BY key; +SELECT * FROM 01686_test WHERE key IN (123, 456, -123) ORDER BY key; SELECT '--'; -SELECT * FROM test WHERE key = 'Hello'; -- { serverError 53 } +SELECT * FROM 01686_test WHERE key = 'Hello'; -- { serverError 53 } -DETACH TABLE test NO DELAY; -ATTACH TABLE test; +DETACH TABLE 01686_test NO DELAY; +ATTACH TABLE 01686_test; -SELECT * FROM test WHERE key IN (99, 999, 9999, -123) ORDER BY key; - -DROP TABLE IF EXISTS test; +SELECT * FROM 01686_test WHERE key IN (99, 999, 9999, -123) ORDER BY key; +DROP TABLE IF EXISTS 01686_test;